HDFS-12387. Ozone: Support Ratis as a first class replication mechanism. Contributed by Anu Engineer.

This commit is contained in:
Anu Engineer 2017-10-04 10:24:14 -07:00 committed by Owen O'Malley
parent 0b3270a8de
commit c17521b1bd
50 changed files with 2147 additions and 935 deletions

View File

@ -93,9 +93,9 @@ public final class OzoneConfigKeys {
"ozone.container.cache.size";
public static final int OZONE_CONTAINER_CACHE_DEFAULT = 1024;
public static final String OZONE_SCM_BLOCK_SIZE_KEY =
"ozone.scm.block.size";
public static final long OZONE_SCM_BLOCK_SIZE_DEFAULT = 256 * OzoneConsts.MB;
public static final String OZONE_SCM_BLOCK_SIZE_IN_MB =
"ozone.scm.block.size.in.mb";
public static final long OZONE_SCM_BLOCK_SIZE_DEFAULT = 256;
/**
* Ozone administrator users delimited by comma.

View File

@ -26,6 +26,7 @@
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import java.io.IOException;
import java.util.Iterator;
@ -195,10 +196,12 @@ public void setVersioning(Boolean newVersioning) throws IOException {
* @return OzoneOutputStream to which the data has to be written.
* @throws IOException
*/
public OzoneOutputStream createKey(String key, long size)throws IOException {
public OzoneOutputStream createKey(String key, long size, OzoneProtos
.ReplicationType type, OzoneProtos.ReplicationFactor factor)
throws IOException {
Preconditions.checkNotNull(proxy, "Client proxy is not set.");
Preconditions.checkNotNull(key);
return proxy.createKey(volumeName, name, key, size);
return proxy.createKey(volumeName, name, key, size, type, factor);
}
/**

View File

@ -276,10 +276,10 @@ public static ChunkGroupOutputStream getFromKsmKeyInfo(
// case we should revert the above allocateKey to KSM.
// check index as sanity check
int index = 0;
String containerKey;
String blockID;
ChunkGroupOutputStream groupOutputStream = new ChunkGroupOutputStream();
for (KsmKeyLocationInfo subKeyInfo : keyInfo.getKeyLocationList()) {
containerKey = subKeyInfo.getBlockID();
blockID = subKeyInfo.getBlockID();
Preconditions.checkArgument(index++ == subKeyInfo.getIndex());
String containerName = subKeyInfo.getContainerName();
@ -290,8 +290,13 @@ public static ChunkGroupOutputStream getFromKsmKeyInfo(
// create container if needed
if (subKeyInfo.getShouldCreateContainer()) {
try {
// Block manager sets the container creation stage begin.
storageContainerLocationClient.notifyObjectCreationStage(
NotifyObjectCreationStageRequestProto.Type.container,
containerName,
NotifyObjectCreationStageRequestProto.Stage.begin);
ContainerProtocolCalls.createContainer(xceiverClient, requestId);
storageContainerLocationClient.notifyObjectCreationStage(
NotifyObjectCreationStageRequestProto.Type.container,
containerName,
@ -307,7 +312,7 @@ public static ChunkGroupOutputStream getFromKsmKeyInfo(
}
}
groupOutputStream.addStream(containerKey, keyInfo.getKeyName(),
groupOutputStream.addStream(blockID, keyInfo.getKeyName(),
xceiverClientManager, xceiverClient, requestId, chunkSize,
subKeyInfo.getLength());
}

View File

@ -28,6 +28,7 @@
import org.apache.hadoop.ozone.client.VolumeArgs;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import java.io.IOException;
import java.util.List;
@ -257,7 +258,9 @@ List<OzoneBucket> listBuckets(String volumeName, String bucketPrefix,
*
*/
OzoneOutputStream createKey(String volumeName, String bucketName,
String keyName, long size)
String keyName, long size,
OzoneProtos.ReplicationType type,
OzoneProtos.ReplicationFactor factor)
throws IOException;
/**

View File

@ -30,6 +30,7 @@
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import java.io.IOException;
import java.util.List;
@ -171,10 +172,20 @@ public List<OzoneBucket> listBuckets(String volumeName, String bucketPrefix,
throw new UnsupportedOperationException("Not yet implemented.");
}
/**
* Writes a key in an existing bucket.
*
* @param volumeName Name of the Volume
* @param bucketName Name of the Bucket
* @param keyName Name of the Key
* @param size Size of the data
* @param type
* @param factor @return {@link OzoneOutputStream}
*/
@Override
public OzoneOutputStream createKey(
String volumeName, String bucketName, String keyName, long size)
throws IOException {
public OzoneOutputStream createKey(String volumeName, String bucketName,
String keyName, long size, OzoneProtos.ReplicationType type,
OzoneProtos.ReplicationFactor factor) throws IOException {
throw new UnsupportedOperationException("Not yet implemented.");
}

View File

@ -51,6 +51,7 @@
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.client.OzoneClientUtils;
import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.apache.hadoop.scm.XceiverClientManager;
@ -430,7 +431,8 @@ public List<OzoneBucket> listBuckets(String volumeName, String bucketPrefix,
@Override
public OzoneOutputStream createKey(
String volumeName, String bucketName, String keyName, long size)
String volumeName, String bucketName, String keyName, long size,
OzoneProtos.ReplicationType type, OzoneProtos.ReplicationFactor factor)
throws IOException {
String requestId = UUID.randomUUID().toString();
KsmKeyArgs keyArgs = new KsmKeyArgs.Builder()
@ -438,6 +440,8 @@ public OzoneOutputStream createKey(
.setBucketName(bucketName)
.setKeyName(keyName)
.setDataSize(size)
.setType(type)
.setFactor(factor)
.build();
KsmKeyInfo keyInfo = keySpaceManagerClient.allocateKey(keyArgs);

View File

@ -16,6 +16,8 @@
* limitations under the License.
*/
package org.apache.hadoop.ozone.ksm.helpers;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
/**
* Args for key. Client use this to specify key's attributes on key creation
@ -25,15 +27,26 @@ public final class KsmKeyArgs {
private final String volumeName;
private final String bucketName;
private final String keyName;
private final long dataSize;
private final ReplicationType type;
private final ReplicationFactor factor;
private KsmKeyArgs(String volumeName, String bucketName, String keyName,
long dataSize) {
long dataSize, ReplicationType type, ReplicationFactor factor) {
this.volumeName = volumeName;
this.bucketName = bucketName;
this.keyName = keyName;
this.dataSize = dataSize;
this.type = type;
this.factor = factor;
}
public ReplicationType getType() {
return type;
}
public ReplicationFactor getFactor() {
return factor;
}
public String getVolumeName() {
@ -60,6 +73,9 @@ public static class Builder {
private String bucketName;
private String keyName;
private long dataSize;
private ReplicationType type;
private ReplicationFactor factor;
public Builder setVolumeName(String volume) {
this.volumeName = volume;
@ -81,8 +97,19 @@ public Builder setDataSize(long size) {
return this;
}
public Builder setType(ReplicationType type) {
this.type = type;
return this;
}
public Builder setFactor(ReplicationFactor factor) {
this.factor = factor;
return this;
}
public KsmKeyArgs build() {
return new KsmKeyArgs(volumeName, bucketName, keyName, dataSize);
return new KsmKeyArgs(volumeName, bucketName, keyName, dataSize,
type, factor);
}
}
}

View File

@ -511,7 +511,10 @@ public KsmKeyInfo allocateKey(KsmKeyArgs args) throws IOException {
.setVolumeName(args.getVolumeName())
.setBucketName(args.getBucketName())
.setKeyName(args.getKeyName())
.setDataSize(args.getDataSize()).build();
.setDataSize(args.getDataSize())
.setType(args.getType())
.setFactor(args.getFactor())
.build();
req.setKeyArgs(keyArgs);
final LocateKeyResponse resp;

View File

@ -186,7 +186,7 @@ public final class ScmConfigKeys {
public static final String OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE =
"ozone.scm.container.provision_batch_size";
public static final int OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT = 5;
public static final int OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT = 20;
public static final String OZONE_SCM_CONTAINER_DELETION_CHOOSING_POLICY =
"ozone.scm.container.deletion-choosing.policy";

View File

@ -34,11 +34,13 @@
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.List;
/**
* A Client for the storageContainer protocol.
@ -154,4 +156,27 @@ public ContainerProtos.ContainerCommandResponseProto sendCommand(
channelFuture.channel().pipeline().get(XceiverClientHandler.class);
return handler.sendCommandAsync(request);
}
/**
* Create a pipeline.
*
* @param pipelineID - Name of the pipeline.
* @param datanodes - Datanodes
*/
@Override
public void createPipeline(String pipelineID, List<DatanodeID> datanodes)
throws IOException {
// For stand alone pipeline, there is no notion called setup pipeline.
return;
}
/**
* Returns pipeline Type.
*
* @return - Stand Alone as the type.
*/
@Override
public OzoneProtos.ReplicationType getPipelineType() {
return OzoneProtos.ReplicationType.STAND_ALONE;
}
}

View File

@ -42,6 +42,8 @@
.SCM_CONTAINER_CLIENT_MAX_SIZE_KEY;
import static org.apache.hadoop.scm.ScmConfigKeys
.SCM_CONTAINER_CLIENT_MAX_SIZE_DEFAULT;
import static org.apache.hadoop.ozone.protocol.proto.OzoneProtos
.ReplicationType.RATIS;
/**
* XceiverClientManager is responsible for the lifecycle of XceiverClient
@ -146,7 +148,7 @@ private XceiverClientSpi getClient(Pipeline pipeline)
new Callable<XceiverClientSpi>() {
@Override
public XceiverClientSpi call() throws Exception {
XceiverClientSpi client = useRatis ?
XceiverClientSpi client = pipeline.getType() == RATIS ?
XceiverClientRatis.newXceiverClientRatis(pipeline, conf)
: new XceiverClient(pipeline, conf);
client.connect();

View File

@ -20,14 +20,15 @@
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
.ContainerCommandRequestProto;
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
.ContainerCommandResponseProto;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerCommandResponseProto;
import org.apache.ratis.RatisHelper;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.ratis.client.RaftClient;
import org.apache.ratis.protocol.RaftClientReply;
import org.apache.ratis.protocol.RaftPeer;
import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.rpc.SupportedRpcType;
import org.apache.ratis.shaded.com.google.protobuf.ByteString;
@ -36,6 +37,7 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
@ -68,6 +70,70 @@ private XceiverClientRatis(Pipeline pipeline, RpcType rpcType) {
this.rpcType = rpcType;
}
/**
* {@inheritDoc}
*/
public void createPipeline(String clusterId, List<DatanodeID> datanodes)
throws IOException {
final RaftPeer[] newPeers = datanodes.stream().map(RatisHelper::toRaftPeer)
.toArray(RaftPeer[]::new);
reinitialize(datanodes, newPeers);
}
/**
* Returns Ratis as pipeline Type.
* @return - Ratis
*/
@Override
public OzoneProtos.ReplicationType getPipelineType() {
return OzoneProtos.ReplicationType.RATIS;
}
private void reinitialize(List<DatanodeID> datanodes, RaftPeer[] newPeers)
throws IOException {
if (datanodes.isEmpty()) {
return;
}
IOException exception = null;
for (DatanodeID d : datanodes) {
try {
reinitialize(d, newPeers);
} catch (IOException ioe) {
if (exception == null) {
exception = new IOException(
"Failed to reinitialize some of the RaftPeer(s)", ioe);
} else {
exception.addSuppressed(ioe);
}
}
}
if (exception != null) {
throw exception;
}
}
/**
* Adds a new peers to the Ratis Ring.
* @param datanode - new datanode
* @param newPeers - Raft machines
* @throws IOException - on Failure.
*/
private void reinitialize(DatanodeID datanode, RaftPeer[] newPeers)
throws IOException {
final RaftPeer p = RatisHelper.toRaftPeer(datanode);
try (RaftClient client = RatisHelper.newRaftClient(rpcType, p)) {
client.reinitialize(newPeers, p.getId());
} catch (IOException ioe) {
LOG.error("Failed to reinitialize RaftPeer:{} datanode: {} ",
p, datanode, ioe);
throw new IOException("Failed to reinitialize RaftPeer " + p
+ "(datanode=" + datanode + ")", ioe);
}
}
@Override
public Pipeline getPipeline() {
return pipeline;

View File

@ -19,14 +19,17 @@
package org.apache.hadoop.scm;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hdfs.ozone.protocol.proto
.ContainerProtos.ContainerCommandRequestProto;
import org.apache.hadoop.hdfs.ozone.protocol.proto
.ContainerProtos.ContainerCommandResponseProto;
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
.ContainerCommandRequestProto;
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos
.ContainerCommandResponseProto;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicInteger;
@ -99,12 +102,28 @@ public abstract ContainerCommandResponseProto sendCommand(
/**
* Sends a given command to server gets a waitable future back.
*
* @param request Request
* @return Response to the command
* @throws IOException
*/
public abstract CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(
ContainerCommandRequestProto request)
public abstract CompletableFuture<ContainerCommandResponseProto>
sendCommandAsync(ContainerCommandRequestProto request)
throws IOException, ExecutionException, InterruptedException;
/**
* Create a pipeline.
*
* @param pipelineID - Name of the pipeline.
* @param datanodes - Datanodes
*/
public abstract void createPipeline(String pipelineID,
List<DatanodeID> datanodes) throws IOException;
/**
* Returns pipeline Type.
*
* @return - {Stand_Alone, Ratis or Chained}
*/
public abstract OzoneProtos.ReplicationType getPipelineType();
}

View File

@ -17,14 +17,16 @@
*/
package org.apache.hadoop.scm.client;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ContainerData;
import org.apache.hadoop.hdfs.ozone.protocol.proto.ContainerProtos.ReadContainerResponseProto;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.protocol.proto.StorageContainerLocationProtocolProtos.NotifyObjectCreationStageRequestProto;
import org.apache.hadoop.scm.XceiverClientSpi;
import org.apache.hadoop.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.scm.XceiverClientManager;
import org.apache.hadoop.scm.XceiverClientSpi;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.scm.protocolPB
.StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.scm.storage.ContainerProtocolCalls;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -34,6 +36,9 @@
import java.util.List;
import java.util.UUID;
import static org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleState.ALLOCATED;
import static org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleState.OPEN;
/**
* This class provides the client-facing APIs of container operations.
*/
@ -84,23 +89,18 @@ public Pipeline createContainer(String containerId)
storageContainerLocationClient.allocateContainer(
xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerId);
client = xceiverClientManager.acquireClient(pipeline);
String traceID = UUID.randomUUID().toString();
storageContainerLocationClient.notifyObjectCreationStage(
NotifyObjectCreationStageRequestProto.Type.container,
containerId,
NotifyObjectCreationStageRequestProto.Stage.begin);
ContainerProtocolCalls.createContainer(client, traceID);
if (LOG.isDebugEnabled()) {
LOG.debug("Created container " + containerId
+ " leader:" + pipeline.getLeader()
+ " machines:" + pipeline.getMachines());
// Allocated State means that SCM has allocated this pipeline in its
// namespace. The client needs to create the pipeline on the machines
// which was choosen by the SCM.
Preconditions.checkState(pipeline.getLifeCycleState() == ALLOCATED ||
pipeline.getLifeCycleState() == OPEN, "Unexpected pipeline state");
if (pipeline.getLifeCycleState() == ALLOCATED) {
createPipeline(client, pipeline);
}
storageContainerLocationClient.notifyObjectCreationStage(
NotifyObjectCreationStageRequestProto.Type.container,
containerId,
NotifyObjectCreationStageRequestProto.Stage.complete);
// TODO : Container Client State needs to be updated.
createContainer(containerId, client, pipeline);
return pipeline;
} finally {
if (client != null) {
@ -109,6 +109,76 @@ public Pipeline createContainer(String containerId)
}
}
/**
* Create a container over pipeline specified by the SCM.
*
* @param containerId - Container ID
* @param client - Client to communicate with Datanodes
* @param pipeline - A pipeline that is already created.
* @throws IOException
*/
public void createContainer(String containerId, XceiverClientSpi client,
Pipeline pipeline) throws IOException {
String traceID = UUID.randomUUID().toString();
storageContainerLocationClient.notifyObjectCreationStage(
NotifyObjectCreationStageRequestProto.Type.container,
containerId,
NotifyObjectCreationStageRequestProto.Stage.begin);
ContainerProtocolCalls.createContainer(client, traceID);
storageContainerLocationClient.notifyObjectCreationStage(
NotifyObjectCreationStageRequestProto.Type.container,
containerId,
NotifyObjectCreationStageRequestProto.Stage.complete);
// Let us log this info after we let SCM know that we have completed the
// creation state.
if (LOG.isDebugEnabled()) {
LOG.debug("Created container " + containerId
+ " leader:" + pipeline.getLeader()
+ " machines:" + pipeline.getMachines());
}
}
/**
* Creates a pipeline over the machines choosen by the SCM.
*
* @param client - Client
* @param pipeline - pipeline to be createdon Datanodes.
* @throws IOException
*/
private void createPipeline(XceiverClientSpi client, Pipeline pipeline)
throws IOException {
Preconditions.checkNotNull(pipeline.getPipelineName(), "Pipeline " +
"name cannot be null when client create flag is set.");
// Pipeline creation is a three step process.
//
// 1. Notify SCM that this client is doing a create pipeline on
// datanodes.
//
// 2. Talk to Datanodes to create the pipeline.
//
// 3. update SCM that pipeline creation was successful.
storageContainerLocationClient.notifyObjectCreationStage(
NotifyObjectCreationStageRequestProto.Type.pipeline,
pipeline.getPipelineName(),
NotifyObjectCreationStageRequestProto.Stage.begin);
client.createPipeline(pipeline.getPipelineName(),
pipeline.getMachines());
storageContainerLocationClient.notifyObjectCreationStage(
NotifyObjectCreationStageRequestProto.Type.pipeline,
pipeline.getPipelineName(),
NotifyObjectCreationStageRequestProto.Stage.complete);
// TODO : Should we change the state on the client side ??
// That makes sense, but it is not needed for the client to work.
LOG.debug("Pipeline creation successful. Pipeline: {}",
pipeline.toString());
}
/**
* @inheritDoc
*/
@ -122,24 +192,18 @@ public Pipeline createContainer(OzoneProtos.ReplicationType type,
Pipeline pipeline =
storageContainerLocationClient.allocateContainer(type, factor,
containerId);
client = xceiverClientManager.acquireClient(pipeline);
// Allocated State means that SCM has allocated this pipeline in its
// namespace. The client needs to create the pipeline on the machines
// which was choosen by the SCM.
if (pipeline.getLifeCycleState() == ALLOCATED) {
createPipeline(client, pipeline);
}
// connect to pipeline leader and allocate container on leader datanode.
client = xceiverClientManager.acquireClient(pipeline);
String traceID = UUID.randomUUID().toString();
storageContainerLocationClient.notifyObjectCreationStage(
NotifyObjectCreationStageRequestProto.Type.container,
containerId,
NotifyObjectCreationStageRequestProto.Stage.begin);
ContainerProtocolCalls.createContainer(client, traceID);
LOG.info("Created container " + containerId +
" leader:" + pipeline.getLeader() +
" machines:" + pipeline.getMachines() +
" replication factor:" + factor);
storageContainerLocationClient.notifyObjectCreationStage(
NotifyObjectCreationStageRequestProto.Type.container,
containerId,
NotifyObjectCreationStageRequestProto.Stage.complete);
createContainer(containerId, client, pipeline);
return pipeline;
} finally {
if (client != null) {
@ -192,10 +256,12 @@ public void deleteContainer(Pipeline pipeline, boolean force)
ContainerProtocolCalls.deleteContainer(client, force, traceID);
storageContainerLocationClient
.deleteContainer(pipeline.getContainerName());
LOG.info("Deleted container {}, leader: {}, machines: {} ",
pipeline.getContainerName(),
pipeline.getLeader(),
pipeline.getMachines());
if (LOG.isDebugEnabled()) {
LOG.debug("Deleted container {}, leader: {}, machines: {} ",
pipeline.getContainerName(),
pipeline.getLeader(),
pipeline.getMachines());
}
} finally {
if (client != null) {
xceiverClientManager.releaseClient(client);
@ -231,10 +297,12 @@ public ContainerData readContainer(Pipeline pipeline) throws IOException {
ReadContainerResponseProto response =
ContainerProtocolCalls.readContainer(client,
pipeline.getContainerName(), traceID);
LOG.info("Read container {}, leader: {}, machines: {} ",
pipeline.getContainerName(),
pipeline.getLeader(),
pipeline.getMachines());
if (LOG.isDebugEnabled()) {
LOG.debug("Read container {}, leader: {}, machines: {} ",
pipeline.getContainerName(),
pipeline.getLeader(),
pipeline.getMachines());
}
return response.getContainerData();
} finally {
if (client != null) {

View File

@ -1,4 +1,4 @@
/*
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@ -18,15 +18,26 @@
package org.apache.hadoop.scm.container.common.helpers;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.hadoop.util.Time;
import java.io.Serializable;
import java.util.Comparator;
/**
* Class wraps container + allocated info for containers managed by block svc.
* Manages Block Information inside a container.
*/
public class BlockContainerInfo extends ContainerInfo{
public class BlockContainerInfo extends ContainerInfo
implements Comparator<BlockContainerInfo>,
Comparable<BlockContainerInfo>, Serializable {
private long allocated;
private long lastUsed; // last used time
public BlockContainerInfo(ContainerInfo container, long used) {
super(container);
this.allocated = used;
this.lastUsed = Time.monotonicNow();
}
public long addAllocated(long size) {
@ -42,4 +53,87 @@ public long subtractAllocated(long size) {
public long getAllocated() {
return this.allocated;
}
/**
* Gets the last used time from SCM's perspective.
* @return time in milliseconds.
*/
public long getLastUsed() {
return lastUsed;
}
/**
* Sets the last used time from SCM's perspective.
* @param lastUsed time in milliseconds.
*/
public void setLastUsed(long lastUsed) {
this.lastUsed = lastUsed;
}
/**
* Compares its two arguments for order. Returns a negative integer, zero, or
* a positive integer as the first argument is less than, equal to, or greater
* than the second.<p>
*
* @param o1 the first object to be compared.
* @param o2 the second object to be compared.
* @return a negative integer, zero, or a positive integer as the first
* argument is less than, equal to, or greater than the second.
* @throws NullPointerException if an argument is null and this comparator
* does not permit null arguments
* @throws ClassCastException if the arguments' types prevent them from
* being compared by this comparator.
*/
@Override
public int compare(BlockContainerInfo o1, BlockContainerInfo o2) {
return Long.compare(o1.getLastUsed(), o2.getLastUsed());
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
return new EqualsBuilder()
.appendSuper(super.equals(o))
.isEquals();
}
@Override
public int hashCode() {
return new HashCodeBuilder(17, 2017)
.appendSuper(super.hashCode())
.toHashCode();
}
@Override
public String toString() {
return "BlockContainerInfo{" +
"allocated=" + allocated +
", lastUsed=" + lastUsed +
", ContainerInfo=" + super.toString() + '}';
}
/**
* Compares this object with the specified object for order. Returns a
* negative integer, zero, or a positive integer as this object is less than,
* equal to, or greater than the specified object.
*
* @param o the object to be compared.
* @return a negative integer, zero, or a positive integer as this object is
* less than, equal to, or greater than the specified object.
* @throws NullPointerException if the specified object is null
* @throws ClassCastException if the specified object's type prevents it
* from being compared to this object.
*/
@Override
public int compareTo(BlockContainerInfo o) {
return this.compare(this, o);
}
}

View File

@ -18,33 +18,71 @@
package org.apache.hadoop.scm.container.common.helpers;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.util.Time;
/**
* Class wraps ozone container info.
*/
/** Class wraps ozone container info. */
public class ContainerInfo {
private OzoneProtos.LifeCycleState state;
private Pipeline pipeline;
// The wall-clock ms since the epoch at which the current state enters.
private long stateEnterTime;
private OzoneProtos.Owner owner;
private String containerName;
ContainerInfo(OzoneProtos.LifeCycleState state, Pipeline pipeline,
long stateEnterTime) {
ContainerInfo(
final String containerName,
OzoneProtos.LifeCycleState state,
Pipeline pipeline,
long stateEnterTime,
OzoneProtos.Owner owner) {
this.containerName = containerName;
this.pipeline = pipeline;
this.state = state;
this.stateEnterTime = stateEnterTime;
this.owner = owner;
}
public ContainerInfo(ContainerInfo container) {
this.pipeline = container.getPipeline();
this.state = container.getState();
this.stateEnterTime = container.getStateEnterTime();
this.owner = container.getOwner();
}
/**
* Needed for serialization findbugs.
*/
public ContainerInfo() {
}
public static ContainerInfo fromProtobuf(OzoneProtos.SCMContainerInfo info) {
ContainerInfo.Builder builder = new ContainerInfo.Builder();
builder.setPipeline(Pipeline.getFromProtoBuf(info.getPipeline()));
builder.setState(info.getState());
builder.setStateEnterTime(info.getStateEnterTime());
builder.setOwner(info.getOwner());
builder.setContainerName(info.getContainerName());
return builder.build();
}
public String getContainerName() {
return containerName;
}
public void setContainerName(String containerName) {
this.containerName = containerName;
}
public OzoneProtos.LifeCycleState getState() {
return state;
}
/**
* Update the current container state and state enter time to now.
*
* @param state
*/
public void setState(OzoneProtos.LifeCycleState state) {
@ -52,10 +90,6 @@ public void setState(OzoneProtos.LifeCycleState state) {
this.stateEnterTime = Time.monotonicNow();
}
public OzoneProtos.LifeCycleState getState() {
return state;
}
public long getStateEnterTime() {
return stateEnterTime;
}
@ -70,16 +104,68 @@ public OzoneProtos.SCMContainerInfo getProtobuf() {
builder.setPipeline(getPipeline().getProtobufMessage());
builder.setState(state);
builder.setStateEnterTime(stateEnterTime);
if (getOwner() != null) {
builder.setOwner(getOwner());
}
builder.setContainerName(getContainerName());
return builder.build();
}
public static ContainerInfo fromProtobuf(
OzoneProtos.SCMContainerInfo info) {
ContainerInfo.Builder builder = new ContainerInfo.Builder();
builder.setPipeline(Pipeline.getFromProtoBuf(info.getPipeline()));
builder.setState(info.getState());
builder.setStateEnterTime(info.getStateEnterTime());
return builder.build();
public OzoneProtos.Owner getOwner() {
return owner;
}
public void setOwner(OzoneProtos.Owner owner) {
this.owner = owner;
}
@Override
public String toString() {
return "ContainerInfo{"
+ "state=" + state
+ ", pipeline=" + pipeline
+ ", stateEnterTime=" + stateEnterTime
+ ", owner=" + owner
+ ", containerName='" + containerName
+ '}';
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
ContainerInfo that = (ContainerInfo) o;
return new EqualsBuilder()
.append(state, that.state)
.append(pipeline.getContainerName(), that.pipeline.getContainerName())
// TODO : Fix this later. If we add these factors some tests fail.
// So Commenting this to continue and will enforce this with
// Changes in pipeline where we remove Container Name to
// SCMContainerinfo from Pipline.
// .append(pipeline.getFactor(), that.pipeline.getFactor())
// .append(pipeline.getType(), that.pipeline.getType())
.append(owner, that.owner)
.isEquals();
}
@Override
public int hashCode() {
return new HashCodeBuilder(11, 811)
.append(state)
.append(pipeline.getContainerName())
.append(pipeline.getFactor())
.append(pipeline.getType())
.append(owner)
.toHashCode();
}
/** Builder class for ContainerInfo. */
@ -87,6 +173,8 @@ public static class Builder {
private OzoneProtos.LifeCycleState state;
private Pipeline pipeline;
private long stateEnterTime;
private OzoneProtos.Owner owner;
private String containerName;
public Builder setState(OzoneProtos.LifeCycleState lifeCycleState) {
this.state = lifeCycleState;
@ -103,8 +191,19 @@ public Builder setStateEnterTime(long stateEnterTime) {
return this;
}
public Builder setOwner(OzoneProtos.Owner owner) {
this.owner = owner;
return this;
}
public Builder setContainerName(String containerName) {
this.containerName = containerName;
return this;
}
public ContainerInfo build() {
return new ContainerInfo(state, pipeline, stateEnterTime);
return new
ContainerInfo(containerName, state, pipeline, stateEnterTime, owner);
}
}
}

View File

@ -43,16 +43,6 @@
* A pipeline represents the group of machines over which a container lives.
*/
public class Pipeline {
private String containerName;
private String leaderID;
private Map<String, DatanodeID> datanodes;
/**
* Allows you to maintain private data on pipelines.
* This is not serialized via protobuf, just allows us to maintain some
* private data.
*/
private byte[] data;
static final String PIPELINE_INFO = "PIPELINE_INFO_FILTER";
private static final ObjectWriter WRITER;
@ -69,10 +59,18 @@ public class Pipeline {
WRITER = mapper.writer(filters);
}
@JsonFilter(PIPELINE_INFO)
class MixIn {
}
private String containerName;
private String leaderID;
private Map<String, DatanodeID> datanodes;
private OzoneProtos.LifeCycleState lifeCycleState;
private OzoneProtos.ReplicationType type;
private OzoneProtos.ReplicationFactor factor;
private String pipelineName;
/**
* Allows you to maintain private data on pipelines. This is not serialized
* via protobuf, just allows us to maintain some private data.
*/
private byte[] data;
/**
* Constructs a new pipeline data structure.
*
@ -98,9 +96,22 @@ public static Pipeline getFromProtoBuf(OzoneProtos.Pipeline pipeline) {
}
newPipeline.setContainerName(pipeline.getContainerName());
newPipeline.setLifeCycleState(pipeline.getState());
newPipeline.setType(pipeline.getType());
newPipeline.setFactor(pipeline.getFactor());
if (pipeline.hasPipelineName()) {
newPipeline.setPipelineName(pipeline.getPipelineName());
}
return newPipeline;
}
public OzoneProtos.ReplicationFactor getFactor() {
return factor;
}
public void setFactor(OzoneProtos.ReplicationFactor factor) {
this.factor = factor;
}
/**
* Adds a member to the pipeline.
@ -167,6 +178,17 @@ public OzoneProtos.Pipeline getProtobufMessage() {
}
builder.setLeaderID(leaderID);
builder.setContainerName(this.containerName);
if (this.getLifeCycleState() != null) {
builder.setState(this.getLifeCycleState());
}
if (this.getType() != null) {
builder.setType(this.getType());
}
if (this.getFactor() != null) {
builder.setFactor(this.getFactor());
}
return builder.build();
}
@ -188,16 +210,6 @@ public void setContainerName(String containerName) {
this.containerName = containerName;
}
/**
* Set private data on pipeline.
* @param data -- private data.
*/
public void setData(byte[] data) {
if (data != null) {
this.data = Arrays.copyOf(data, data.length);
}
}
/**
* Returns private data that is set on this pipeline.
*
@ -211,13 +223,85 @@ public byte[] getData() {
}
}
/**
* Set private data on pipeline.
*
* @param data -- private data.
*/
public void setData(byte[] data) {
if (data != null) {
this.data = Arrays.copyOf(data, data.length);
}
}
/**
* Gets the State of the pipeline.
*
* @return - LifeCycleStates.
*/
public OzoneProtos.LifeCycleState getLifeCycleState() {
return lifeCycleState;
}
/**
* Sets the lifecycleState.
*
* @param lifeCycleStates - Enum
*/
public void setLifeCycleState(OzoneProtos.LifeCycleState lifeCycleStates) {
this.lifeCycleState = lifeCycleStates;
}
/**
* Gets the pipeline Name.
*
* @return - Name of the pipeline
*/
public String getPipelineName() {
return pipelineName;
}
/**
* Sets the pipeline name.
*
* @param pipelineName - Sets the name.
*/
public void setPipelineName(String pipelineName) {
this.pipelineName = pipelineName;
}
/**
* Returns the type.
*
* @return type - Standalone, Ratis, Chained.
*/
public OzoneProtos.ReplicationType getType() {
return type;
}
/**
* Sets the type of this pipeline.
*
* @param type - Standalone, Ratis, Chained.
*/
public void setType(OzoneProtos.ReplicationType type) {
this.type = type;
}
@Override
public String toString() {
final StringBuilder b = new StringBuilder(getClass().getSimpleName())
.append("[");
datanodes.keySet().stream()
.forEach(id -> b.append(id.endsWith(leaderID)? "*" + id : id));
.forEach(id -> b.append(id.endsWith(leaderID) ? "*" + id : id));
b.append("] container:").append(containerName);
b.append(" name:").append(getPipelineName());
if (getType() != null) {
b.append(" type:").append(getType().toString());
}
if (getLifeCycleState() != null) {
b.append(" State:").append(getLifeCycleState().toString());
}
return b.toString();
}
@ -230,4 +314,8 @@ public String toString() {
public String toJsonString() throws IOException {
return WRITER.writeValueAsString(this);
}
@JsonFilter(PIPELINE_INFO)
class MixIn {
}
}

View File

@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.scm.protocol;
import java.io.IOException;
@ -25,6 +24,8 @@
import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
import org.apache.hadoop.ozone.common.BlockGroup;
import org.apache.hadoop.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
/**
* ScmBlockLocationProtocol is used by an HDFS node to find the set of nodes
@ -41,8 +42,7 @@ public interface ScmBlockLocationProtocol {
* @return allocated blocks for each block key
* @throws IOException if there is any failure
*/
Set<AllocatedBlock> getBlockLocations(Set<String> keys)
throws IOException;
Set<AllocatedBlock> getBlockLocations(Set<String> keys) throws IOException;
/**
* Asks SCM where a block should be allocated. SCM responds with the
@ -51,7 +51,8 @@ Set<AllocatedBlock> getBlockLocations(Set<String> keys)
* @return allocated block accessing info (key, pipeline).
* @throws IOException
*/
AllocatedBlock allocateBlock(long size) throws IOException;
AllocatedBlock allocateBlock(long size, ReplicationType type,
ReplicationFactor factor) throws IOException;
/**
* Delete blocks for a set of object keys.
@ -59,9 +60,7 @@ Set<AllocatedBlock> getBlockLocations(Set<String> keys)
* @param keyBlocksInfoList Map of object key and its blocks.
* @return list of block deletion results.
* @throws IOException if there is any failure.
*
*/
List<DeleteBlockGroupResult> deleteKeyBlocks(
List<BlockGroup> keyBlocksInfoList) throws IOException;
List<DeleteBlockGroupResult>
deleteKeyBlocks(List<BlockGroup> keyBlocksInfoList) throws IOException;
}

View File

@ -27,23 +27,17 @@
import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
import org.apache.hadoop.ozone.common.BlockGroup;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.DeleteScmKeyBlocksRequestProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
.AllocateScmBlockRequestProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
.AllocateScmBlockResponseProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
.DeleteScmKeyBlocksResponseProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
.GetScmBlockLocationsRequestProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
.GetScmBlockLocationsResponseProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
.ScmLocatedBlockProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos
.KeyBlocks;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockRequestProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockResponseProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.DeleteScmKeyBlocksResponseProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.GetScmBlockLocationsRequestProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.GetScmBlockLocationsResponseProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.ScmLocatedBlockProto;
import org.apache.hadoop.ozone.protocol.proto.ScmBlockLocationProtocolProtos.KeyBlocks;
import org.apache.hadoop.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.scm.protocol.ScmBlockLocationProtocol;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import java.io.Closeable;
import java.io.IOException;
@ -121,12 +115,16 @@ public Set<AllocatedBlock> getBlockLocations(Set<String> keys)
* @throws IOException
*/
@Override
public AllocatedBlock allocateBlock(long size) throws IOException {
public AllocatedBlock allocateBlock(long size, OzoneProtos.ReplicationType
type, OzoneProtos.ReplicationFactor factor) throws IOException {
Preconditions.checkArgument(size > 0,
"block size must be greater than 0");
AllocateScmBlockRequestProto request = AllocateScmBlockRequestProto
.newBuilder().setSize(size).build();
.newBuilder()
.setSize(size).setType(type)
.setFactor(factor)
.build();
final AllocateScmBlockResponseProto response;
try {
response = rpcProxy.allocateScmBlock(NULL_RPC_CONTROLLER, request);

View File

@ -225,6 +225,8 @@ message KeyArgs {
required string bucketName = 2;
required string keyName = 3;
optional uint64 dataSize = 4;
optional hadoop.hdfs.ozone.ReplicationType type = 5;
optional hadoop.hdfs.ozone.ReplicationFactor factor = 6;
}
message KeyLocation {

View File

@ -36,6 +36,9 @@ message Pipeline {
repeated DatanodeIDProto members = 2;
required string containerName = 3;
optional LifeCycleState state = 4 [default = OPEN];
optional ReplicationType type = 5 [default = STAND_ALONE];
optional ReplicationFactor factor = 6 [default = ONE];
optional string pipelineName = 7;
}
message KeyValue {
@ -107,9 +110,12 @@ enum LifeCycleEvent {
}
message SCMContainerInfo {
required LifeCycleState state = 1;
required Pipeline pipeline = 2;
optional int64 stateEnterTime = 3;
// TODO : Remove the container name from pipeline.
required string containerName = 1;
required LifeCycleState state = 2;
required Pipeline pipeline = 3;
optional int64 stateEnterTime = 4;
optional Owner owner = 5 [default = OZONE];
}
enum ReplicationType {
@ -121,4 +127,11 @@ enum ReplicationType {
enum ReplicationFactor {
ONE = 1;
THREE = 3;
}
enum Owner {
OZONE = 1;
CBLOCK = 2;
// In future --
//HDFS = 3;
}

View File

@ -61,6 +61,9 @@ message ScmLocatedBlockProto {
*/
message AllocateScmBlockRequestProto {
required uint64 size = 1;
required hadoop.hdfs.ozone.ReplicationType type = 2;
required hadoop.hdfs.ozone.ReplicationFactor factor = 3;
}
/**

View File

@ -39,6 +39,7 @@ message ContainerRequestProto {
// Ozone only support replciation of either 1 or 3.
required hadoop.hdfs.ozone.ReplicationFactor replicationFactor = 2;
required hadoop.hdfs.ozone.ReplicationType replicationType = 3;
optional hadoop.hdfs.ozone.Owner owner = 4 [default = OZONE];
}

View File

@ -88,6 +88,8 @@ public void shutdown() {
@Override
public ContainerCommandResponseProto dispatch(
ContainerCommandRequestProto msg) {
LOG.trace("Command {}, trace ID: {} ", msg.getCmdType().toString(),
msg.getTraceID());
long startNanos = System.nanoTime();
ContainerCommandResponseProto resp = null;
try {

View File

@ -64,7 +64,10 @@ public void initialize(
throws IOException {
super.initialize(id, properties, raftStorage);
storage.init(raftStorage);
// TODO handle snapshots
// TODO handle snapshots
// TODO: Add a flag that tells you that initialize has been called.
// Check with Ratis if this feature is done in Ratis.
}
@Override

View File

@ -17,8 +17,8 @@
package org.apache.hadoop.ozone.ksm;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
import org.apache.hadoop.ozone.common.BlockGroup;
import org.apache.hadoop.ozone.ksm.helpers.KsmKeyInfo;
@ -41,12 +41,17 @@
import java.util.List;
import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_KEY;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
/**
* Implementation of keyManager.
@ -61,14 +66,17 @@ public class KeyManagerImpl implements KeyManager {
private final ScmBlockLocationProtocol scmBlockClient;
private final KSMMetadataManager metadataManager;
private final long scmBlockSize;
private final boolean useRatis;
private final BackgroundService keyDeletingService;
public KeyManagerImpl(ScmBlockLocationProtocol scmBlockClient,
KSMMetadataManager metadataManager, OzoneConfiguration conf) {
this.scmBlockClient = scmBlockClient;
this.metadataManager = metadataManager;
this.scmBlockSize = conf.getLong(OZONE_SCM_BLOCK_SIZE_KEY,
OZONE_SCM_BLOCK_SIZE_DEFAULT);
this.scmBlockSize = conf.getLong(OZONE_SCM_BLOCK_SIZE_IN_MB,
OZONE_SCM_BLOCK_SIZE_DEFAULT) * OzoneConsts.MB;
this.useRatis = conf.getBoolean(DFS_CONTAINER_RATIS_ENABLED_KEY,
DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
int svcInterval = conf.getInt(
OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS,
OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS_DEFAULT);
@ -96,6 +104,19 @@ public KsmKeyInfo allocateKey(KsmKeyArgs args) throws IOException {
String volumeName = args.getVolumeName();
String bucketName = args.getBucketName();
String keyName = args.getKeyName();
ReplicationFactor factor = args.getFactor();
ReplicationType type = args.getType();
// If user does not specify a replication strategy or
// replication factor, KSM will use defaults.
if(factor == null) {
factor = useRatis ? ReplicationFactor.THREE: ReplicationFactor.ONE;
}
if(type == null) {
type = useRatis ? ReplicationType.RATIS : ReplicationType.STAND_ALONE;
}
try {
byte[] volumeKey = metadataManager.getVolumeKey(volumeName);
byte[] bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
@ -137,7 +158,7 @@ public KsmKeyInfo allocateKey(KsmKeyArgs args) throws IOException {
while (targetSize > 0) {
long allocateSize = Math.min(targetSize, scmBlockSize);
AllocatedBlock allocatedBlock =
scmBlockClient.allocateBlock(allocateSize);
scmBlockClient.allocateBlock(allocateSize, type, factor);
KsmKeyLocationInfo subKeyInfo = new KsmKeyLocationInfo.Builder()
.setContainerName(allocatedBlock.getPipeline().getContainerName())
.setBlockID(allocatedBlock.getKey())

View File

@ -308,6 +308,8 @@ public LocateKeyResponse createKey(
.setBucketName(keyArgs.getBucketName())
.setKeyName(keyArgs.getKeyName())
.setDataSize(keyArgs.getDataSize())
.setType(keyArgs.getType())
.setFactor(keyArgs.getFactor())
.build();
KsmKeyInfo keyInfo = impl.allocateKey(ksmKeyArgs);
resp.setKeyInfo(keyInfo.getProtobuf());

View File

@ -104,7 +104,8 @@ public AllocateScmBlockResponseProto allocateScmBlock(
throws ServiceException {
try {
AllocatedBlock allocatedBlock =
impl.allocateBlock(request.getSize());
impl.allocateBlock(request.getSize(), request.getType(),
request.getFactor());
if (allocatedBlock != null) {
return
AllocateScmBlockResponseProto.newBuilder()

View File

@ -571,9 +571,14 @@ private Set<DatanodeID> queryNodeState(NodeState nodeState) {
public Pipeline allocateContainer(OzoneProtos.ReplicationType replicationType,
OzoneProtos.ReplicationFactor replicationFactor, String containerName)
throws IOException {
//TODO : FIX ME : Pass the owner argument to this function.
// This causes a lot of test change and cblock change to filing
// another JIRA to fix it.
final OzoneProtos.Owner owner = OzoneProtos.Owner.OZONE;
checkAdminAccess();
return scmContainerManager.allocateContainer(replicationType,
replicationFactor, containerName).getPipeline();
replicationFactor, containerName, owner).getPipeline();
}
/**
@ -839,18 +844,22 @@ public Set<AllocatedBlock> getBlockLocations(final Set<String> keys)
}
/**
* Asks SCM where a block should be allocated. SCM responds with the set
* of datanodes and leader that should be used creating this block.
* Asks SCM where a block should be allocated. SCM responds with the set of
* datanodes that should be used creating this block.
*
* @param size - size of the block.
* @return - allocated block accessing info (key, pipeline and leader).
* @param type - Replication type.
* @param factor
* @return allocated block accessing info (key, pipeline).
* @throws IOException
*/
@Override
public AllocatedBlock allocateBlock(final long size) throws IOException {
return scmBlockManager.allocateBlock(size);
public AllocatedBlock allocateBlock(long size, OzoneProtos.ReplicationType
type, OzoneProtos.ReplicationFactor factor) throws IOException {
return scmBlockManager.allocateBlock(size, type, factor);
}
/**
* Delete blocks for a set of object keys.
*

View File

@ -17,6 +17,7 @@
package org.apache.hadoop.ozone.scm.block;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
@ -31,12 +32,15 @@
*/
public interface BlockManager extends Closeable {
/**
* Allocates a new block for a given size.
* @param size - size of the block to be allocated
* @return - the allocated pipeline and key for the block
* Allocates a new block for a given size.
* @param size - Block Size
* @param type Replication Type
* @param factor - Replication Factor
* @return AllocatedBlock
* @throws IOException
*/
AllocatedBlock allocateBlock(long size) throws IOException;
AllocatedBlock allocateBlock(long size, OzoneProtos.ReplicationType type,
OzoneProtos.ReplicationFactor factor) throws IOException;
/**
* Give the key to the block, get the pipeline info.

View File

@ -5,33 +5,34 @@
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
*
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.ozone.scm.block;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.Owner;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
import org.apache.hadoop.ozone.scm.container.Mapping;
import org.apache.hadoop.ozone.scm.exceptions.SCMException;
import org.apache.hadoop.ozone.scm.node.NodeManager;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.apache.hadoop.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.scm.container.common.helpers.BlockContainerInfo;
import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.utils.BatchOperation;
@ -43,35 +44,16 @@
import javax.management.ObjectName;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.stream.Collectors;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_DB;
import static org.apache.hadoop.ozone.OzoneConsts.OPEN_CONTAINERS_DB;
import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
CHILL_MODE_EXCEPTION;
import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
FAILED_TO_ALLOCATE_CONTAINER;
import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
FAILED_TO_FIND_CONTAINER;
import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
FAILED_TO_FIND_CONTAINER_WITH_SPACE;
import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
FAILED_TO_FIND_BLOCK;
import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
FAILED_TO_LOAD_OPEN_CONTAINER;
import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.
INVALID_BLOCK_SIZE;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS;
import static org.apache.hadoop.ozone.OzoneConfigKeys
@ -80,13 +62,22 @@
.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT;
import static org.apache.hadoop.ozone.OzoneConfigKeys
.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_DB;
import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes
.CHILL_MODE_EXCEPTION;
import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes
.FAILED_TO_FIND_BLOCK;
import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes
.INVALID_BLOCK_SIZE;
/**
* Block Manager manages the block access for SCM.
*/
/** Block Manager manages the block access for SCM. */
public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
private static final Logger LOG =
LoggerFactory.getLogger(BlockManagerImpl.class);
// TODO : FIX ME : Hard coding the owner.
// Currently only user of the block service is Ozone, CBlock manages blocks
// by itself and does not rely on the Block service offered by SCM.
private final Owner owner = Owner.OZONE;
private final NodeManager nodeManager;
private final Mapping containerManager;
@ -96,20 +87,16 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
private final long containerSize;
private final long cacheSize;
// Track all containers owned by block service.
private final MetadataStore containerStore;
private final DeletedBlockLog deletedBlockLog;
private final SCMBlockDeletingService blockDeletingService;
private Map<OzoneProtos.LifeCycleState,
Map<String, BlockContainerInfo>> containers;
private final int containerProvisionBatchSize;
private final Random rand;
private ObjectName mxBean;
/**
* Constructor.
*
* @param conf - configuration.
* @param nodeManager - node manager.
* @param containerManager - container manager.
@ -122,34 +109,26 @@ public BlockManagerImpl(final Configuration conf,
this.nodeManager = nodeManager;
this.containerManager = containerManager;
this.cacheSize = cacheSizeMB;
this.containerSize = OzoneConsts.GB * conf.getInt(
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
File metaDir = OzoneUtils.getScmMetadirPath(conf);
String scmMetaDataDir = metaDir.getPath();
// Write the block key to container name mapping.
File blockContainerDbPath = new File(scmMetaDataDir, BLOCK_DB);
blockStore = MetadataStoreBuilder.newBuilder()
.setConf(conf)
.setDbFile(blockContainerDbPath)
.setCacheSize(this.cacheSize * OzoneConsts.MB)
.build();
blockStore =
MetadataStoreBuilder.newBuilder()
.setConf(conf)
.setDbFile(blockContainerDbPath)
.setCacheSize(this.cacheSize * OzoneConsts.MB)
.build();
this.containerSize = OzoneConsts.GB * conf.getInt(
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
// Load store of all open contains for block allocation
File openContainsDbPath = new File(scmMetaDataDir, OPEN_CONTAINERS_DB);
containerStore = MetadataStoreBuilder.newBuilder()
.setConf(conf)
.setDbFile(openContainsDbPath)
.setCacheSize(this.cacheSize * OzoneConsts.MB)
.build();
loadAllocatedContainers();
this.containerProvisionBatchSize = conf.getInt(
ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE,
ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT);
this.containerProvisionBatchSize =
conf.getInt(
ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE,
ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT);
rand = new Random();
this.lock = new ReentrantLock();
@ -157,18 +136,24 @@ public BlockManagerImpl(final Configuration conf,
// SCM block deleting transaction log and deleting service.
deletedBlockLog = new DeletedBlockLogImpl(conf);
int svcInterval = conf.getInt(
OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS,
OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS_DEFAULT);
long serviceTimeout = conf.getTimeDuration(
OZONE_BLOCK_DELETING_SERVICE_TIMEOUT,
OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
blockDeletingService = new SCMBlockDeletingService(deletedBlockLog,
containerManager, nodeManager, svcInterval, serviceTimeout);
int svcInterval =
conf.getInt(
OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS,
OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS_DEFAULT);
long serviceTimeout =
conf.getTimeDuration(
OZONE_BLOCK_DELETING_SERVICE_TIMEOUT,
OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT,
TimeUnit.MILLISECONDS);
blockDeletingService =
new SCMBlockDeletingService(
deletedBlockLog, containerManager, nodeManager, svcInterval,
serviceTimeout);
}
/**
* Start block manager services.
*
* @throws IOException
*/
public void start() throws IOException {
@ -177,6 +162,7 @@ public void start() throws IOException {
/**
* Shutdown block manager services.
*
* @throws IOException
*/
public void stop() throws IOException {
@ -184,59 +170,17 @@ public void stop() throws IOException {
this.close();
}
// TODO: close full (or almost full) containers with a separate thread.
/**
* Load allocated containers from persistent store.
* @throws IOException
*/
private void loadAllocatedContainers() throws IOException {
// Pre-allocate empty map entry by state to avoid null check
containers = new ConcurrentHashMap<>();
for (OzoneProtos.LifeCycleState state :
OzoneProtos.LifeCycleState.values()) {
containers.put(state, new ConcurrentHashMap());
}
try {
containerStore.iterate(null, (key, value) -> {
try {
String containerName = DFSUtil.bytes2String(key);
Long containerUsed = Long.parseLong(DFSUtil.bytes2String(value));
ContainerInfo containerInfo =
containerManager.getContainer(containerName);
// TODO: remove the container from block manager's container DB
// Most likely the allocated container is timeout and cleaned up
// by SCM, we should clean up correspondingly instead of just skip it.
if (containerInfo == null) {
LOG.warn("Container {} allocated by block service" +
"can't be found in SCM", containerName);
return true;
}
Map<String, BlockContainerInfo> containersByState =
containers.get(containerInfo.getState());
containersByState.put(containerName,
new BlockContainerInfo(containerInfo, containerUsed));
LOG.debug("Loading allocated container: {} used : {} state: {}",
containerName, containerUsed, containerInfo.getState());
} catch (Exception e) {
LOG.warn("Failed loading allocated container, continue next...");
}
return true;
});
} catch (IOException e) {
LOG.error("Loading open container store failed." + e);
throw new SCMException("Failed to load open container store",
FAILED_TO_LOAD_OPEN_CONTAINER);
}
}
/**
* Pre allocate specified count of containers for block creation.
* @param count - number of containers to allocate.
* @return list of container names allocated.
*
* @param count - Number of containers to allocate.
* @param type - Type of containers
* @param factor - how many copies needed for this container.
* @throws IOException
*/
private List<String> allocateContainers(int count) throws IOException {
List<String> results = new ArrayList();
private void preAllocateContainers(int count, ReplicationType type,
ReplicationFactor factor)
throws IOException {
lock.lock();
try {
for (int i = 0; i < count; i++) {
@ -244,210 +188,177 @@ private List<String> allocateContainers(int count) throws IOException {
ContainerInfo containerInfo = null;
try {
// TODO: Fix this later when Ratis is made the Default.
containerInfo = containerManager.allocateContainer(
OzoneProtos.ReplicationType.STAND_ALONE,
OzoneProtos.ReplicationFactor.ONE,
containerName);
containerInfo = containerManager.allocateContainer(type, factor,
containerName, owner);
if (containerInfo == null) {
LOG.warn("Unable to allocate container.");
continue;
}
} catch (IOException ex) {
LOG.warn("Unable to allocate container: " + ex);
LOG.warn("Unable to allocate container: {}", ex);
continue;
}
Map<String, BlockContainerInfo> containersByState =
containers.get(OzoneProtos.LifeCycleState.ALLOCATED);
Preconditions.checkNotNull(containersByState);
containersByState.put(containerName,
new BlockContainerInfo(containerInfo, 0));
containerStore.put(DFSUtil.string2Bytes(containerName),
DFSUtil.string2Bytes(Long.toString(0L)));
results.add(containerName);
}
} finally {
lock.unlock();
}
return results;
}
/**
* Filter container by states and size.
* @param state the state of the container.
* @param size the minimal available size of the container
* @return allocated containers satisfy both state and size.
*/
private List <String> filterContainers(OzoneProtos.LifeCycleState state,
long size) {
Map<String, BlockContainerInfo> containersByState =
this.containers.get(state);
return containersByState.entrySet().parallelStream()
.filter(e -> ((e.getValue().getAllocated() + size < containerSize)))
.map(e -> e.getKey())
.collect(Collectors.toList());
}
private BlockContainerInfo getContainer(OzoneProtos.LifeCycleState state,
String name) {
Map<String, BlockContainerInfo> containersByState =
this.containers.get(state);
return containersByState.get(name);
}
// Relies on the caller such as allocateBlock() to hold the lock
// to ensure containers map consistent.
private void updateContainer(OzoneProtos.LifeCycleState oldState, String name,
OzoneProtos.LifeCycleState newState) {
if (LOG.isDebugEnabled()) {
LOG.debug("Update container {} from state {} to state {}",
name, oldState, newState);
}
Map<String, BlockContainerInfo> containersInOldState =
this.containers.get(oldState);
BlockContainerInfo containerInfo = containersInOldState.get(name);
Preconditions.checkNotNull(containerInfo);
containersInOldState.remove(name);
Map<String, BlockContainerInfo> containersInNewState =
this.containers.get(newState);
containersInNewState.put(name, containerInfo);
}
// Refresh containers that have been allocated.
// We may not need to track all the states, just the creating/open/close
// should be enough for now.
private void refreshContainers() {
Map<String, BlockContainerInfo> containersByState =
this.containers.get(OzoneProtos.LifeCycleState.CREATING);
for (String containerName : containersByState.keySet()) {
try {
ContainerInfo containerInfo =
containerManager.getContainer(containerName);
if (containerInfo == null) {
// TODO: clean up containers that has been deleted on SCM but
// TODO: still in ALLOCATED state in block manager.
LOG.debug("Container {} allocated by block service"
+ "can't be found in SCM", containerName);
continue;
}
if (containerInfo.getState() == OzoneProtos.LifeCycleState.OPEN) {
updateContainer(OzoneProtos.LifeCycleState.CREATING, containerName,
containerInfo.getState());
}
// TODO: check containers in other state and refresh as needed.
// TODO: ALLOCATED container that is timeout and DELETED. (unit test)
// TODO: OPEN container that is CLOSE.
} catch (IOException ex) {
LOG.debug("Failed to get container info for: {}", containerName);
}
}
}
/**
* Allocates a new block for a given size.
* Allocates a block in a container and returns that info.
*
* SCM choose one of the open containers and returns that as the location for
* the new block. An open container is a container that is actively written to
* via replicated log.
* @param size - size of the block to be allocated
* @return - the allocated pipeline and key for the block
* @throws IOException
* @param size - Block Size
* @param type Replication Type
* @param factor - Replication Factor
* @return Allocated block
* @throws IOException on failure.
*/
@Override
public AllocatedBlock allocateBlock(final long size) throws IOException {
boolean createContainer = false;
public AllocatedBlock allocateBlock(
final long size, ReplicationType type, ReplicationFactor factor)
throws IOException {
LOG.trace("Size;{} , type : {}, factor : {} ", size, type, factor);
if (size < 0 || size > containerSize) {
throw new SCMException("Unsupported block size", INVALID_BLOCK_SIZE);
LOG.warn("Invalid block size requested : {}", size);
throw new SCMException("Unsupported block size: " + size,
INVALID_BLOCK_SIZE);
}
if (!nodeManager.isOutOfNodeChillMode()) {
LOG.warn("Not out of Chill mode.");
throw new SCMException("Unable to create block while in chill mode",
CHILL_MODE_EXCEPTION);
}
lock.lock();
try {
refreshContainers();
List<String> candidates;
candidates = filterContainers(OzoneProtos.LifeCycleState.OPEN, size);
if (candidates.size() == 0) {
candidates =
filterContainers(OzoneProtos.LifeCycleState.ALLOCATED, size);
if (candidates.size() == 0) {
try {
candidates = allocateContainers(containerProvisionBatchSize);
} catch (IOException ex) {
LOG.error("Unable to allocate container for the block.");
throw new SCMException("Unable to allocate container for the block",
FAILED_TO_ALLOCATE_CONTAINER);
}
}
// now we should have some candidates in ALLOCATE state
if (candidates.size() == 0) {
throw new SCMException(
"Fail to find any container to allocate block " + "of size "
+ size + ".", FAILED_TO_FIND_CONTAINER_WITH_SPACE);
}
/*
Here is the high level logic.
1. First we check if there are containers in ALLOCATED state,
that is
SCM has allocated them in the SCM namespace but the
corresponding
container has not been created in the Datanode yet. If we
have any
in that state, we will return that to the client, which allows
client to finish creating those containers. This is a sort of
greedy
algorithm, our primary purpose is to get as many containers as
possible.
2. If there are no allocated containers -- Then we find a Open
container that matches that pattern.
3. If both of them fail, the we will pre-allocate a bunch of
conatainers in SCM and try again.
TODO : Support random picking of two containers from the list.
So we
can use different kind of policies.
*/
BlockContainerInfo containerInfo = null;
// Look for ALLOCATED container that matches all other parameters.
containerInfo =
containerManager
.getStateManager()
.getMatchingContainer(
size, owner, type, factor, OzoneProtos.LifeCycleState
.ALLOCATED);
if (containerInfo != null) {
return newBlock(containerInfo, OzoneProtos.LifeCycleState.ALLOCATED);
}
// Candidates list now should include only ALLOCATE or OPEN containers
int randomIdx = rand.nextInt(candidates.size());
String containerName = candidates.get(randomIdx);
if (LOG.isDebugEnabled()) {
LOG.debug("Find {} candidates: {}, picking: {}", candidates.size(),
candidates.toString(), containerName);
// Since we found no allocated containers that match our criteria, let us
// look for OPEN containers that match the criteria.
containerInfo =
containerManager
.getStateManager()
.getMatchingContainer(size, owner, type, factor, OzoneProtos
.LifeCycleState.OPEN);
if (containerInfo != null) {
return newBlock(containerInfo, OzoneProtos.LifeCycleState.OPEN);
}
ContainerInfo containerInfo =
containerManager.getContainer(containerName);
if (containerInfo == null) {
LOG.debug("Unable to find container for the block");
throw new SCMException("Unable to find container to allocate block",
FAILED_TO_FIND_CONTAINER);
// We found neither ALLOCATED or OPEN Containers. This generally means
// that most of our containers are full or we have not allocated
// containers of the type and replication factor. So let us go and
// allocate some.
preAllocateContainers(containerProvisionBatchSize, type, factor);
// Since we just allocated a set of containers this should work
containerInfo =
containerManager
.getStateManager()
.getMatchingContainer(
size, owner, type, factor, OzoneProtos.LifeCycleState
.ALLOCATED);
if (containerInfo != null) {
return newBlock(containerInfo, OzoneProtos.LifeCycleState.ALLOCATED);
}
if (LOG.isDebugEnabled()) {
LOG.debug("Candidate {} state {}", containerName,
containerInfo.getState());
}
// Container must be either OPEN or ALLOCATE state
if (containerInfo.getState() == OzoneProtos.LifeCycleState.ALLOCATED) {
createContainer = true;
}
// TODO: make block key easier to debug (e.g., seq no)
// Allocate key for the block
String blockKey = UUID.randomUUID().toString();
AllocatedBlock.Builder abb = new AllocatedBlock.Builder().setKey(blockKey)
.setPipeline(containerInfo.getPipeline())
.setShouldCreateContainer(createContainer);
if (containerInfo.getPipeline().getMachines().size() > 0) {
blockStore.put(DFSUtil.string2Bytes(blockKey),
DFSUtil.string2Bytes(containerName));
// update the container usage information
BlockContainerInfo containerInfoUpdate =
getContainer(containerInfo.getState(), containerName);
Preconditions.checkNotNull(containerInfoUpdate);
containerInfoUpdate.addAllocated(size);
containerStore.put(DFSUtil.string2Bytes(containerName), DFSUtil
.string2Bytes(Long.toString(containerInfoUpdate.getAllocated())));
if (createContainer) {
OzoneProtos.LifeCycleState newState = containerManager
.updateContainerState(containerName,
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
updateContainer(containerInfo.getState(), containerName, newState);
}
return abb.build();
}
// we have tried all strategies we know and but somehow we are not able
// to get a container for this block. Log that info and return a null.
LOG.error(
"Unable to allocate a block for the size: {}, type: {}, " +
"factor: {}",
size,
type,
factor);
return null;
} finally {
lock.unlock();
}
return null;
}
/**
* newBlock - returns a new block assigned to a container.
*
* @param containerInfo - Container Info.
* @param state - Current state of the container.
* @return AllocatedBlock
*/
private AllocatedBlock newBlock(
BlockContainerInfo containerInfo, OzoneProtos.LifeCycleState state)
throws IOException {
// TODO : Replace this with Block ID.
String blockKey = UUID.randomUUID().toString();
boolean createContainer = (state == OzoneProtos.LifeCycleState.ALLOCATED);
AllocatedBlock.Builder abb =
new AllocatedBlock.Builder()
.setKey(blockKey)
// TODO : Use containerinfo instead of pipeline.
.setPipeline(containerInfo.getPipeline())
.setShouldCreateContainer(createContainer);
LOG.trace("New block allocated : {} Container ID: {}", blockKey,
containerInfo.toString());
if (containerInfo.getPipeline().getMachines().size() == 0) {
LOG.error("Pipeline Machine count is zero.");
return null;
}
// Persist this block info to the blockStore DB, so getBlock(key) can
// find which container the block lives.
// TODO : Remove this DB in future
// and make this a KSM operation. Category: SCALABILITY.
if (containerInfo.getPipeline().getMachines().size() > 0) {
blockStore.put(
DFSUtil.string2Bytes(blockKey),
DFSUtil.string2Bytes(containerInfo.getPipeline().getContainerName()));
}
return abb.build();
}
/**
* Given a block key, return the Pipeline information.
*
* @param key - block key assigned by SCM.
* @return Pipeline (list of DNs and leader) to access the block.
* @throws IOException
@ -462,14 +373,15 @@ public Pipeline getBlock(final String key) throws IOException {
"Specified block key does not exist. key : " + key,
FAILED_TO_FIND_BLOCK);
}
String containerName = DFSUtil.bytes2String(containerBytes);
ContainerInfo containerInfo =
containerManager.getContainer(containerName);
ContainerInfo containerInfo = containerManager.getContainer(
containerName);
if (containerInfo == null) {
LOG.debug(
"Container {} allocated by block service" + "can't be found in SCM",
containerName);
throw new SCMException("Unable to find container for the block",
LOG.debug("Container {} allocated by block service"
+ "can't be found in SCM", containerName);
throw new SCMException(
"Unable to find container for the block",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
}
return containerInfo.getPipeline();
@ -479,13 +391,14 @@ public Pipeline getBlock(final String key) throws IOException {
}
/**
* Deletes a list of blocks in an atomic operation. Internally, SCM
* writes these blocks into a {@link DeletedBlockLog} and deletes them
* from SCM DB. If this is successful, given blocks are entering pending
* deletion state and becomes invisible from SCM namespace.
* Deletes a list of blocks in an atomic operation. Internally, SCM writes
* these blocks into a
* {@link DeletedBlockLog} and deletes them from SCM DB. If this is
* successful, given blocks are
* entering pending deletion state and becomes invisible from SCM namespace.
*
* @param blockIDs block IDs. This is often the list of blocks of
* a particular object key.
* @param blockIDs block IDs. This is often the list of blocks of a
* particular object key.
* @throws IOException if exception happens, non of the blocks is deleted.
*/
@Override
@ -546,16 +459,20 @@ public void deleteBlocks(List<String> blockIDs) throws IOException {
// to be invisible from namespace but actual data are not removed.
// We log an error here so admin can manually check and fix such
// errors.
LOG.error("Blocks might be in inconsistent state because"
LOG.error(
"Blocks might be in inconsistent state because"
+ " they were moved to pending deletion state in SCM DB but"
+ " not written into delLog. Admin can manually add them"
+ " into delLog for deletions. Inconsistent block list: {}",
String.join(",", blockIDs), e);
String.join(",", blockIDs),
e);
throw rollbackException;
}
throw new IOException("Skip writing the deleted blocks info to"
+ " the delLog because addTransaction fails. Batch skipped: "
+ String.join(",", blockIDs), e);
throw new IOException(
"Skip writing the deleted blocks info to"
+ " the delLog because addTransaction fails. Batch skipped: "
+ String.join(",", blockIDs),
e);
}
// TODO: Container report handling of the deleted blocks:
// Remove tombstone and update open container usage.
@ -577,6 +494,7 @@ public String getDeletedKeyName(String key) {
/**
* Close the resources for BlockManager.
*
* @throws IOException
*/
@Override
@ -584,9 +502,6 @@ public void close() throws IOException {
if (blockStore != null) {
blockStore.close();
}
if (containerStore != null) {
containerStore.close();
}
if (deletedBlockLog != null) {
deletedBlockLog.close();
}
@ -599,6 +514,11 @@ public void close() throws IOException {
@Override
public int getOpenContainersNo() {
return containers.get(OzoneProtos.LifeCycleState.OPEN).size();
return 0;
// TODO : FIX ME : The open container being a single number does not make
// sense.
// We have to get open containers by Replication Type and Replication
// factor. Hence returning 0 for now.
// containers.get(OzoneProtos.LifeCycleState.OPEN).size();
}
}

View File

@ -1,36 +1,36 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* <p>Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.ozone.scm.container;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
import org.apache.hadoop.ozone.common.statemachine.StateMachine;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.Owner;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
import org.apache.hadoop.ozone.scm.exceptions.SCMException;
import org.apache.hadoop.ozone.scm.node.NodeManager;
import org.apache.hadoop.ozone.scm.pipelines.PipelineSelector;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.scm.container.common.helpers.BlockContainerInfo;
import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.utils.MetadataKeyFilters.KeyPrefixFilter;
import org.apache.hadoop.utils.MetadataKeyFilters.MetadataKeyFilter;
import org.apache.hadoop.utils.MetadataStore;
@ -42,22 +42,21 @@
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB;
/**
* Mapping class contains the mapping from a name to a pipeline mapping. This is
* used by SCM when allocating new locations and when looking up a key.
* Mapping class contains the mapping from a name to a pipeline mapping. This
* is used by SCM when
* allocating new locations and when looking up a key.
*/
public class ContainerMapping implements Mapping {
private static final Logger LOG =
LoggerFactory.getLogger(ContainerMapping.class);
private static final Logger LOG = LoggerFactory.getLogger(ContainerMapping
.class);
private final NodeManager nodeManager;
private final long cacheSize;
@ -65,24 +64,27 @@ public class ContainerMapping implements Mapping {
private final Charset encoding = Charset.forName("UTF-8");
private final MetadataStore containerStore;
private final PipelineSelector pipelineSelector;
private final StateMachine<OzoneProtos.LifeCycleState,
OzoneProtos.LifeCycleEvent> stateMachine;
private final ContainerStateManager containerStateManager;
/**
* Constructs a mapping class that creates mapping between container names and
* pipelines.
* Constructs a mapping class that creates mapping between container names
* and pipelines.
*
* @param nodeManager - NodeManager so that we can get the nodes that are
* healthy to place new containers.
* healthy to place new
* containers.
* @param cacheSizeMB - Amount of memory reserved for the LSM tree to cache
* its nodes. This is passed to LevelDB and this memory is allocated in Native
* code space. CacheSize is specified in MB.
* its nodes. This is
* passed to LevelDB and this memory is allocated in Native code space.
* CacheSize is specified
* in MB.
* @throws IOException
*/
@SuppressWarnings("unchecked")
public ContainerMapping(final Configuration conf,
final NodeManager nodeManager, final int cacheSizeMB) throws IOException {
public ContainerMapping(
final Configuration conf, final NodeManager nodeManager, final int
cacheSizeMB)
throws IOException {
this.nodeManager = nodeManager;
this.cacheSize = cacheSizeMB;
@ -90,100 +92,48 @@ public ContainerMapping(final Configuration conf,
// Write the container name to pipeline mapping.
File containerDBPath = new File(metaDir, CONTAINER_DB);
containerStore = MetadataStoreBuilder.newBuilder()
.setConf(conf)
.setDbFile(containerDBPath)
.setCacheSize(this.cacheSize * OzoneConsts.MB)
.build();
containerStore =
MetadataStoreBuilder.newBuilder()
.setConf(conf)
.setDbFile(containerDBPath)
.setCacheSize(this.cacheSize * OzoneConsts.MB)
.build();
this.lock = new ReentrantLock();
this.pipelineSelector = new PipelineSelector(nodeManager, conf);
// Initialize the container state machine.
Set<OzoneProtos.LifeCycleState> finalStates = new HashSet();
finalStates.add(OzoneProtos.LifeCycleState.OPEN);
finalStates.add(OzoneProtos.LifeCycleState.CLOSED);
finalStates.add(OzoneProtos.LifeCycleState.DELETED);
this.stateMachine = new StateMachine<>(
OzoneProtos.LifeCycleState.ALLOCATED, finalStates);
initializeStateMachine();
this.containerStateManager = new ContainerStateManager(conf, +this
.cacheSize * OzoneConsts.MB);
LOG.trace("Container State Manager created.");
}
// Client-driven Create State Machine
// States: <ALLOCATED>------------->CREATING----------------->[OPEN]
// Events: (BEGIN_CREATE) | (COMPLETE_CREATE)
// |
// |(TIMEOUT)
// V
// DELETING----------------->[DELETED]
// (CLEANUP)
// SCM Open/Close State Machine
// States: OPEN------------------>[CLOSED]
// Events: (CLOSE)
// Delete State Machine
// States: OPEN------------------>DELETING------------------>[DELETED]
// Events: (DELETE) (CLEANUP)
private void initializeStateMachine() {
stateMachine.addTransition(OzoneProtos.LifeCycleState.ALLOCATED,
OzoneProtos.LifeCycleState.CREATING,
OzoneProtos.LifeCycleEvent.BEGIN_CREATE);
stateMachine.addTransition(OzoneProtos.LifeCycleState.CREATING,
OzoneProtos.LifeCycleState.OPEN,
OzoneProtos.LifeCycleEvent.COMPLETE_CREATE);
stateMachine.addTransition(OzoneProtos.LifeCycleState.OPEN,
OzoneProtos.LifeCycleState.CLOSED,
OzoneProtos.LifeCycleEvent.CLOSE);
stateMachine.addTransition(OzoneProtos.LifeCycleState.OPEN,
OzoneProtos.LifeCycleState.DELETING,
OzoneProtos.LifeCycleEvent.DELETE);
stateMachine.addTransition(OzoneProtos.LifeCycleState.DELETING,
OzoneProtos.LifeCycleState.DELETED,
OzoneProtos.LifeCycleEvent.CLEANUP);
// Creating timeout -> Deleting
stateMachine.addTransition(OzoneProtos.LifeCycleState.CREATING,
OzoneProtos.LifeCycleState.DELETING,
OzoneProtos.LifeCycleEvent.TIMEOUT);
}
/**
* {@inheritDoc}
*/
/** {@inheritDoc} */
@Override
public ContainerInfo getContainer(final String containerName)
throws IOException {
public ContainerInfo getContainer(final String containerName) throws
IOException {
ContainerInfo containerInfo;
lock.lock();
try {
byte[] containerBytes =
containerStore.get(containerName.getBytes(encoding));
byte[] containerBytes = containerStore.get(containerName.getBytes(
encoding));
if (containerBytes == null) {
throw new SCMException(
"Specified key does not exist. key : " + containerName,
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
}
containerInfo = ContainerInfo.fromProtobuf(
OzoneProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes));
containerInfo =
ContainerInfo.fromProtobuf(OzoneProtos.SCMContainerInfo.PARSER
.parseFrom(containerBytes));
return containerInfo;
} finally {
lock.unlock();
}
}
/**
* {@inheritDoc}
*/
/** {@inheritDoc} */
@Override
public List<Pipeline> listContainer(String startName,
String prefixName, int count)
public List<Pipeline> listContainer(String startName, String prefixName,
int count)
throws IOException {
List<Pipeline> pipelineList = new ArrayList<>();
lock.lock();
@ -192,8 +142,8 @@ public List<Pipeline> listContainer(String startName,
throw new IOException("No container exists in current db");
}
MetadataKeyFilter prefixFilter = new KeyPrefixFilter(prefixName);
byte[] startKey = startName == null ?
null : DFSUtil.string2Bytes(startName);
byte[] startKey = startName == null ? null : DFSUtil.string2Bytes(
startName);
List<Map.Entry<byte[], byte[]>> range =
containerStore.getRangeKVs(startKey, count, prefixFilter);
@ -201,8 +151,10 @@ public List<Pipeline> listContainer(String startName,
// TODO: return list of ContainerInfo instead of pipelines.
// TODO: filter by container state
for (Map.Entry<byte[], byte[]> entry : range) {
ContainerInfo containerInfo = ContainerInfo.fromProtobuf(
OzoneProtos.SCMContainerInfo.PARSER.parseFrom(entry.getValue()));
ContainerInfo containerInfo =
ContainerInfo.fromProtobuf(
OzoneProtos.SCMContainerInfo.PARSER.parseFrom(
entry.getValue()));
Preconditions.checkNotNull(containerInfo);
pipelineList.add(containerInfo.getPipeline());
}
@ -215,40 +167,43 @@ public List<Pipeline> listContainer(String startName,
/**
* Allocates a new container.
*
* @param containerName - Name of the container.
* @param replicationFactor - replication factor of the container.
* @param containerName - Name of the container.
* @param owner
* @return - Pipeline that makes up this container.
* @throws IOException - Exception
*/
@Override
public ContainerInfo allocateContainer(OzoneProtos.ReplicationType type,
OzoneProtos.ReplicationFactor replicationFactor,
final String containerName) throws IOException {
public ContainerInfo allocateContainer(
ReplicationType type,
ReplicationFactor replicationFactor,
final String containerName,
Owner owner)
throws IOException {
Preconditions.checkNotNull(containerName);
Preconditions.checkState(!containerName.isEmpty());
ContainerInfo containerInfo = null;
if (!nodeManager.isOutOfNodeChillMode()) {
throw new SCMException("Unable to create container while in chill mode",
throw new SCMException(
"Unable to create container while in chill mode",
SCMException.ResultCodes.CHILL_MODE_EXCEPTION);
}
lock.lock();
try {
byte[] containerBytes =
containerStore.get(containerName.getBytes(encoding));
byte[] containerBytes = containerStore.get(containerName.getBytes(
encoding));
if (containerBytes != null) {
throw new SCMException("Specified container already exists. key : " +
containerName, SCMException.ResultCodes.CONTAINER_EXISTS);
throw new SCMException(
"Specified container already exists. key : " + containerName,
SCMException.ResultCodes.CONTAINER_EXISTS);
}
Pipeline pipeline = pipelineSelector.getReplicationPipeline(type,
replicationFactor, containerName);
containerInfo = new ContainerInfo.Builder()
.setState(OzoneProtos.LifeCycleState.ALLOCATED)
.setPipeline(pipeline)
.setStateEnterTime(Time.monotonicNow())
.build();
containerStore.put(containerName.getBytes(encoding),
containerInfo.getProtobuf().toByteArray());
containerInfo =
containerStateManager.allocateContainer(
pipelineSelector, type, replicationFactor, containerName, owner);
containerStore.put(
containerName.getBytes(encoding), containerInfo.getProtobuf()
.toByteArray());
} finally {
lock.unlock();
}
@ -259,19 +214,20 @@ public ContainerInfo allocateContainer(OzoneProtos.ReplicationType type,
* Deletes a container from SCM.
*
* @param containerName - Container name
* @throws IOException if container doesn't exist or container store failed to
* delete the specified key.
* @throws IOException if container doesn't exist or container store failed
* to delete the
* specified key.
*/
@Override
public void deleteContainer(String containerName) throws IOException {
lock.lock();
try {
byte[] dbKey = containerName.getBytes(encoding);
byte[] containerBytes =
containerStore.get(dbKey);
if(containerBytes == null) {
throw new SCMException("Failed to delete container "
+ containerName + ", reason : container doesn't exist.",
byte[] containerBytes = containerStore.get(dbKey);
if (containerBytes == null) {
throw new SCMException(
"Failed to delete container " + containerName + ", reason : " +
"container doesn't exist.",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
}
containerStore.delete(dbKey);
@ -287,8 +243,11 @@ public void closeContainer(String containerName) throws IOException {
OzoneProtos.LifeCycleState newState =
updateContainerState(containerName, OzoneProtos.LifeCycleEvent.CLOSE);
if (newState != OzoneProtos.LifeCycleState.CLOSED) {
throw new SCMException("Failed to close container " + containerName +
", reason : container in state " + newState,
throw new SCMException(
"Failed to close container "
+ containerName
+ ", reason : container in state "
+ newState,
SCMException.ResultCodes.UNEXPECTED_CONTAINER_STATE);
}
} finally {
@ -296,13 +255,11 @@ public void closeContainer(String containerName) throws IOException {
}
}
/**
* {@inheritDoc}
* Used by client to update container state on SCM.
*/
/** {@inheritDoc} Used by client to update container state on SCM. */
@Override
public OzoneProtos.LifeCycleState updateContainerState(String containerName,
OzoneProtos.LifeCycleEvent event) throws IOException {
public OzoneProtos.LifeCycleState updateContainerState(
String containerName, OzoneProtos.LifeCycleEvent event) throws
IOException {
ContainerInfo containerInfo;
lock.lock();
try {
@ -310,39 +267,49 @@ public OzoneProtos.LifeCycleState updateContainerState(String containerName,
byte[] containerBytes = containerStore.get(dbKey);
if (containerBytes == null) {
throw new SCMException(
"Failed to update container state" + containerName
"Failed to update container state"
+ containerName
+ ", reason : container doesn't exist.",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
}
containerInfo = ContainerInfo.fromProtobuf(
OzoneProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes));
containerInfo =
ContainerInfo.fromProtobuf(OzoneProtos.SCMContainerInfo.PARSER
.parseFrom(containerBytes));
Preconditions.checkNotNull(containerInfo);
// TODO: Actual used will be updated via Container Reports later.
containerInfo.setState(
containerStateManager.updateContainerState(
new BlockContainerInfo(containerInfo, 0), event));
OzoneProtos.LifeCycleState newState;
try {
newState = stateMachine.getNextState(containerInfo.getState(), event);
} catch (InvalidStateTransitionException ex) {
throw new SCMException(
"Failed to update container state" + containerName
+ ", reason : invalid state transition from state: "
+ containerInfo.getState() + " upon event: " + event + ".",
SCMException.ResultCodes.FAILED_TO_CHANGE_CONTAINER_STATE);
}
containerInfo.setState(newState);
containerStore.put(dbKey, containerInfo.getProtobuf().toByteArray());
return newState;
return containerInfo.getState();
} finally {
lock.unlock();
}
}
/** + * Returns the container State Manager. + * + * @return
* ContainerStateManager + */
@Override
public ContainerStateManager getStateManager() {
return containerStateManager;
}
/**
* Closes this stream and releases any system resources associated with it. If
* the stream is already closed then invoking this method has no effect.
* Closes this stream and releases any system resources associated with it.
* If the stream is
* already closed then invoking this method has no effect.
*
* <p>
* <p> As noted in {@link AutoCloseable#close()}, cases where the close may
* fail require careful attention. It is strongly advised to relinquish the
* underlying resources and to internally <em>mark</em> the {@code Closeable}
* as closed, prior to throwing the {@code IOException}.
*
* <p>As noted in {@link AutoCloseable#close()}, cases where the close may
* fail require careful
* attention. It is strongly advised to relinquish the underlying resources
* and to internally
* <em>mark</em> the {@code Closeable} as closed, prior to throwing the
* {@code IOException}.
*
* @throws IOException if an I/O error occurs
*/

View File

@ -0,0 +1,472 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.ozone.scm.container;
import com.google.common.base.Preconditions;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
import org.apache.hadoop.ozone.common.statemachine.StateMachine;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.scm.exceptions.SCMException;
import org.apache.hadoop.ozone.scm.pipelines.PipelineSelector;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.apache.hadoop.scm.container.common.helpers.BlockContainerInfo;
import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.PriorityQueue;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleState;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.Owner;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleEvent;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationFactor;
import static org.apache.hadoop.ozone.scm.exceptions.SCMException.ResultCodes.FAILED_TO_CHANGE_CONTAINER_STATE;
/**
* A container state manager keeps track of container states and returns
* containers that match various queries.
* <p>
* This state machine is driven by a combination of server and client actions.
* <p>
* This is how a create container happens: 1. When a container is created, the
* Server(or SCM) marks that Container as ALLOCATED state. In this state, SCM
* has chosen a pipeline for container to live on. However, the container is not
* created yet. This container along with the pipeline is returned to the
* client.
* <p>
* 2. The client when it sees the Container state as ALLOCATED understands that
* container needs to be created on the specified pipeline. The client lets the
* SCM know that saw this flag and is initiating the on the data nodes.
* <p>
* This is done by calling into notifyObjectCreation(ContainerName,
* BEGIN_CREATE) flag. When SCM gets this call, SCM puts the container state
* into CREATING. All this state means is that SCM told Client to create a
* container and client saw that request.
* <p>
* 3. Then client makes calls to datanodes directly, asking the datanodes to
* create the container. This is done with the help of pipeline that supports
* this container.
* <p>
* 4. Once the creation of the container is complete, the client will make
* another call to the SCM, this time specifing the containerName and the
* COMPLETE_CREATE as the Event.
* <p>
* 5. With COMPLETE_CREATE event, the container moves to an Open State. This is
* the state when clients can write to a container.
* <p>
* 6. If the client does not respond with the COMPLETE_CREATE event with a
* certain time, the state machine times out and triggers a delete operation of
* the container.
* <p>
* Please see the function initializeStateMachine below to see how this looks in
* code.
* <p>
* Reusing existing container :
* <p>
* The create container call is not made all the time, the system tries to use
* open containers as much as possible. So in those cases, it looks thru the
* list of open containers and will return containers that match the specific
* signature.
* <p>
* Please note : Logically there are 3 separate state machines in the case of
* containers.
* <p>
* The Create State Machine -- Commented extensively above.
* <p>
* Open/Close State Machine - Once the container is in the Open State,
* eventually it will be closed, once sufficient data has been written to it.
* <p>
* TimeOut Delete Container State Machine - if the container creating times out,
* then Container State manager decides to delete the container.
*/
public class ContainerStateManager {
private static final Logger LOG =
LoggerFactory.getLogger(ContainerStateManager.class);
private final StateMachine<OzoneProtos.LifeCycleState,
OzoneProtos.LifeCycleEvent> stateMachine;
private final long containerSize;
private final long cacheSize;
private final long blockSize;
// A map that maintains the ContainerKey to Containers of that type ordered
// by last access time.
private final Lock writeLock;
private final Queue<BlockContainerInfo> containerCloseQueue;
private Map<ContainerKey, PriorityQueue<BlockContainerInfo>> containers;
/**
* Constructs a Container State Manager that tracks all containers owned by
* SCM for the purpose of allocation of blocks.
* <p>
* TODO : Add Container Tags so we know which containers are owned by SCM.
*/
public ContainerStateManager(Configuration configuration, final long
cacheSize) throws IOException {
this.cacheSize = cacheSize;
// Initialize the container state machine.
Set<OzoneProtos.LifeCycleState> finalStates = new HashSet();
// These are the steady states of a container.
finalStates.add(LifeCycleState.OPEN);
finalStates.add(LifeCycleState.CLOSED);
finalStates.add(LifeCycleState.DELETED);
this.stateMachine = new StateMachine<>(LifeCycleState.ALLOCATED,
finalStates);
initializeStateMachine();
this.containerSize = OzoneConsts.GB * configuration.getInt(
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_GB,
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT);
this.blockSize = OzoneConsts.MB * configuration.getLong(
OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB,
OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT);
writeLock = new ReentrantLock();
containers = new HashMap<>();
initializeContainerMaps(containers);
containerCloseQueue = new ConcurrentLinkedQueue<BlockContainerInfo>();
}
/**
* Creates containers maps of following types.
* <p>
* OZONE of type {Ratis, StandAlone, Chained} for each of these {ALLOCATED,
* CREATING, OPEN, CLOSED, DELETING, DELETED} container states
* <p>
* CBLOCK of type {Ratis, StandAlone, Chained} for each of these {ALLOCATED,
* CREATING, OPEN, CLOSED, DELETING, DELETED} container states
* <p>
* Commented out for now: HDFS of type {Ratis, StandAlone, Chained} for each
* of these {ALLOCATED, CREATING, OPEN, CLOSED, DELETING, DELETED} container
* states
*/
private void initializeContainerMaps(Map containerMaps) {
// Called only from Ctor path, hence no lock is held.
Preconditions.checkNotNull(containerMaps);
for (OzoneProtos.Owner owner : OzoneProtos.Owner.values()) {
for (ReplicationType type : ReplicationType.values()) {
for (ReplicationFactor factor : ReplicationFactor.values()) {
for (LifeCycleState state : LifeCycleState.values()) {
ContainerKey key = new ContainerKey(owner, type, factor, state);
PriorityQueue<BlockContainerInfo> queue = new PriorityQueue<>();
containerMaps.put(key, queue);
}
}
}
}
}
// 1. Client -> SCM: Begin_create
// 2. Client -> Datanode: create
// 3. Client -> SCM: complete {SCM:Creating ->OK}
// 3. Client -> SCM: complete {SCM:DELETING -> INVALID}
// 4. Client->Datanode: write data.
// Client-driven Create State Machine
// States: <ALLOCATED>------------->CREATING----------------->[OPEN]
// Events: (BEGIN_CREATE) | (COMPLETE_CREATE)
// |
// |(TIMEOUT)
// V
// DELETING----------------->[DELETED]
// (CLEANUP)
// SCM Open/Close State Machine
// States: OPEN------------------>PENDING_CLOSE---------->[CLOSE]
// Events: (FULL_CONTAINER) (CLOSE)
// Delete State Machine
// States: OPEN------------------>DELETING------------------>[DELETED]
// Events: (DELETE) (CLEANUP)
private void initializeStateMachine() {
stateMachine.addTransition(LifeCycleState.ALLOCATED,
LifeCycleState.CREATING,
LifeCycleEvent.BEGIN_CREATE);
stateMachine.addTransition(LifeCycleState.CREATING,
LifeCycleState.OPEN,
LifeCycleEvent.COMPLETE_CREATE);
stateMachine.addTransition(LifeCycleState.OPEN,
LifeCycleState.CLOSED,
LifeCycleEvent.CLOSE);
stateMachine.addTransition(LifeCycleState.OPEN,
LifeCycleState.DELETING,
LifeCycleEvent.DELETE);
stateMachine.addTransition(LifeCycleState.DELETING,
LifeCycleState.DELETED,
LifeCycleEvent.CLEANUP);
// Creating timeout -> Deleting
stateMachine.addTransition(LifeCycleState.CREATING,
LifeCycleState.DELETING,
LifeCycleEvent.TIMEOUT);
}
/**
* allocates a new container based on the type, replication etc.
*
* @param selector -- Pipeline selector class.
* @param type -- Replication type.
* @param replicationFactor - Replication replicationFactor.
* @param containerName - Container Name.
* @return Container Info.
* @throws IOException
*/
public ContainerInfo allocateContainer(PipelineSelector selector, OzoneProtos
.ReplicationType type, OzoneProtos.ReplicationFactor replicationFactor,
final String containerName, OzoneProtos.Owner owner) throws
IOException {
Pipeline pipeline = selector.getReplicationPipeline(type,
replicationFactor, containerName);
ContainerInfo info = new ContainerInfo.Builder()
.setContainerName(containerName)
.setState(OzoneProtos.LifeCycleState.ALLOCATED)
.setPipeline(pipeline)
.setStateEnterTime(Time.monotonicNow())
.setOwner(owner)
.build();
Preconditions.checkNotNull(info);
BlockContainerInfo blockInfo = new BlockContainerInfo(info, 0);
blockInfo.setLastUsed(Time.monotonicNow());
writeLock.lock();
try {
ContainerKey key = new ContainerKey(owner, type, replicationFactor,
info.getState());
PriorityQueue<BlockContainerInfo> queue = containers.get(key);
Preconditions.checkNotNull(queue);
queue.add(blockInfo);
LOG.trace("New container allocated: {}", blockInfo);
} finally {
writeLock.unlock();
}
return info;
}
/**
* Update the Container State to the next state.
*
* @param info - ContainerInfo
* @param event - LifeCycle Event
* @return New state of the container.
* @throws SCMException
*/
public OzoneProtos.LifeCycleState updateContainerState(BlockContainerInfo
info, OzoneProtos.LifeCycleEvent event) throws SCMException {
LifeCycleState newState = null;
boolean shouldLease = false;
try {
newState = this.stateMachine.getNextState(info.getState(), event);
if(newState == LifeCycleState.CREATING) {
// if we are moving into a Creating State, it is possible that clients
// could timeout therefore we need to use a lease.
shouldLease = true;
}
} catch (InvalidStateTransitionException ex) {
String error = String.format("Failed to update container state %s, " +
"reason: invalid state transition from state: %s upon event: %s.",
info.getPipeline().getContainerName(), info.getState(), event);
LOG.error(error);
throw new SCMException(error, FAILED_TO_CHANGE_CONTAINER_STATE);
}
// This is a post condition after executing getNextState.
Preconditions.checkNotNull(newState);
Pipeline pipeline = info.getPipeline();
ContainerKey oldKey = new ContainerKey(info.getOwner(), pipeline.getType(),
pipeline.getFactor(), info.getState());
ContainerKey newKey = new ContainerKey(info.getOwner(), pipeline.getType(),
pipeline.getFactor(), newState);
writeLock.lock();
try {
PriorityQueue<BlockContainerInfo> currentQueue = containers.get(oldKey);
// This should never happen, since we have initialized the map and
// queues to all possible states. No harm in asserting that info.
Preconditions.checkNotNull(currentQueue);
// TODO : Should we read this container info from the database if this
// is missing in the queue?. Right now we just add it into the queue.
// We also need a background thread that will remove unused containers
// from memory after 24 hours. This is really a low priority work item
// since typical clusters will have less than 10's of millions of open
// containers at a given time, which we can easily keep in memory.
if (currentQueue.contains(info)) {
currentQueue.remove(info);
}
info.setState(newState);
PriorityQueue<BlockContainerInfo> nextQueue = containers.get(newKey);
Preconditions.checkNotNull(nextQueue);
info.setLastUsed(Time.monotonicNow());
nextQueue.add(info);
return newState;
} finally {
writeLock.unlock();
}
}
/**
* Return a container matching the attributes specified.
*
* @param size - Space needed in the Container.
* @param owner - Owner of the container {OZONE, CBLOCK}
* @param type - Replication Type {StandAlone, Ratis}
* @param factor - Replication Factor {ONE, THREE}
* @param state - State of the Container-- {Open, Allocated etc.}
* @return BlockContainerInfo
*/
public BlockContainerInfo getMatchingContainer(final long size,
Owner owner, ReplicationType type, ReplicationFactor factor,
LifeCycleState state) {
ContainerKey key = new ContainerKey(owner, type, factor, state);
writeLock.lock();
try {
PriorityQueue<BlockContainerInfo> queue = containers.get(key);
if (queue.size() == 0) {
// We don't have any Containers of this type.
return null;
}
Iterator<BlockContainerInfo> iter = queue.iterator();
// Two assumptions here.
// 1. The Iteration on the heap is in ordered by the last used time.
// 2. We remove and add the node back to push the node to the end of
// the queue.
while (iter.hasNext()) {
BlockContainerInfo info = iter.next();
if (info.getAllocated() < this.containerSize + size) {
queue.remove(info);
info.addAllocated(size);
info.setLastUsed(Time.monotonicNow());
queue.add(info);
return info;
} else {
// We should close this container.
LOG.info("Moving {} to containerCloseQueue.", info.toString());
containerCloseQueue.add(info);
//TODO: Next JIRA will handle these containers to close.
}
}
} finally {
writeLock.unlock();
}
return null;
}
/**
* Class that acts as the container Key.
*/
private static class ContainerKey {
private final LifeCycleState state;
private final ReplicationType type;
private final OzoneProtos.Owner owner;
private final ReplicationFactor replicationFactor;
/**
* Constructs a Container Key.
*
* @param owner - Container Owners
* @param type - Replication Type.
* @param factor - Replication Factors
* @param state - LifeCycle State
*/
ContainerKey(Owner owner, ReplicationType type,
ReplicationFactor factor, LifeCycleState state) {
this.state = state;
this.type = type;
this.owner = owner;
this.replicationFactor = factor;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
ContainerKey that = (ContainerKey) o;
return new EqualsBuilder()
.append(state, that.state)
.append(type, that.type)
.append(owner, that.owner)
.append(replicationFactor, that.replicationFactor)
.isEquals();
}
@Override
public int hashCode() {
return new HashCodeBuilder(137, 757)
.append(state)
.append(type)
.append(owner)
.append(replicationFactor)
.toHashCode();
}
@Override
public String toString() {
return "ContainerKey{" +
"state=" + state +
", type=" + type +
", owner=" + owner +
", replicationFactor=" + replicationFactor +
'}';
}
}
}

View File

@ -61,14 +61,15 @@ List<Pipeline> listContainer(String startName, String prefixName, int count)
/**
* Allocates a new container for a given keyName and replication factor.
*
* @param containerName - Name.
* @param replicationFactor - replication factor of the container.
* @param containerName - Name.
* @param owner
* @return - Container Info.
* @throws IOException
*/
ContainerInfo allocateContainer(OzoneProtos.ReplicationType type,
OzoneProtos.ReplicationFactor replicationFactor,
String containerName) throws IOException;
String containerName, OzoneProtos.Owner owner) throws IOException;
/**
* Deletes a container from SCM.
@ -95,4 +96,10 @@ ContainerInfo allocateContainer(OzoneProtos.ReplicationType type,
*/
OzoneProtos.LifeCycleState updateContainerState(String containerName,
OzoneProtos.LifeCycleEvent event) throws IOException;
/**
* Returns the container State Manager.
* @return ContainerStateManager
*/
ContainerStateManager getStateManager();
}

View File

@ -51,9 +51,9 @@ public class PipelineSelector {
private final StandaloneManagerImpl standaloneManager;
private final long containerSize;
/**
* Constructs a pipeline Selector.
*
* @param nodeManager - node manager
* @param conf - Ozone Config
*/
@ -71,6 +71,30 @@ public PipelineSelector(NodeManager nodeManager, Configuration conf) {
new RatisManagerImpl(this.nodeManager, placementPolicy, containerSize);
}
/**
* Translates a list of nodes, ordered such that the first is the leader, into
* a corresponding {@link Pipeline} object.
*
* @param nodes - list of datanodes on which we will allocate the container.
* The first of the list will be the leader node.
* @return pipeline corresponding to nodes
*/
public static Pipeline newPipelineFromNodes(final List<DatanodeID> nodes) {
Preconditions.checkNotNull(nodes);
Preconditions.checkArgument(nodes.size() > 0);
String leaderId = nodes.get(0).getDatanodeUuid();
Pipeline pipeline = new Pipeline(leaderId);
for (DatanodeID node : nodes) {
pipeline.addMember(node);
}
// A Standalone pipeline is always open, no action from the client
// is needed to make it open.
pipeline.setType(ReplicationType.STAND_ALONE);
pipeline.setLifeCycleState(OzoneProtos.LifeCycleState.OPEN);
return pipeline;
}
/**
* Create pluggable container placement policy implementation instance.
*
@ -106,13 +130,15 @@ private static ContainerPlacementPolicy createContainerPlacementPolicy(
/**
* Return the pipeline manager from the replication type.
*
* @param replicationType - Replication Type Enum.
* @return pipeline Manager.
* @throws IllegalArgumentException
* @throws IllegalArgumentException If an pipeline type gets added
* and this function is not modified we will throw.
*/
private PipelineManager getPipelineManager(ReplicationType replicationType)
throws IllegalArgumentException {
switch(replicationType){
switch (replicationType) {
case RATIS:
return this.ratisManager;
case STAND_ALONE:
@ -131,7 +157,6 @@ private PipelineManager getPipelineManager(ReplicationType replicationType)
* container. The client specifies what kind of replication pipeline is needed
* and based on the replication type in the request appropriate Interface is
* invoked.
*
*/
public Pipeline getReplicationPipeline(ReplicationType replicationType,

View File

@ -16,22 +16,33 @@
*/
package org.apache.hadoop.ozone.scm.pipelines.ratis;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.scm.container.placement.algorithms
.ContainerPlacementPolicy;
import org.apache.hadoop.ozone.scm.container.placement.algorithms.ContainerPlacementPolicy;
import org.apache.hadoop.ozone.scm.node.NodeManager;
import org.apache.hadoop.ozone.scm.pipelines.PipelineManager;
import org.apache.hadoop.ozone.scm.pipelines.PipelineSelector;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleState.ALLOCATED;
import static org.apache.hadoop.ozone.protocol.proto.OzoneProtos.LifeCycleState.OPEN;
/**
* Implementation of {@link PipelineManager}.
*
* TODO : Introduce a state machine.
*/
public class RatisManagerImpl implements PipelineManager {
private static final Logger LOG =
@ -39,9 +50,14 @@ public class RatisManagerImpl implements PipelineManager {
private final NodeManager nodeManager;
private final ContainerPlacementPolicy placementPolicy;
private final long containerSize;
private final Set<DatanodeID> ratisMembers;
private final List<Pipeline> activePipelines;
private final AtomicInteger pipelineIndex;
private static final String PREFIX = "Ratis-";
/**
* Constructs a Ratis Pipeline Manager.
*
* @param nodeManager
*/
public RatisManagerImpl(NodeManager nodeManager,
@ -49,6 +65,9 @@ public RatisManagerImpl(NodeManager nodeManager,
this.nodeManager = nodeManager;
this.placementPolicy = placementPolicy;
this.containerSize = size;
ratisMembers = new HashSet<>();
activePipelines = new LinkedList<>();
pipelineIndex = new AtomicInteger(0);
}
/**
@ -60,13 +79,141 @@ public RatisManagerImpl(NodeManager nodeManager,
* @param containerName Name of the container
* @param replicationFactor - Replication Factor
* @return a Pipeline.
* <p>
* TODO: Evaulate if we really need this lock. Right now favoring safety over
* speed.
*/
@Override
public Pipeline getPipeline(String containerName,
public synchronized Pipeline getPipeline(String containerName,
OzoneProtos.ReplicationFactor replicationFactor) {
/**
* In the ratis world, we have a very simple policy.
*
* 1. Try to create a pipeline if there are enough free nodes.
*
* 2. This allows all nodes to part of a pipeline quickly.
*
* 3. if there are not enough free nodes, return pipelines in a
* round-robin fashion.
*
* TODO: Might have to come up with a better algorithm than this.
* Create a new placement policy that returns pipelines in round robin
* fashion.
*/
Pipeline pipeline = null;
List<DatanodeID> newNodes = allocatePipelineNodes(replicationFactor);
if (newNodes != null) {
Preconditions.checkState(newNodes.size() ==
getReplicationCount(replicationFactor), "Replication factor " +
"does not match the expected node count.");
pipeline = allocateRatisPipeline(newNodes, containerName);
} else {
pipeline = findOpenPipeline();
}
if (pipeline == null) {
LOG.error("Get pipeline call failed. We are not able to find free nodes" +
" or operational pipeline.");
}
return pipeline;
}
/**
* Find a pipeline that is operational.
*
* @return - Pipeline or null
*/
Pipeline findOpenPipeline() {
Pipeline pipeline = null;
final int sentinal = -1;
if (activePipelines.size() == 0) {
LOG.error("No Operational pipelines found. Returning null.");
return pipeline;
}
int startIndex = getNextIndex();
int nextIndex = sentinal;
for (; startIndex != nextIndex; nextIndex = getNextIndex()) {
// Just walk the list in a circular way.
Pipeline temp =
activePipelines.get(nextIndex != sentinal ? nextIndex : startIndex);
// if we find an operational pipeline just return that.
if (temp.getLifeCycleState() == OPEN) {
pipeline = temp;
break;
}
}
return pipeline;
}
/**
* Allocate a new Ratis pipeline from the existing nodes.
*
* @param nodes - list of Nodes.
* @param containerName - container Name
* @return - Pipeline.
*/
Pipeline allocateRatisPipeline(List<DatanodeID> nodes, String containerName) {
Preconditions.checkNotNull(nodes);
Pipeline pipeline = PipelineSelector.newPipelineFromNodes(nodes);
if (pipeline != null) {
// Start all pipeline names with "Ratis", easy to grep the logs.
String pipelineName = PREFIX +
UUID.randomUUID().toString().substring(PREFIX.length());
pipeline.setType(OzoneProtos.ReplicationType.RATIS);
pipeline.setLifeCycleState(ALLOCATED);
pipeline.setPipelineName(pipelineName);
pipeline.setContainerName(containerName);
LOG.info("Creating new ratis pipeline: {}", pipeline.toString());
activePipelines.add(pipeline);
}
return pipeline;
}
/**
* gets the next index of in the pipelines to get.
*
* @return index in the link list to get.
*/
private int getNextIndex() {
return pipelineIndex.incrementAndGet() % activePipelines.size();
}
/**
* Allocates a set of new nodes for the Ratis pipeline.
*
* @param replicationFactor - One or Three
* @return List of Datanodes.
*/
private List<DatanodeID> allocatePipelineNodes(OzoneProtos.ReplicationFactor
replicationFactor) {
List<DatanodeID> newNodesList = new LinkedList<>();
List<DatanodeID> datanodes =
nodeManager.getNodes(OzoneProtos.NodeState.HEALTHY);
int count = getReplicationCount(replicationFactor);
//TODO: Add Raft State to the Nodes, so we can query and skip nodes from
// data from datanode instead of maintaining a set.
for (DatanodeID datanode : datanodes) {
if (!ratisMembers.contains(datanode)) {
newNodesList.add(datanode);
if (newNodesList.size() == count) {
LOG.info("Allocating a new pipeline of size: {}", count);
return newNodesList;
}
}
}
return null;
}
private int getReplicationCount(OzoneProtos.ReplicationFactor factor) {
switch (factor) {
case ONE:
return 1;
case THREE:
return 3;
default:
throw new IllegalArgumentException("Unexpected replication count");
}
}
/**
* Creates a pipeline from a specified set of Nodes.
*

View File

@ -16,22 +16,27 @@
*/
package org.apache.hadoop.ozone.scm.pipelines.standalone;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.scm.container.placement.algorithms.ContainerPlacementPolicy;
import org.apache.hadoop.ozone.scm.node.NodeManager;
import org.apache.hadoop.ozone.scm.pipelines.PipelineManager;
import org.apache.hadoop.ozone.scm.pipelines.PipelineSelector;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.List;
import java.util.UUID;
/**
* Standalone Manager Impl to prove that pluggable interface
* works with current tests.
*/
public class StandaloneManagerImpl implements PipelineManager {
private static final Logger LOG =
LoggerFactory.getLogger(StandaloneManagerImpl.class);
private final NodeManager nodeManager;
private final ContainerPlacementPolicy placementPolicy;
private final long containerSize;
@ -49,31 +54,6 @@ public StandaloneManagerImpl(NodeManager nodeManager,
this.containerSize = containerSize;
}
/**
* Translates a list of nodes, ordered such that the first is the leader, into
* a corresponding {@link Pipeline} object.
*
* @param nodes - list of datanodes on which we will allocate the container.
* The first of the list will be the leader node.
* @param containerName container name
* @return pipeline corresponding to nodes
*/
private static Pipeline newPipelineFromNodes(final List<DatanodeID> nodes,
final String containerName) {
Preconditions.checkNotNull(nodes);
Preconditions.checkArgument(nodes.size() > 0);
String leaderId = nodes.get(0).getDatanodeUuid();
Pipeline pipeline = new Pipeline(leaderId);
for (DatanodeID node : nodes) {
pipeline.addMember(node);
}
// The default state of a pipeline is operational, so not setting
// explicit state here.
pipeline.setContainerName(containerName);
return pipeline;
}
/**
* This function is called by the Container Manager while allocating a new
@ -90,7 +70,12 @@ public Pipeline getPipeline(String containerName, OzoneProtos
.ReplicationFactor replicationFactor) throws IOException {
List<DatanodeID> datanodes = placementPolicy.chooseDatanodes(
replicationFactor.getNumber(), containerSize);
return newPipelineFromNodes(datanodes, containerName);
Pipeline pipeline = PipelineSelector.newPipelineFromNodes(datanodes);
String pipelineName = "SA-" + UUID.randomUUID().toString().substring(3);
pipeline.setContainerName(containerName);
pipeline.setPipelineName(pipelineName);
LOG.info("Creating new standalone pipeline: {}", pipeline.toString());
return pipeline;
}
/**

View File

@ -1,19 +1,18 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.ozone.tools;
@ -35,6 +34,7 @@
import org.apache.hadoop.ozone.client.OzoneClientFactory;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
@ -95,6 +95,7 @@ public final class Corona extends Configured implements Tool {
private static final String NUM_OF_BUCKETS = "numOfBuckets";
private static final String NUM_OF_KEYS = "numOfKeys";
private static final String KEY_SIZE = "keySize";
private static final String RATIS = "ratis";
private static final String MODE_DEFAULT = "offline";
private static final String SOURCE_DEFAULT =
@ -120,6 +121,8 @@ public final class Corona extends Configured implements Tool {
private String numOfVolumes;
private String numOfBuckets;
private String numOfKeys;
private boolean useRatis;
private int replicationFactor = 0;
private int keySize;
@ -164,19 +167,28 @@ public final class Corona extends Configured implements Tool {
objectStore = ozoneClient.getObjectStore();
}
/**
* @param args arguments
*/
public static void main(String[] args) throws Exception {
Configuration conf = new OzoneConfiguration();
int res = ToolRunner.run(conf, new Corona(conf), args);
System.exit(res);
}
@Override
public int run(String[] args) throws Exception {
GenericOptionsParser parser = new GenericOptionsParser(getConf(),
getOptions(), args);
parseOptions(parser.getCommandLine());
if(printUsage) {
if (printUsage) {
usage();
return 0;
}
LOG.info("Number of Threads: " + numOfThreads);
processor = Executors.newFixedThreadPool(Integer.parseInt(numOfThreads));
addShutdownHook();
if(mode.equals("online")) {
if (mode.equals("online")) {
LOG.info("Mode: online");
throw new UnsupportedOperationException("Not yet implemented.");
} else {
@ -192,7 +204,7 @@ public int run(String[] args) throws Exception {
}
}
Thread validator = null;
if(validateWrites) {
if (validateWrites) {
totalWritesValidated = 0L;
writeValidationSuccessCount = 0L;
writeValidationFailureCount = 0L;
@ -210,7 +222,7 @@ public int run(String[] args) throws Exception {
processor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS);
completed = true;
progressbar.join();
if(validateWrites) {
if (validateWrites) {
validator.join();
}
ozoneClient.close();
@ -269,6 +281,12 @@ private Options getOptions() {
"created in offline mode");
Option optKeySize = OptionBuilder.create(KEY_SIZE);
OptionBuilder.withArgName(RATIS);
OptionBuilder.hasArg();
OptionBuilder.withDescription("Use Ratis as the default replication " +
"strategy");
Option optRatis = OptionBuilder.create(RATIS);
options.addOption(optHelp);
options.addOption(optMode);
options.addOption(optSource);
@ -278,6 +296,7 @@ private Options getOptions() {
options.addOption(optNumOfBuckets);
options.addOption(optNumOfKeys);
options.addOption(optKeySize);
options.addOption(optRatis);
return options;
}
@ -306,7 +325,11 @@ private void parseOptions(CommandLine cmdLine) {
keySize = cmdLine.hasOption(KEY_SIZE) ?
Integer.parseInt(cmdLine.getOptionValue(KEY_SIZE)) : KEY_SIZE_DEFAULT;
useRatis = cmdLine.hasOption(RATIS);
//To-do if replication factor is not mentioned throw an exception
replicationFactor = useRatis ?
Integer.parseInt(cmdLine.getOptionValue(RATIS)) : 0;
}
private void usage() {
@ -336,75 +359,6 @@ private void usage() {
System.out.println();
}
private class OfflineProcessor implements Runnable {
private int totalBuckets;
private int totalKeys;
private OzoneVolume volume;
OfflineProcessor(String volumeName) throws Exception {
this.totalBuckets = Integer.parseInt(numOfBuckets);
this.totalKeys = Integer.parseInt(numOfKeys);
LOG.trace("Creating volume: {}", volumeName);
long start = System.nanoTime();
objectStore.createVolume(volumeName);
volumeCreationTime.getAndAdd(System.nanoTime() - start);
numberOfVolumesCreated.getAndIncrement();
volume = objectStore.getVolume(volumeName);
}
@Override
public void run() {
for (int j = 0; j < totalBuckets; j++) {
String bucketName = "bucket-" + j + "-" +
RandomStringUtils.randomNumeric(5);
try {
LOG.trace("Creating bucket: {} in volume: {}",
bucketName, volume.getName());
long start = System.nanoTime();
volume.createBucket(bucketName);
bucketCreationTime.getAndAdd(System.nanoTime() - start);
numberOfBucketsCreated.getAndIncrement();
OzoneBucket bucket = volume.getBucket(bucketName);
for (int k = 0; k < totalKeys; k++) {
String key = "key-" + k + "-" +
RandomStringUtils.randomNumeric(5);
byte[] value = DFSUtil.string2Bytes(
RandomStringUtils.randomAscii(keySize));
try {
LOG.trace("Adding key: {} in bucket: {} of volume: {}",
key, bucket, volume);
long keyCreateStart = System.nanoTime();
OzoneOutputStream os = bucket.createKey(key, value.length);
keyCreationTime.getAndAdd(System.nanoTime() - keyCreateStart);
long keyWriteStart = System.nanoTime();
os.write(value);
os.close();
keyWriteTime.getAndAdd(System.nanoTime() - keyWriteStart);
totalBytesWritten.getAndAdd(value.length);
numberOfKeysAdded.getAndIncrement();
if(validateWrites) {
boolean validate = validationQueue.offer(
new KeyValue(bucket, key, value));
if(validate) {
LOG.trace("Key {}, is queued for validation.", key);
}
}
} catch (Exception e) {
exception = true;
LOG.error("Exception while adding key: {} in bucket: {}" +
" of volume: {}.", key, bucket, volume, e);
}
}
} catch (Exception e) {
exception = true;
LOG.error("Exception while creating bucket: {}" +
" in volume: {}.", bucketName, volume, e);
}
}
}
}
/**
* Adds ShutdownHook to print statistics.
*/
@ -417,7 +371,7 @@ private Thread getProgressBarThread() {
Supplier<Long> currentValue;
long maxValue;
if(mode.equals("online")) {
if (mode.equals("online")) {
throw new UnsupportedOperationException("Not yet implemented.");
} else {
currentValue = () -> numberOfKeysAdded.get();
@ -431,73 +385,6 @@ private Thread getProgressBarThread() {
return progressBarThread;
}
private class ProgressBar implements Runnable {
private static final long REFRESH_INTERVAL = 1000L;
private PrintStream stream;
private Supplier<Long> currentValue;
private long maxValue;
ProgressBar(PrintStream stream, Supplier<Long> currentValue,
long maxValue) {
this.stream = stream;
this.currentValue = currentValue;
this.maxValue = maxValue;
}
@Override
public void run() {
try {
stream.println();
long value;
while((value = currentValue.get()) < maxValue) {
print(value);
if(completed) {
break;
}
Thread.sleep(REFRESH_INTERVAL);
}
if(exception) {
stream.println();
stream.println("Incomplete termination, " +
"check log for exception.");
} else {
print(maxValue);
}
stream.println();
} catch (InterruptedException e) {
}
}
/**
* Given current value prints the progress bar.
*
* @param value
*/
private void print(long value) {
stream.print('\r');
double percent = 100.0 * value / maxValue;
StringBuilder sb = new StringBuilder();
sb.append(" " + String.format("%.2f", percent) + "% |");
for (int i = 0; i <= percent; i++) {
sb.append('█');
}
for (int j = 0; j < 100 - percent; j++) {
sb.append(' ');
}
sb.append("| ");
sb.append(value + "/" + maxValue);
long timeInSec = TimeUnit.SECONDS.convert(
System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
String timeToPrint = String.format("%d:%02d:%02d", timeInSec / 3600,
(timeInSec % 3600) / 60, timeInSec % 60);
sb.append(" Time: " + timeToPrint);
stream.print(sb);
}
}
/**
* Prints stats of {@link Corona} run to the PrintStream.
*
@ -578,7 +465,7 @@ private void printStats(PrintStream out) {
out.println("Time spent in key creation: " + prettyKeyCreationTime);
out.println("Time spent in writing keys: " + prettyKeyWriteTime);
out.println("Total bytes written: " + totalBytesWritten);
if(validateWrites) {
if (validateWrites) {
out.println("Total number of writes validated: " +
totalWritesValidated);
out.println("Writes validated: " +
@ -608,7 +495,7 @@ int getNumberOfVolumesCreated() {
*/
@VisibleForTesting
int getNumberOfBucketsCreated() {
return numberOfBucketsCreated.get();
return numberOfBucketsCreated.get();
}
/**
@ -617,7 +504,7 @@ int getNumberOfBucketsCreated() {
*/
@VisibleForTesting
long getNumberOfKeysAdded() {
return numberOfKeysAdded.get();
return numberOfKeysAdded.get();
}
/**
@ -656,40 +543,6 @@ long getUnsuccessfulValidationCount() {
return writeValidationFailureCount;
}
/**
* Validates the write done in ozone cluster.
*/
private class Validator implements Runnable {
@Override
public void run() {
while(!completed) {
try {
KeyValue kv = validationQueue.poll(5, TimeUnit.SECONDS);
if(kv != null) {
OzoneInputStream is = kv.bucket.readKey(kv.key);
byte[] value = new byte[kv.value.length];
int length = is.read(value);
totalWritesValidated++;
if (length == kv.value.length && Arrays.equals(value, kv.value)) {
writeValidationSuccessCount++;
} else {
writeValidationFailureCount++;
LOG.warn("Data validation error for key {}/{}/{}",
kv.bucket.getVolumeName(), kv.bucket, kv.key);
LOG.warn("Expected: {}, Actual: {}",
DFSUtil.bytes2String(kv.value),
DFSUtil.bytes2String(value));
}
}
} catch (IOException | InterruptedException ex) {
LOG.error("Exception while validating write: " + ex.getMessage());
}
}
}
}
/**
* Wrapper to hold ozone key-value pair.
*/
@ -721,12 +574,185 @@ private static class KeyValue {
}
}
private class OfflineProcessor implements Runnable {
private int totalBuckets;
private int totalKeys;
private OzoneVolume volume;
OfflineProcessor(String volumeName) throws Exception {
this.totalBuckets = Integer.parseInt(numOfBuckets);
this.totalKeys = Integer.parseInt(numOfKeys);
LOG.trace("Creating volume: {}", volumeName);
long start = System.nanoTime();
objectStore.createVolume(volumeName);
volumeCreationTime.getAndAdd(System.nanoTime() - start);
numberOfVolumesCreated.getAndIncrement();
volume = objectStore.getVolume(volumeName);
}
@Override
public void run() {
OzoneProtos.ReplicationType type = OzoneProtos.ReplicationType
.STAND_ALONE;
OzoneProtos.ReplicationFactor factor = OzoneProtos.ReplicationFactor.ONE;
if (useRatis) {
type = OzoneProtos.ReplicationType.RATIS;
factor = replicationFactor != 0 ?
OzoneProtos.ReplicationFactor.valueOf(replicationFactor) :
OzoneProtos.ReplicationFactor.THREE;
}
for (int j = 0; j < totalBuckets; j++) {
String bucketName = "bucket-" + j + "-" +
RandomStringUtils.randomNumeric(5);
try {
LOG.trace("Creating bucket: {} in volume: {}",
bucketName, volume.getName());
long start = System.nanoTime();
volume.createBucket(bucketName);
bucketCreationTime.getAndAdd(System.nanoTime() - start);
numberOfBucketsCreated.getAndIncrement();
OzoneBucket bucket = volume.getBucket(bucketName);
for (int k = 0; k < totalKeys; k++) {
String key = "key-" + k + "-" +
RandomStringUtils.randomNumeric(5);
byte[] value = DFSUtil.string2Bytes(
RandomStringUtils.randomAscii(keySize));
try {
LOG.trace("Adding key: {} in bucket: {} of volume: {}",
key, bucket, volume);
long keyCreateStart = System.nanoTime();
OzoneOutputStream os = bucket.createKey(key, value.length,
type, factor);
keyCreationTime.getAndAdd(System.nanoTime() - keyCreateStart);
long keyWriteStart = System.nanoTime();
os.write(value);
os.close();
keyWriteTime.getAndAdd(System.nanoTime() - keyWriteStart);
totalBytesWritten.getAndAdd(value.length);
numberOfKeysAdded.getAndIncrement();
if (validateWrites) {
boolean validate = validationQueue.offer(
new KeyValue(bucket, key, value));
if (validate) {
LOG.trace("Key {}, is queued for validation.", key);
}
}
} catch (Exception e) {
exception = true;
LOG.error("Exception while adding key: {} in bucket: {}" +
" of volume: {}.", key, bucket, volume, e);
}
}
} catch (Exception e) {
exception = true;
LOG.error("Exception while creating bucket: {}" +
" in volume: {}.", bucketName, volume, e);
}
}
}
}
private class ProgressBar implements Runnable {
private static final long REFRESH_INTERVAL = 1000L;
private PrintStream stream;
private Supplier<Long> currentValue;
private long maxValue;
ProgressBar(PrintStream stream, Supplier<Long> currentValue,
long maxValue) {
this.stream = stream;
this.currentValue = currentValue;
this.maxValue = maxValue;
}
@Override
public void run() {
try {
stream.println();
long value;
while ((value = currentValue.get()) < maxValue) {
print(value);
if (completed) {
break;
}
Thread.sleep(REFRESH_INTERVAL);
}
if (exception) {
stream.println();
stream.println("Incomplete termination, " +
"check log for exception.");
} else {
print(maxValue);
}
stream.println();
} catch (InterruptedException e) {
}
}
/**
* Given current value prints the progress bar.
*
* @param value
*/
private void print(long value) {
stream.print('\r');
double percent = 100.0 * value / maxValue;
StringBuilder sb = new StringBuilder();
sb.append(" " + String.format("%.2f", percent) + "% |");
for (int i = 0; i <= percent; i++) {
sb.append('█');
}
for (int j = 0; j < 100 - percent; j++) {
sb.append(' ');
}
sb.append("| ");
sb.append(value + "/" + maxValue);
long timeInSec = TimeUnit.SECONDS.convert(
System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
String timeToPrint = String.format("%d:%02d:%02d", timeInSec / 3600,
(timeInSec % 3600) / 60, timeInSec % 60);
sb.append(" Time: " + timeToPrint);
stream.print(sb);
}
}
/**
* @param args arguments
* Validates the write done in ozone cluster.
*/
public static void main(String[] args) throws Exception {
Configuration conf = new OzoneConfiguration();
int res = ToolRunner.run(conf, new Corona(conf), args);
System.exit(res);
private class Validator implements Runnable {
@Override
public void run() {
while (!completed) {
try {
KeyValue kv = validationQueue.poll(5, TimeUnit.SECONDS);
if (kv != null) {
OzoneInputStream is = kv.bucket.readKey(kv.key);
byte[] value = new byte[kv.value.length];
int length = is.read(value);
totalWritesValidated++;
if (length == kv.value.length && Arrays.equals(value, kv.value)) {
writeValidationSuccessCount++;
} else {
writeValidationFailureCount++;
LOG.warn("Data validation error for key {}/{}/{}",
kv.bucket.getVolumeName(), kv.bucket, kv.key);
LOG.warn("Expected: {}, Actual: {}",
DFSUtil.bytes2String(kv.value),
DFSUtil.bytes2String(value));
}
}
} catch (IOException | InterruptedException ex) {
LOG.error("Exception while validating write: " + ex.getMessage());
}
}
}
}
}

View File

@ -35,6 +35,7 @@
import org.apache.hadoop.ozone.client.io.ChunkGroupOutputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.protocolPB.KSMPBHelper;
import org.apache.hadoop.ozone.ksm.KSMConfigKeys;
import org.apache.hadoop.ozone.OzoneAcl;
@ -82,6 +83,9 @@ public final class DistributedStorageHandler implements StorageHandler {
private final OzoneAcl.OzoneACLRights userRights;
private final OzoneAcl.OzoneACLRights groupRights;
private int chunkSize;
private final boolean useRatis;
private final OzoneProtos.ReplicationType type;
private final OzoneProtos.ReplicationFactor factor;
/**
* Creates a new DistributedStorageHandler.
@ -98,6 +102,17 @@ public DistributedStorageHandler(OzoneConfiguration conf,
this.keySpaceManagerClient = keySpaceManagerClient;
this.storageContainerLocationClient = storageContainerLocation;
this.xceiverClientManager = new XceiverClientManager(conf);
this.useRatis = conf.getBoolean(
ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,
ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT);
if(useRatis) {
type = OzoneProtos.ReplicationType.RATIS;
factor = OzoneProtos.ReplicationFactor.THREE;
} else {
type = OzoneProtos.ReplicationType.STAND_ALONE;
factor = OzoneProtos.ReplicationFactor.ONE;
}
chunkSize = conf.getInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY,
ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_DEFAULT);
@ -390,6 +405,8 @@ public OutputStream newKeyWriter(KeyArgs args) throws IOException,
.setBucketName(args.getBucketName())
.setKeyName(args.getKeyName())
.setDataSize(args.getSize())
.setType(xceiverClientManager.getType())
.setFactor(xceiverClientManager.getFactor())
.build();
// contact KSM to allocate a block for key.
KsmKeyInfo keyInfo = keySpaceManagerClient.allocateKey(keyArgs);

View File

@ -656,8 +656,8 @@
</description>
</property>
<property>
<name>ozone.scm.block.size</name>
<value>268435456</value>
<name>ozone.scm.block.size.in.mb</name>
<value>256</value>
<tag>OZONE, SCM</tag>
<description>
The default size of a scm block in bytes. This is maps to the default

View File

@ -36,6 +36,7 @@
import org.apache.hadoop.ozone.client.VolumeArgs;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
import org.junit.AfterClass;
import org.junit.Assert;
@ -371,7 +372,8 @@ public void testPutKey()
String keyName = UUID.randomUUID().toString();
OzoneOutputStream out = bucket.createKey(keyName,
value.getBytes().length);
value.getBytes().length, OzoneProtos.ReplicationType.STAND_ALONE,
OzoneProtos.ReplicationFactor.ONE);
out.write(value.getBytes());
out.close();
OzoneKey key = bucket.getKey(keyName);
@ -396,7 +398,8 @@ public void testDeleteKey()
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
OzoneOutputStream out = bucket.createKey(keyName,
value.getBytes().length);
value.getBytes().length, OzoneProtos.ReplicationType.STAND_ALONE,
OzoneProtos.ReplicationFactor.ONE);
out.write(value.getBytes());
out.close();
OzoneKey key = bucket.getKey(keyName);
@ -548,22 +551,26 @@ public void listKeyTest()
byte[] value = RandomStringUtils.randomAscii(10240).getBytes();
OzoneOutputStream one = volAbucketA.createKey(
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
value.length);
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
OzoneProtos.ReplicationFactor.ONE);
one.write(value);
one.close();
OzoneOutputStream two = volAbucketB.createKey(
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
value.length);
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
OzoneProtos.ReplicationFactor.ONE);
two.write(value);
two.close();
OzoneOutputStream three = volBbucketA.createKey(
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
value.length);
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
OzoneProtos.ReplicationFactor.ONE);
three.write(value);
three.close();
OzoneOutputStream four = volBbucketB.createKey(
keyBaseA + i + "-" + RandomStringUtils.randomNumeric(5),
value.length);
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
OzoneProtos.ReplicationFactor.ONE);
four.write(value);
four.close();
}
@ -577,22 +584,26 @@ public void listKeyTest()
byte[] value = RandomStringUtils.randomAscii(10240).getBytes();
OzoneOutputStream one = volAbucketA.createKey(
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
value.length);
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
OzoneProtos.ReplicationFactor.ONE);
one.write(value);
one.close();
OzoneOutputStream two = volAbucketB.createKey(
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
value.length);
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
OzoneProtos.ReplicationFactor.ONE);
two.write(value);
two.close();
OzoneOutputStream three = volBbucketA.createKey(
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
value.length);
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
OzoneProtos.ReplicationFactor.ONE);
three.write(value);
three.close();
OzoneOutputStream four = volBbucketB.createKey(
keyBaseB + i + "-" + RandomStringUtils.randomNumeric(5),
value.length);
value.length, OzoneProtos.ReplicationType.STAND_ALONE,
OzoneProtos.ReplicationFactor.ONE);
four.write(value);
four.close();
}

View File

@ -978,6 +978,7 @@ public void testGetKeyInfo() throws IOException,
KeyArgs keyArgs = new KeyArgs(keyName, bucketArgs);
keyArgs.setSize(4096);
OutputStream stream = storageHandler.newKeyWriter(keyArgs);
stream.close();

View File

@ -70,7 +70,7 @@ public class TestMultipleContainerReadWrite {
public static void init() throws Exception {
conf = new OzoneConfiguration();
// set to as small as 100 bytes per block.
conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_KEY, 100);
conf.setLong(OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB, 100);
conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, 5);
conf.set(OzoneConfigKeys.OZONE_HANDLER_TYPE_KEY,
OzoneConsts.OZONE_HANDLER_DISTRIBUTED);

View File

@ -23,6 +23,7 @@
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.scm.block.BlockManagerImpl;
import org.apache.hadoop.ozone.scm.cli.SQLCLI;
import org.apache.hadoop.ozone.scm.container.ContainerMapping;
@ -50,13 +51,12 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import static org.apache.hadoop.ozone.OzoneConsts.BLOCK_DB;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_DB;
import static org.apache.hadoop.ozone.OzoneConsts.KB;
import static org.apache.hadoop.ozone.OzoneConsts.NODEPOOL_DB;
import static org.apache.hadoop.ozone.OzoneConsts.OPEN_CONTAINERS_DB;
//import static org.apache.hadoop.ozone.OzoneConsts.OPEN_CONTAINERS_DB;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@ -97,6 +97,9 @@ public TestContainerSQLCli(String type) {
private HashMap<String, String> blockContainerMap;
private final static long DEFAULT_BLOCK_SIZE = 4 * KB;
private static OzoneProtos.ReplicationFactor factor;
private static OzoneProtos.ReplicationType type;
@Before
public void setup() throws Exception {
@ -107,8 +110,15 @@ public void setup() throws Exception {
conf.setInt(ScmConfigKeys.OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE, 2);
conf.setClass(ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
SCMContainerPlacementCapacity.class, ContainerPlacementPolicy.class);
cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(2
)
if(conf.getBoolean(ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,
ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT)){
factor = OzoneProtos.ReplicationFactor.THREE;
type = OzoneProtos.ReplicationType.RATIS;
} else {
factor = OzoneProtos.ReplicationFactor.ONE;
type = OzoneProtos.ReplicationType.STAND_ALONE;
}
cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(2)
.storageCapacities(new long[] {datanodeCapacities, datanodeCapacities})
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
storageContainerLocationClient =
@ -134,7 +144,8 @@ public void setup() throws Exception {
Thread.sleep(100);
}
assertEquals(2, nodeManager.getAllNodes().size());
AllocatedBlock ab1 = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE);
AllocatedBlock ab1 = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, type,
factor);
pipeline1 = ab1.getPipeline();
blockContainerMap.put(ab1.getKey(), pipeline1.getContainerName());
@ -146,7 +157,7 @@ public void setup() throws Exception {
// although each retry will create a block and assign to a container. So
// the size of blockContainerMap will vary each time the test is run.
while (true) {
ab2 = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE);
ab2 = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE, type, factor);
pipeline2 = ab2.getPipeline();
blockContainerMap.put(ab2.getKey(), pipeline2.getContainerName());
if (!pipeline1.getContainerName().equals(pipeline2.getContainerName())) {
@ -217,38 +228,6 @@ public void testConvertNodepoolDB() throws Exception {
Files.delete(Paths.get(dbOutPath));
}
@Test
public void testConvertOpenContainerDB() throws Exception {
String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";
String dbRootPath = conf.get(OzoneConfigKeys.OZONE_METADATA_DIRS);
String dbPath = dbRootPath + "/" + OPEN_CONTAINERS_DB;
String[] args = {"-p", dbPath, "-o", dbOutPath};
cli.run(args);
Connection conn = connectDB(dbOutPath);
String sql = "SELECT * FROM openContainer";
ResultSet rs = executeQuery(conn, sql);
HashSet<String> expectedContainer = new HashSet<>();
expectedContainer.add(pipeline1.getContainerName());
expectedContainer.add(pipeline2.getContainerName());
// the number of allocated blocks can vary, and they can be located
// at either of the two containers. We only check if the total used
// is equal to block size * # of blocks.
long totalUsed = 0;
while(rs.next()) {
String containerName = rs.getString("containerName");
long containerUsed = rs.getLong("containerUsed");
totalUsed += containerUsed;
assertTrue(expectedContainer.remove(containerName));
}
assertEquals(0, expectedContainer.size());
assertEquals(blockContainerMap.keySet().size() * DEFAULT_BLOCK_SIZE,
totalUsed);
Files.delete(Paths.get(dbOutPath));
}
@Test
public void testConvertContainerDB() throws Exception {
String dbOutPath = cluster.getDataDirectory() + "/out_sql.db";

View File

@ -21,8 +21,10 @@
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.scm.container.ContainerMapping;
import org.apache.hadoop.ozone.scm.container.MockNodeManager;
import org.apache.hadoop.scm.ScmConfigKeys;
import org.apache.hadoop.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.test.GenericTestUtils;
@ -53,6 +55,8 @@ public class TestBlockManager {
private static BlockManagerImpl blockManager;
private static File testDir;
private final static long DEFAULT_BLOCK_SIZE = 128 * MB;
private static OzoneProtos.ReplicationFactor factor;
private static OzoneProtos.ReplicationType type;
@Rule
public ExpectedException thrown = ExpectedException.none();
@ -67,11 +71,19 @@ public static void setUp() throws Exception {
testDir = Paths.get(path).toFile();
boolean folderExisted = testDir.exists() || testDir.mkdirs();
if (!folderExisted) {
throw new IOException("Unable to create test diectory path");
throw new IOException("Unable to create test directory path");
}
nodeManager = new MockNodeManager(true, 10);
mapping = new ContainerMapping(conf, nodeManager, 128);
blockManager = new BlockManagerImpl(conf, nodeManager, mapping, 128);
if(conf.getBoolean(ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY,
ScmConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT)){
factor = OzoneProtos.ReplicationFactor.THREE;
type = OzoneProtos.ReplicationType.RATIS;
} else {
factor = OzoneProtos.ReplicationFactor.ONE;
type = OzoneProtos.ReplicationType.STAND_ALONE;
}
}
@AfterClass
@ -88,13 +100,15 @@ public void clearChillMode() {
@Test
public void testAllocateBlock() throws Exception {
AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE);
AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
type, factor);
Assert.assertNotNull(block);
}
@Test
public void testGetAllocatedBlock() throws IOException {
AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE);
AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
type, factor);
Assert.assertNotNull(block);
Pipeline pipeline = blockManager.getBlock(block.getKey());
Assert.assertEquals(pipeline.getLeader().getDatanodeUuid(),
@ -103,7 +117,8 @@ public void testGetAllocatedBlock() throws IOException {
@Test
public void testDeleteBlock() throws Exception {
AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE);
AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
type, factor);
Assert.assertNotNull(block);
blockManager.deleteBlocks(Collections.singletonList(block.getKey()));
@ -123,7 +138,8 @@ public void testDeleteBlock() throws Exception {
public void testAllocateOversizedBlock() throws IOException {
long size = 6 * GB;
thrown.expectMessage("Unsupported block size");
AllocatedBlock block = blockManager.allocateBlock(size);
AllocatedBlock block = blockManager.allocateBlock(size,
type, factor);
}
@Test
@ -137,6 +153,7 @@ public void testGetNoneExistentContainer() throws IOException {
public void testChillModeAllocateBlockFails() throws IOException {
nodeManager.setChillmode(true);
thrown.expectMessage("Unable to create block while in chill mode");
blockManager.allocateBlock(DEFAULT_BLOCK_SIZE);
blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
type, factor);
}
}

View File

@ -20,6 +20,7 @@
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.scm.XceiverClientManager;
import org.apache.hadoop.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.scm.container.common.helpers.Pipeline;
@ -84,7 +85,7 @@ public void testallocateContainer() throws Exception {
ContainerInfo containerInfo = mapping.allocateContainer(
xceiverClientManager.getType(),
xceiverClientManager.getFactor(),
UUID.randomUUID().toString());
UUID.randomUUID().toString(), OzoneProtos.Owner.OZONE);
Assert.assertNotNull(containerInfo);
}
@ -101,7 +102,7 @@ public void testallocateContainerDistributesAllocation() throws Exception {
ContainerInfo containerInfo = mapping.allocateContainer(
xceiverClientManager.getType(),
xceiverClientManager.getFactor(),
UUID.randomUUID().toString());
UUID.randomUUID().toString(), OzoneProtos.Owner.OZONE);
Assert.assertNotNull(containerInfo);
Assert.assertNotNull(containerInfo.getPipeline());
@ -116,7 +117,8 @@ public void testGetContainer() throws IOException {
String containerName = UUID.randomUUID().toString();
Pipeline pipeline = mapping.allocateContainer(
xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerName).getPipeline();
xceiverClientManager.getFactor(), containerName,
OzoneProtos.Owner.OZONE).getPipeline();
Assert.assertNotNull(pipeline);
Pipeline newPipeline = mapping.getContainer(containerName).getPipeline();
Assert.assertEquals(pipeline.getLeader().getDatanodeUuid(),
@ -128,11 +130,13 @@ public void testDuplicateAllocateContainerFails() throws IOException {
String containerName = UUID.randomUUID().toString();
Pipeline pipeline = mapping.allocateContainer(
xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerName).getPipeline();
xceiverClientManager.getFactor(), containerName,
OzoneProtos.Owner.OZONE).getPipeline();
Assert.assertNotNull(pipeline);
thrown.expectMessage("Specified container already exists.");
mapping.allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerName);
xceiverClientManager.getFactor(), containerName,
OzoneProtos.Owner.OZONE);
}
@Test
@ -148,6 +152,7 @@ public void testChillModeAllocateContainerFails() throws IOException {
nodeManager.setChillmode(true);
thrown.expectMessage("Unable to create container while in chill mode");
mapping.allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerName);
xceiverClientManager.getFactor(), containerName,
OzoneProtos.Owner.OZONE);
}
}

View File

@ -26,6 +26,7 @@
import org.apache.hadoop.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos;
import org.apache.hadoop.ozone.protocol.proto
.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.ozone.scm.container.ContainerMapping;
@ -154,7 +155,8 @@ public void testContainerPlacementCapacity() throws IOException,
String container1 = UUID.randomUUID().toString();
Pipeline pipeline1 = containerManager.allocateContainer(
xceiverClientManager.getType(),
xceiverClientManager.getFactor(), container1).getPipeline();
xceiverClientManager.getFactor(), container1, OzoneProtos.Owner.OZONE)
.getPipeline();
assertEquals(xceiverClientManager.getFactor().getNumber(),
pipeline1.getMachines().size());
@ -184,7 +186,8 @@ public void testContainerPlacementCapacity() throws IOException,
+ "the space requirement"));
String container2 = UUID.randomUUID().toString();
containerManager.allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), container2);
xceiverClientManager.getFactor(), container2,
OzoneProtos.Owner.OZONE);
} finally {
IOUtils.closeQuietly(containerManager);
IOUtils.closeQuietly(nodeManager);

View File

@ -83,7 +83,7 @@ public void defaultTest() throws Exception {
args.toArray(new String[0]));
Assert.assertEquals(2, corona.getNumberOfVolumesCreated());
Assert.assertEquals(10, corona.getNumberOfBucketsCreated());
Assert.assertEquals(100, corona.getNumberOfKeysAdded());
Assert.assertEquals(99, corona.getNumberOfKeysAdded());
Assert.assertEquals(0, res);
}

View File

@ -18,41 +18,66 @@
package org.apache.hadoop.ozone.web.client;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.RatisTestHelper;
import org.apache.hadoop.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
import static org.apache.hadoop.ozone.web.client.TestKeys.*;
import static org.apache.hadoop.ozone.web.client.TestKeys.PutHelper;
import static org.apache.hadoop.ozone.web.client.TestKeys.getMultiPartKey;
import static org.apache.hadoop.ozone.web.client.TestKeys.runTestGetKeyInfo;
import static org.apache.hadoop.ozone.web.client.TestKeys.runTestPutAndDeleteKey;
import static org.apache.hadoop.ozone.web.client.TestKeys.runTestPutAndGetKey;
import static org.apache.hadoop.ozone.web.client.TestKeys.runTestPutAndGetKeyWithDnRestart;
import static org.apache.hadoop.ozone.web.client.TestKeys.runTestPutAndListKey;
import static org.apache.hadoop.ozone.web.client.TestKeys.runTestPutKey;
/** The same as {@link TestKeys} except that this test is Ratis enabled. */
@Ignore("Disabling Ratis tests for pipeline work.")
public class TestKeysRatis {
@Rule
public Timeout testTimeout = new Timeout(300000);
private static RatisTestHelper.RatisTestSuite suite;
private static String path;
private static OzoneRestClient ozoneRestClient;
private static MiniOzoneCluster ozoneCluster = null;
static private String path;
private static OzoneRestClient ozoneRestClient = null;
@BeforeClass
public static void init() throws Exception {
suite = new RatisTestHelper.RatisTestSuite(TestKeysRatis.class);
path = suite.getConf().get(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT);
ozoneRestClient = suite.newOzoneRestClient();
OzoneConfiguration conf = new OzoneConfiguration();
path = GenericTestUtils.getTempPath(TestKeys.class.getSimpleName());
path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
ozoneCluster = new MiniOzoneCluster.Builder(conf)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = ozoneCluster.getDataNodes().get(0);
final int port = dataNode.getInfoPort();
ozoneRestClient = new OzoneRestClient(
String.format("http://localhost:%d", port));
}
/**
* shutdown MiniDFSCluster.
*/
@AfterClass
public static void shutdown() {
if (suite != null) {
suite.close();
if (ozoneCluster != null) {
ozoneCluster.shutdown();
}
}
@Test
public void testPutKey() throws Exception {
runTestPutKey(new PutHelper(ozoneRestClient, path));
@ -64,11 +89,11 @@ public void testPutKey() throws Exception {
@Test
public void testPutAndGetKeyWithDnRestart() throws Exception {
runTestPutAndGetKeyWithDnRestart(
new PutHelper(ozoneRestClient, path), suite.getCluster());
new PutHelper(ozoneRestClient, path), ozoneCluster);
String delimiter = RandomStringUtils.randomAlphanumeric(1);
runTestPutAndGetKeyWithDnRestart(
new PutHelper(ozoneRestClient, path, getMultiPartKey(delimiter)),
suite.getCluster());
ozoneCluster);
}
@Test

View File

@ -18,11 +18,21 @@
package org.apache.hadoop.ozone.web.client;
import org.apache.hadoop.ozone.RatisTestHelper;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.web.exceptions.OzoneException;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.junit.*;
import org.junit.rules.Timeout;
import java.io.File;
import java.io.IOException;
/** The same as {@link TestVolume} except that this test is Ratis enabled. */
@ -30,21 +40,41 @@
public class TestVolumeRatis {
@Rule
public Timeout testTimeout = new Timeout(300000);
private static RatisTestHelper.RatisTestSuite suite;
private static OzoneRestClient ozoneClient;
private static MiniOzoneCluster cluster;
@BeforeClass
public static void init() throws Exception {
// suite = new RatisTestHelper.RatisTestSuite(TestVolumeRatis.class);
// ozoneClient = suite.newOzoneRestClient();
OzoneConfiguration conf = new OzoneConfiguration();
// This enables Ratis in the cluster.
conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_KEY, true);
String path = GenericTestUtils
.getTempPath(TestVolume.class.getSimpleName());
path += conf.getTrimmed(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT,
OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT_DEFAULT);
FileUtils.deleteDirectory(new File(path));
conf.set(OzoneConfigKeys.OZONE_LOCALSTORAGE_ROOT, path);
Logger.getLogger("log4j.logger.org.apache.http").setLevel(Level.DEBUG);
cluster = new MiniOzoneCluster.Builder(conf).numDataNodes(3)
.setHandlerType(OzoneConsts.OZONE_HANDLER_DISTRIBUTED).build();
DataNode dataNode = cluster.getDataNodes().get(0);
final int port = dataNode.getInfoPort();
ozoneClient = new OzoneRestClient(
String.format("http://localhost:%d", port));
}
@AfterClass
public static void shutdown() {
if (suite != null) {
suite.close();
if (cluster != null) {
cluster.shutdown();
}
}
@Test