HDFS-12387. Ozone: Support Ratis as a first class replication mechanism. Contributed by Anu Engineer.
This commit is contained in:
parent
0b3270a8de
commit
c17521b1bd
@ -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.
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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());
|
||||
}
|
||||
|
@ -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;
|
||||
|
||||
/**
|
||||
|
@ -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.");
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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";
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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) {
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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 {
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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 {
|
||||
|
@ -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;
|
||||
}
|
@ -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;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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];
|
||||
|
||||
}
|
||||
|
||||
|
@ -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 {
|
||||
|
@ -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
|
||||
|
@ -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())
|
||||
|
@ -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());
|
||||
|
@ -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()
|
||||
|
@ -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.
|
||||
*
|
||||
|
@ -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.
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
*/
|
||||
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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.
|
||||
*
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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";
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user