HDDS-248. Refactor DatanodeContainerProtocol.proto Contributed by Hanisha Koneru.
This commit is contained in:
parent
feb795b58d
commit
007e6f5113
|
@ -28,6 +28,8 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
|||
.ContainerCommandRequestProto;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
.ContainerCommandResponseProto;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
.CloseContainerRequestProto;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
.DatanodeBlockID;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
|
@ -86,15 +88,18 @@ public final class ContainerProtocolCalls {
|
|||
.newBuilder()
|
||||
.setBlockID(datanodeBlockID);
|
||||
String id = xceiverClient.getPipeline().getLeader().getUuidString();
|
||||
|
||||
ContainerCommandRequestProto request = ContainerCommandRequestProto
|
||||
.newBuilder()
|
||||
.setCmdType(Type.GetKey)
|
||||
.setContainerID(datanodeBlockID.getContainerID())
|
||||
.setTraceID(traceID)
|
||||
.setDatanodeUuid(id)
|
||||
.setGetKey(readKeyRequest)
|
||||
.build();
|
||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
||||
validateContainerResponse(response);
|
||||
|
||||
return response.getGetKey();
|
||||
}
|
||||
|
||||
|
@ -118,7 +123,9 @@ public final class ContainerProtocolCalls {
|
|||
String id = xceiverClient.getPipeline().getLeader().getUuidString();
|
||||
ContainerCommandRequestProto request =
|
||||
ContainerCommandRequestProto.newBuilder()
|
||||
.setCmdType(Type.GetCommittedBlockLength).setTraceID(traceID)
|
||||
.setCmdType(Type.GetCommittedBlockLength)
|
||||
.setContainerID(blockID.getContainerID())
|
||||
.setTraceID(traceID)
|
||||
.setDatanodeUuid(id)
|
||||
.setGetCommittedBlockLength(getBlockLengthRequestBuilder).build();
|
||||
ContainerCommandResponseProto response = xceiverClient.sendCommand(request);
|
||||
|
@ -143,6 +150,7 @@ public final class ContainerProtocolCalls {
|
|||
ContainerCommandRequestProto request = ContainerCommandRequestProto
|
||||
.newBuilder()
|
||||
.setCmdType(Type.PutKey)
|
||||
.setContainerID(containerKeyData.getBlockID().getContainerID())
|
||||
.setTraceID(traceID)
|
||||
.setDatanodeUuid(id)
|
||||
.setPutKey(createKeyRequest)
|
||||
|
@ -171,6 +179,7 @@ public final class ContainerProtocolCalls {
|
|||
ContainerCommandRequestProto request = ContainerCommandRequestProto
|
||||
.newBuilder()
|
||||
.setCmdType(Type.ReadChunk)
|
||||
.setContainerID(blockID.getContainerID())
|
||||
.setTraceID(traceID)
|
||||
.setDatanodeUuid(id)
|
||||
.setReadChunk(readChunkRequest)
|
||||
|
@ -202,6 +211,7 @@ public final class ContainerProtocolCalls {
|
|||
ContainerCommandRequestProto request = ContainerCommandRequestProto
|
||||
.newBuilder()
|
||||
.setCmdType(Type.WriteChunk)
|
||||
.setContainerID(blockID.getContainerID())
|
||||
.setTraceID(traceID)
|
||||
.setDatanodeUuid(id)
|
||||
.setWriteChunk(writeChunkRequest)
|
||||
|
@ -250,6 +260,7 @@ public final class ContainerProtocolCalls {
|
|||
ContainerCommandRequestProto request =
|
||||
ContainerCommandRequestProto.newBuilder()
|
||||
.setCmdType(Type.PutSmallFile)
|
||||
.setContainerID(blockID.getContainerID())
|
||||
.setTraceID(traceID)
|
||||
.setDatanodeUuid(id)
|
||||
.setPutSmallFile(putSmallFileRequest)
|
||||
|
@ -270,7 +281,6 @@ public final class ContainerProtocolCalls {
|
|||
ContainerProtos.CreateContainerRequestProto.Builder createRequest =
|
||||
ContainerProtos.CreateContainerRequestProto
|
||||
.newBuilder();
|
||||
createRequest.setContainerID(containerID);
|
||||
createRequest.setContainerType(ContainerProtos.ContainerType
|
||||
.KeyValueContainer);
|
||||
|
||||
|
@ -278,6 +288,7 @@ public final class ContainerProtocolCalls {
|
|||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.CreateContainer);
|
||||
request.setContainerID(containerID);
|
||||
request.setCreateContainer(createRequest.build());
|
||||
request.setDatanodeUuid(id);
|
||||
request.setTraceID(traceID);
|
||||
|
@ -298,12 +309,13 @@ public final class ContainerProtocolCalls {
|
|||
boolean force, String traceID) throws IOException {
|
||||
ContainerProtos.DeleteContainerRequestProto.Builder deleteRequest =
|
||||
ContainerProtos.DeleteContainerRequestProto.newBuilder();
|
||||
deleteRequest.setContainerID(containerID);
|
||||
deleteRequest.setForceDelete(force);
|
||||
String id = client.getPipeline().getLeader().getUuidString();
|
||||
|
||||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.DeleteContainer);
|
||||
request.setContainerID(containerID);
|
||||
request.setDeleteContainer(deleteRequest);
|
||||
request.setTraceID(traceID);
|
||||
request.setDatanodeUuid(id);
|
||||
|
@ -322,15 +334,13 @@ public final class ContainerProtocolCalls {
|
|||
*/
|
||||
public static void closeContainer(XceiverClientSpi client,
|
||||
long containerID, String traceID) throws IOException {
|
||||
ContainerProtos.CloseContainerRequestProto.Builder closeRequest =
|
||||
ContainerProtos.CloseContainerRequestProto.newBuilder();
|
||||
closeRequest.setContainerID(containerID);
|
||||
|
||||
String id = client.getPipeline().getLeader().getUuidString();
|
||||
|
||||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(Type.CloseContainer);
|
||||
request.setCloseContainer(closeRequest);
|
||||
request.setContainerID(containerID);
|
||||
request.setCloseContainer(CloseContainerRequestProto.getDefaultInstance());
|
||||
request.setTraceID(traceID);
|
||||
request.setDatanodeUuid(id);
|
||||
ContainerCommandResponseProto response =
|
||||
|
@ -348,19 +358,19 @@ public final class ContainerProtocolCalls {
|
|||
public static ReadContainerResponseProto readContainer(
|
||||
XceiverClientSpi client, long containerID,
|
||||
String traceID) throws IOException {
|
||||
ReadContainerRequestProto.Builder readRequest =
|
||||
ReadContainerRequestProto.newBuilder();
|
||||
readRequest.setContainerID(containerID);
|
||||
String id = client.getPipeline().getLeader().getUuidString();
|
||||
|
||||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(Type.ReadContainer);
|
||||
request.setReadContainer(readRequest);
|
||||
request.setContainerID(containerID);
|
||||
request.setReadContainer(ReadContainerRequestProto.getDefaultInstance());
|
||||
request.setDatanodeUuid(id);
|
||||
request.setTraceID(traceID);
|
||||
ContainerCommandResponseProto response =
|
||||
client.sendCommand(request.build());
|
||||
validateContainerResponse(response);
|
||||
|
||||
return response.getReadContainer();
|
||||
}
|
||||
|
||||
|
@ -383,15 +393,18 @@ public final class ContainerProtocolCalls {
|
|||
.newBuilder().setKey(getKey)
|
||||
.build();
|
||||
String id = client.getPipeline().getLeader().getUuidString();
|
||||
|
||||
ContainerCommandRequestProto request = ContainerCommandRequestProto
|
||||
.newBuilder()
|
||||
.setCmdType(Type.GetSmallFile)
|
||||
.setContainerID(blockID.getContainerID())
|
||||
.setTraceID(traceID)
|
||||
.setDatanodeUuid(id)
|
||||
.setGetSmallFile(getSmallFileRequest)
|
||||
.build();
|
||||
ContainerCommandResponseProto response = client.sendCommand(request);
|
||||
validateContainerResponse(response);
|
||||
|
||||
return response.getGetSmallFile();
|
||||
}
|
||||
|
||||
|
|
|
@ -165,80 +165,81 @@ enum ContainerLifeCycleState {
|
|||
}
|
||||
|
||||
message ContainerCommandRequestProto {
|
||||
required Type cmdType = 1; // Type of the command
|
||||
required Type cmdType = 1; // Type of the command
|
||||
|
||||
// A string that identifies this command, we generate Trace ID in Ozone
|
||||
// frontend and this allows us to trace that command all over ozone.
|
||||
optional string traceID = 2;
|
||||
optional string traceID = 2;
|
||||
|
||||
required int64 containerID = 3;
|
||||
required string datanodeUuid = 4;
|
||||
|
||||
// One of the following command is available when the corresponding
|
||||
// cmdType is set. At the protocol level we allow only
|
||||
// one command in each packet.
|
||||
// TODO : Upgrade to Protobuf 2.6 or later.
|
||||
optional CreateContainerRequestProto createContainer = 3;
|
||||
optional ReadContainerRequestProto readContainer = 4;
|
||||
optional UpdateContainerRequestProto updateContainer = 5;
|
||||
optional DeleteContainerRequestProto deleteContainer = 6;
|
||||
optional ListContainerRequestProto listContainer = 7;
|
||||
optional CreateContainerRequestProto createContainer = 5;
|
||||
optional ReadContainerRequestProto readContainer = 6;
|
||||
optional UpdateContainerRequestProto updateContainer = 7;
|
||||
optional DeleteContainerRequestProto deleteContainer = 8;
|
||||
optional ListContainerRequestProto listContainer = 9;
|
||||
optional CloseContainerRequestProto closeContainer = 10;
|
||||
|
||||
optional PutKeyRequestProto putKey = 8;
|
||||
optional GetKeyRequestProto getKey = 9;
|
||||
optional DeleteKeyRequestProto deleteKey = 10;
|
||||
optional ListKeyRequestProto listKey = 11;
|
||||
optional PutKeyRequestProto putKey = 11;
|
||||
optional GetKeyRequestProto getKey = 12;
|
||||
optional DeleteKeyRequestProto deleteKey = 13;
|
||||
optional ListKeyRequestProto listKey = 14;
|
||||
|
||||
optional ReadChunkRequestProto readChunk = 12;
|
||||
optional WriteChunkRequestProto writeChunk = 13;
|
||||
optional DeleteChunkRequestProto deleteChunk = 14;
|
||||
optional ListChunkRequestProto listChunk = 15;
|
||||
optional ReadChunkRequestProto readChunk = 15;
|
||||
optional WriteChunkRequestProto writeChunk = 16;
|
||||
optional DeleteChunkRequestProto deleteChunk = 17;
|
||||
optional ListChunkRequestProto listChunk = 18;
|
||||
|
||||
optional PutSmallFileRequestProto putSmallFile = 16;
|
||||
optional GetSmallFileRequestProto getSmallFile = 17;
|
||||
optional CloseContainerRequestProto closeContainer = 18;
|
||||
optional GetCommittedBlockLengthRequestProto getCommittedBlockLength = 19;
|
||||
required string datanodeUuid = 20;
|
||||
optional PutSmallFileRequestProto putSmallFile = 19;
|
||||
optional GetSmallFileRequestProto getSmallFile = 20;
|
||||
|
||||
optional GetCommittedBlockLengthRequestProto getCommittedBlockLength = 21;
|
||||
}
|
||||
|
||||
message ContainerCommandResponseProto {
|
||||
required Type cmdType = 1;
|
||||
optional string traceID = 2;
|
||||
required Type cmdType = 1;
|
||||
optional string traceID = 2;
|
||||
|
||||
optional CreateContainerResponseProto createContainer = 3;
|
||||
optional ReadContainerResponseProto readContainer = 4;
|
||||
optional UpdateContainerResponseProto updateContainer = 5;
|
||||
optional DeleteContainerResponseProto deleteContainer = 6;
|
||||
optional ListContainerResponseProto listContainer = 7;
|
||||
required Result result = 3;
|
||||
optional string message = 4;
|
||||
|
||||
optional PutKeyResponseProto putKey = 8;
|
||||
optional GetKeyResponseProto getKey = 9;
|
||||
optional DeleteKeyResponseProto deleteKey = 10;
|
||||
optional ListKeyResponseProto listKey = 11;
|
||||
optional CreateContainerResponseProto createContainer = 5;
|
||||
optional ReadContainerResponseProto readContainer = 6;
|
||||
optional UpdateContainerResponseProto updateContainer = 7;
|
||||
optional DeleteContainerResponseProto deleteContainer = 8;
|
||||
optional ListContainerResponseProto listContainer = 9;
|
||||
optional CloseContainerResponseProto closeContainer = 10;
|
||||
|
||||
optional WriteChunkResponseProto writeChunk = 12;
|
||||
optional ReadChunkResponseProto readChunk = 13;
|
||||
optional DeleteChunkResponseProto deleteChunk = 14;
|
||||
optional ListChunkResponseProto listChunk = 15;
|
||||
optional PutKeyResponseProto putKey = 11;
|
||||
optional GetKeyResponseProto getKey = 12;
|
||||
optional DeleteKeyResponseProto deleteKey = 13;
|
||||
optional ListKeyResponseProto listKey = 14;
|
||||
|
||||
required Result result = 17;
|
||||
optional string message = 18;
|
||||
optional WriteChunkResponseProto writeChunk = 15;
|
||||
optional ReadChunkResponseProto readChunk = 16;
|
||||
optional DeleteChunkResponseProto deleteChunk = 17;
|
||||
optional ListChunkResponseProto listChunk = 18;
|
||||
|
||||
optional PutSmallFileResponseProto putSmallFile = 19;
|
||||
optional GetSmallFileResponseProto getSmallFile = 20;
|
||||
optional CloseContainerResponseProto closeContainer = 21;
|
||||
optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 22;
|
||||
optional PutSmallFileResponseProto putSmallFile = 19;
|
||||
optional GetSmallFileResponseProto getSmallFile = 20;
|
||||
|
||||
optional GetCommittedBlockLengthResponseProto getCommittedBlockLength = 21;
|
||||
}
|
||||
|
||||
message ContainerData {
|
||||
required int64 containerID = 1;
|
||||
repeated KeyValue metadata = 2;
|
||||
optional string dbPath = 3;
|
||||
optional string containerPath = 4;
|
||||
optional int64 bytesUsed = 6;
|
||||
optional int64 size = 7;
|
||||
optional int64 keyCount = 8;
|
||||
optional int64 blockCount = 8;
|
||||
optional ContainerLifeCycleState state = 9 [default = OPEN];
|
||||
optional ContainerType containerType = 10 [default = KeyValueContainer];
|
||||
optional string containerDBType = 11;
|
||||
}
|
||||
|
||||
enum ContainerType {
|
||||
|
@ -248,7 +249,6 @@ enum ContainerType {
|
|||
|
||||
// Container Messages.
|
||||
message CreateContainerRequestProto {
|
||||
required int64 containerID = 1;
|
||||
repeated KeyValue metadata = 2;
|
||||
optional ContainerType containerType = 3 [default = KeyValueContainer];
|
||||
}
|
||||
|
@ -257,7 +257,6 @@ message CreateContainerResponseProto {
|
|||
}
|
||||
|
||||
message ReadContainerRequestProto {
|
||||
required int64 containerID = 1;
|
||||
}
|
||||
|
||||
message ReadContainerResponseProto {
|
||||
|
@ -265,7 +264,6 @@ message ReadContainerResponseProto {
|
|||
}
|
||||
|
||||
message UpdateContainerRequestProto {
|
||||
required int64 containerID = 1;
|
||||
repeated KeyValue metadata = 2;
|
||||
optional bool forceUpdate = 3 [default = false];
|
||||
}
|
||||
|
@ -274,7 +272,6 @@ message UpdateContainerResponseProto {
|
|||
}
|
||||
|
||||
message DeleteContainerRequestProto {
|
||||
required int64 containerID = 1;
|
||||
optional bool forceDelete = 2 [default = false];
|
||||
}
|
||||
|
||||
|
@ -282,7 +279,6 @@ message DeleteContainerResponseProto {
|
|||
}
|
||||
|
||||
message ListContainerRequestProto {
|
||||
required int64 startContainerID = 1;
|
||||
optional uint32 count = 2; // Max Results to return
|
||||
}
|
||||
|
||||
|
@ -291,7 +287,6 @@ message ListContainerResponseProto {
|
|||
}
|
||||
|
||||
message CloseContainerRequestProto {
|
||||
required int64 containerID = 1;
|
||||
}
|
||||
|
||||
message CloseContainerResponseProto {
|
||||
|
@ -341,7 +336,6 @@ message DeleteKeyResponseProto {
|
|||
}
|
||||
|
||||
message ListKeyRequestProto {
|
||||
required int64 containerID = 1;
|
||||
optional int64 startLocalID = 2;
|
||||
required uint32 count = 3;
|
||||
|
||||
|
|
|
@ -98,13 +98,16 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|||
long startTime = System.nanoTime();
|
||||
ContainerProtos.Type cmdType = msg.getCmdType();
|
||||
try {
|
||||
long containerID = getContainerID(msg);
|
||||
long containerID = msg.getContainerID();
|
||||
|
||||
metrics.incContainerOpsMetrics(cmdType);
|
||||
if (cmdType != ContainerProtos.Type.CreateContainer) {
|
||||
container = getContainer(containerID);
|
||||
containerType = getContainerType(container);
|
||||
} else {
|
||||
if (!msg.hasCreateContainer()) {
|
||||
return ContainerUtils.malformedRequest(msg);
|
||||
}
|
||||
containerType = msg.getCreateContainer().getContainerType();
|
||||
}
|
||||
} catch (StorageContainerException ex) {
|
||||
|
@ -143,52 +146,6 @@ public class HddsDispatcher implements ContainerDispatcher {
|
|||
}
|
||||
}
|
||||
|
||||
private long getContainerID(ContainerCommandRequestProto request)
|
||||
throws StorageContainerException {
|
||||
ContainerProtos.Type cmdType = request.getCmdType();
|
||||
|
||||
switch(cmdType) {
|
||||
case CreateContainer:
|
||||
return request.getCreateContainer().getContainerID();
|
||||
case ReadContainer:
|
||||
return request.getReadContainer().getContainerID();
|
||||
case UpdateContainer:
|
||||
return request.getUpdateContainer().getContainerID();
|
||||
case DeleteContainer:
|
||||
return request.getDeleteContainer().getContainerID();
|
||||
case ListContainer:
|
||||
return request.getListContainer().getStartContainerID();
|
||||
case CloseContainer:
|
||||
return request.getCloseContainer().getContainerID();
|
||||
case PutKey:
|
||||
return request.getPutKey().getKeyData().getBlockID().getContainerID();
|
||||
case GetKey:
|
||||
return request.getGetKey().getBlockID().getContainerID();
|
||||
case DeleteKey:
|
||||
return request.getDeleteKey().getBlockID().getContainerID();
|
||||
case ListKey:
|
||||
return request.getListKey().getContainerID();
|
||||
case ReadChunk:
|
||||
return request.getReadChunk().getBlockID().getContainerID();
|
||||
case DeleteChunk:
|
||||
return request.getDeleteChunk().getBlockID().getContainerID();
|
||||
case WriteChunk:
|
||||
return request.getWriteChunk().getBlockID().getContainerID();
|
||||
case ListChunk:
|
||||
return request.getListChunk().getBlockID().getContainerID();
|
||||
case PutSmallFile:
|
||||
return request.getPutSmallFile().getKey().getKeyData().getBlockID()
|
||||
.getContainerID();
|
||||
case GetSmallFile:
|
||||
return request.getGetSmallFile().getKey().getBlockID().getContainerID();
|
||||
case GetCommittedBlockLength:
|
||||
return request.getGetCommittedBlockLength().getBlockID().getContainerID();
|
||||
}
|
||||
|
||||
throw new StorageContainerException(
|
||||
ContainerProtos.Result.UNSUPPORTED_REQUEST);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Container getContainer(long containerID)
|
||||
throws StorageContainerException {
|
||||
|
|
|
@ -76,14 +76,12 @@ public class CloseContainerCommandHandler implements CommandHandler {
|
|||
HddsProtos.ReplicationType replicationType =
|
||||
closeContainerProto.getReplicationType();
|
||||
|
||||
ContainerProtos.CloseContainerRequestProto.Builder closeRequest =
|
||||
ContainerProtos.CloseContainerRequestProto.newBuilder();
|
||||
closeRequest.setContainerID(containerID);
|
||||
|
||||
ContainerProtos.ContainerCommandRequestProto.Builder request =
|
||||
ContainerProtos.ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.CloseContainer);
|
||||
request.setCloseContainer(closeRequest);
|
||||
request.setContainerID(containerID);
|
||||
request.setCloseContainer(
|
||||
ContainerProtos.CloseContainerRequestProto.getDefaultInstance());
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(
|
||||
context.getParent().getDatanodeDetails().getUuidString());
|
||||
|
|
|
@ -207,8 +207,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
|
||||
private CompletableFuture<Message> handleCreateContainer(
|
||||
ContainerCommandRequestProto requestProto) {
|
||||
long containerID =
|
||||
requestProto.getCreateContainer().getContainerID();
|
||||
long containerID = requestProto.getContainerID();
|
||||
createContainerFutureMap.
|
||||
computeIfAbsent(containerID, k -> new CompletableFuture<>());
|
||||
return CompletableFuture.completedFuture(() -> ByteString.EMPTY);
|
||||
|
@ -264,8 +263,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
} else {
|
||||
Message message = runCommand(requestProto);
|
||||
if (cmdType == ContainerProtos.Type.CreateContainer) {
|
||||
long containerID =
|
||||
requestProto.getCreateContainer().getContainerID();
|
||||
long containerID = requestProto.getContainerID();
|
||||
createContainerFutureMap.remove(containerID).complete(message);
|
||||
}
|
||||
return CompletableFuture.completedFuture(message);
|
||||
|
|
|
@ -177,7 +177,6 @@ public class KeyValueContainerData extends ContainerData {
|
|||
ContainerProtos.ContainerData.Builder builder = ContainerProtos
|
||||
.ContainerData.newBuilder();
|
||||
builder.setContainerID(this.getContainerID());
|
||||
builder.setDbPath(this.getDbFile().getPath());
|
||||
builder.setContainerPath(this.getMetadataPath());
|
||||
builder.setState(this.getState());
|
||||
|
||||
|
@ -196,10 +195,6 @@ public class KeyValueContainerData extends ContainerData {
|
|||
builder.setContainerType(ContainerProtos.ContainerType.KeyValueContainer);
|
||||
}
|
||||
|
||||
if(this.getContainerDBType() != null) {
|
||||
builder.setContainerDBType(containerDBType);
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -239,10 +234,6 @@ public class KeyValueContainerData extends ContainerData {
|
|||
data.setBytesUsed(protoData.getBytesUsed());
|
||||
}
|
||||
|
||||
if(protoData.hasContainerDBType()) {
|
||||
data.setContainerDBType(protoData.getContainerDBType());
|
||||
}
|
||||
|
||||
return data;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -231,13 +231,7 @@ public class KeyValueHandler extends Handler {
|
|||
// container would be created here.
|
||||
Preconditions.checkArgument(kvContainer == null);
|
||||
|
||||
CreateContainerRequestProto createContainerReq =
|
||||
request.getCreateContainer();
|
||||
long containerID = createContainerReq.getContainerID();
|
||||
if (createContainerReq.hasContainerType()) {
|
||||
Preconditions.checkArgument(createContainerReq.getContainerType()
|
||||
.equals(ContainerType.KeyValueContainer));
|
||||
}
|
||||
long containerID = request.getContainerID();
|
||||
|
||||
KeyValueContainerData newContainerData = new KeyValueContainerData(
|
||||
containerID, maxContainerSizeGB);
|
||||
|
@ -381,15 +375,15 @@ public class KeyValueHandler extends Handler {
|
|||
try {
|
||||
checkContainerOpen(kvContainer);
|
||||
|
||||
KeyValueContainerData kvData = kvContainer.getContainerData();
|
||||
|
||||
// remove the container from open block map once, all the blocks
|
||||
// have been committed and the container is closed
|
||||
kvContainer.getContainerData()
|
||||
.setState(ContainerProtos.ContainerLifeCycleState.CLOSING);
|
||||
kvData.setState(ContainerProtos.ContainerLifeCycleState.CLOSING);
|
||||
commitPendingKeys(kvContainer);
|
||||
kvContainer.close();
|
||||
// make sure the the container open keys from BlockMap gets removed
|
||||
openContainerBlockMap.removeContainer(
|
||||
request.getCloseContainer().getContainerID());
|
||||
openContainerBlockMap.removeContainer(kvData.getContainerID());
|
||||
} catch (StorageContainerException ex) {
|
||||
return ContainerUtils.logAndReturnError(LOG, ex, request);
|
||||
} catch (IOException ex) {
|
||||
|
|
|
@ -196,7 +196,7 @@ public class OzoneContainer {
|
|||
ContainerProtos.Type type = request.getCmdType();
|
||||
switch (type) {
|
||||
case CloseContainer:
|
||||
return request.getCloseContainer().getContainerID();
|
||||
return request.getContainerID();
|
||||
// Right now, we handle only closeContainer via queuing it over the
|
||||
// over the XceiVerServer. For all other commands we throw Illegal
|
||||
// argument exception here. Will need to extend the switch cases
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
|||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
|
||||
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
|
||||
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TestRule;
|
||||
|
@ -55,6 +56,7 @@ public class TestKeyValueHandler {
|
|||
private final String baseDir = MiniDFSCluster.getBaseDirectory();
|
||||
private final String volume = baseDir + "disk1";
|
||||
|
||||
private static final long DUMMY_CONTAINER_ID = 9999;
|
||||
|
||||
@Test
|
||||
/**
|
||||
|
@ -74,8 +76,13 @@ public class TestKeyValueHandler {
|
|||
|
||||
// Test Create Container Request handling
|
||||
ContainerCommandRequestProto createContainerRequest =
|
||||
getDummyCommandRequestProto(ContainerProtos.Type.CreateContainer);
|
||||
|
||||
ContainerProtos.ContainerCommandRequestProto.newBuilder()
|
||||
.setCmdType(ContainerProtos.Type.CreateContainer)
|
||||
.setContainerID(DUMMY_CONTAINER_ID)
|
||||
.setDatanodeUuid(DATANODE_UUID)
|
||||
.setCreateContainer(ContainerProtos.CreateContainerRequestProto
|
||||
.getDefaultInstance())
|
||||
.build();
|
||||
dispatcher.dispatch(createContainerRequest);
|
||||
Mockito.verify(handler, times(1)).handleCreateContainer(
|
||||
any(ContainerCommandRequestProto.class), any());
|
||||
|
@ -191,6 +198,7 @@ public class TestKeyValueHandler {
|
|||
ContainerCommandRequestProto request =
|
||||
ContainerProtos.ContainerCommandRequestProto.newBuilder()
|
||||
.setCmdType(cmdType)
|
||||
.setContainerID(DUMMY_CONTAINER_ID)
|
||||
.setDatanodeUuid(DATANODE_UUID)
|
||||
.build();
|
||||
|
||||
|
|
|
@ -81,7 +81,6 @@ public class InfoContainerHandler extends OzoneCommandHandler {
|
|||
containerData.getState() == ContainerLifeCycleState.OPEN ? "OPEN" :
|
||||
"CLOSED";
|
||||
logOut("Container State: %s", openStatus);
|
||||
logOut("Container DB Path: %s", containerData.getDbPath());
|
||||
logOut("Container Path: %s", containerData.getContainerPath());
|
||||
|
||||
// Output meta data.
|
||||
|
|
|
@ -220,6 +220,7 @@ public final class ContainerTestHelper {
|
|||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.WriteChunk);
|
||||
request.setContainerID(blockID.getContainerID());
|
||||
request.setWriteChunk(writeRequest);
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
|
@ -261,6 +262,7 @@ public final class ContainerTestHelper {
|
|||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.PutSmallFile);
|
||||
request.setContainerID(blockID.getContainerID());
|
||||
request.setPutSmallFile(smallFileRequest);
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
|
@ -279,6 +281,7 @@ public final class ContainerTestHelper {
|
|||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.GetSmallFile);
|
||||
request.setContainerID(getKey.getGetKey().getBlockID().getContainerID());
|
||||
request.setGetSmallFile(smallFileRequest);
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
|
@ -308,6 +311,7 @@ public final class ContainerTestHelper {
|
|||
ContainerCommandRequestProto.Builder newRequest =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
newRequest.setCmdType(ContainerProtos.Type.ReadChunk);
|
||||
newRequest.setContainerID(readRequest.getBlockID().getContainerID());
|
||||
newRequest.setReadChunk(readRequest);
|
||||
newRequest.setTraceID(UUID.randomUUID().toString());
|
||||
newRequest.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
|
@ -340,6 +344,7 @@ public final class ContainerTestHelper {
|
|||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.DeleteChunk);
|
||||
request.setContainerID(writeRequest.getBlockID().getContainerID());
|
||||
request.setDeleteChunk(deleteRequest);
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
|
@ -356,15 +361,12 @@ public final class ContainerTestHelper {
|
|||
long containerID, Pipeline pipeline) throws IOException {
|
||||
LOG.trace("addContainer: {}", containerID);
|
||||
|
||||
ContainerProtos.CreateContainerRequestProto.Builder createRequest =
|
||||
ContainerProtos.CreateContainerRequestProto
|
||||
.newBuilder();
|
||||
createRequest.setContainerID(containerID);
|
||||
|
||||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.CreateContainer);
|
||||
request.setCreateContainer(createRequest);
|
||||
request.setContainerID(containerID);
|
||||
request.setCreateContainer(
|
||||
ContainerProtos.CreateContainerRequestProto.getDefaultInstance());
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
|
||||
|
@ -385,7 +387,6 @@ public final class ContainerTestHelper {
|
|||
long containerID, Map<String, String> metaData) throws IOException {
|
||||
ContainerProtos.UpdateContainerRequestProto.Builder updateRequestBuilder =
|
||||
ContainerProtos.UpdateContainerRequestProto.newBuilder();
|
||||
updateRequestBuilder.setContainerID(containerID);
|
||||
String[] keys = metaData.keySet().toArray(new String[]{});
|
||||
for(int i=0; i<keys.length; i++) {
|
||||
KeyValue.Builder kvBuilder = KeyValue.newBuilder();
|
||||
|
@ -399,6 +400,7 @@ public final class ContainerTestHelper {
|
|||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.UpdateContainer);
|
||||
request.setContainerID(containerID);
|
||||
request.setUpdateContainer(updateRequestBuilder.build());
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
|
@ -412,14 +414,13 @@ public final class ContainerTestHelper {
|
|||
*/
|
||||
public static ContainerCommandResponseProto
|
||||
getCreateContainerResponse(ContainerCommandRequestProto request) {
|
||||
ContainerProtos.CreateContainerResponseProto.Builder createResponse =
|
||||
ContainerProtos.CreateContainerResponseProto.newBuilder();
|
||||
|
||||
ContainerCommandResponseProto.Builder response =
|
||||
ContainerCommandResponseProto.newBuilder();
|
||||
response.setCmdType(ContainerProtos.Type.CreateContainer);
|
||||
response.setTraceID(request.getTraceID());
|
||||
response.setCreateContainer(createResponse.build());
|
||||
response.setCreateContainer(
|
||||
ContainerProtos.CreateContainerResponseProto.getDefaultInstance());
|
||||
response.setResult(ContainerProtos.Result.SUCCESS);
|
||||
return response.build();
|
||||
}
|
||||
|
@ -448,6 +449,7 @@ public final class ContainerTestHelper {
|
|||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.PutKey);
|
||||
request.setContainerID(keyData.getContainerID());
|
||||
request.setPutKey(putRequest);
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
|
@ -474,6 +476,7 @@ public final class ContainerTestHelper {
|
|||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.GetKey);
|
||||
request.setContainerID(blockID.getContainerID());
|
||||
request.setGetKey(getRequest);
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
|
@ -501,14 +504,16 @@ public final class ContainerTestHelper {
|
|||
*/
|
||||
public static ContainerCommandRequestProto getDeleteKeyRequest(
|
||||
Pipeline pipeline, ContainerProtos.PutKeyRequestProto putKeyRequest) {
|
||||
LOG.trace("deleteKey: name={}",
|
||||
putKeyRequest.getKeyData().getBlockID());
|
||||
ContainerProtos.DatanodeBlockID blockID = putKeyRequest.getKeyData()
|
||||
.getBlockID();
|
||||
LOG.trace("deleteKey: name={}", blockID);
|
||||
ContainerProtos.DeleteKeyRequestProto.Builder delRequest =
|
||||
ContainerProtos.DeleteKeyRequestProto.newBuilder();
|
||||
delRequest.setBlockID(putKeyRequest.getKeyData().getBlockID());
|
||||
delRequest.setBlockID(blockID);
|
||||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.DeleteKey);
|
||||
request.setContainerID(blockID.getContainerID());
|
||||
request.setDeleteKey(delRequest);
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
|
@ -523,12 +528,12 @@ public final class ContainerTestHelper {
|
|||
*/
|
||||
public static ContainerCommandRequestProto getCloseContainer(
|
||||
Pipeline pipeline, long containerID) {
|
||||
ContainerProtos.CloseContainerRequestProto closeRequest =
|
||||
ContainerProtos.CloseContainerRequestProto.newBuilder().
|
||||
setContainerID(containerID).build();
|
||||
ContainerProtos.ContainerCommandRequestProto cmd =
|
||||
ContainerCommandRequestProto.newBuilder().setCmdType(ContainerProtos
|
||||
.Type.CloseContainer).setCloseContainer(closeRequest)
|
||||
ContainerCommandRequestProto.newBuilder()
|
||||
.setCmdType(ContainerProtos.Type.CloseContainer)
|
||||
.setContainerID(containerID)
|
||||
.setCloseContainer(
|
||||
ContainerProtos.CloseContainerRequestProto.getDefaultInstance())
|
||||
.setTraceID(UUID.randomUUID().toString())
|
||||
.setDatanodeUuid(pipeline.getLeader().getUuidString())
|
||||
.build();
|
||||
|
@ -545,14 +550,14 @@ public final class ContainerTestHelper {
|
|||
public static ContainerCommandRequestProto getRequestWithoutTraceId(
|
||||
Pipeline pipeline, long containerID) {
|
||||
Preconditions.checkNotNull(pipeline);
|
||||
ContainerProtos.CloseContainerRequestProto closeRequest =
|
||||
ContainerProtos.CloseContainerRequestProto.newBuilder().
|
||||
setContainerID(containerID).build();
|
||||
ContainerProtos.ContainerCommandRequestProto cmd =
|
||||
ContainerCommandRequestProto.newBuilder().setCmdType(ContainerProtos
|
||||
.Type.CloseContainer).setCloseContainer(closeRequest)
|
||||
.setDatanodeUuid(pipeline.getLeader().getUuidString())
|
||||
.build();
|
||||
ContainerCommandRequestProto.newBuilder()
|
||||
.setCmdType(ContainerProtos.Type.CloseContainer)
|
||||
.setContainerID(containerID)
|
||||
.setCloseContainer(
|
||||
ContainerProtos.CloseContainerRequestProto.getDefaultInstance())
|
||||
.setDatanodeUuid(pipeline.getLeader().getUuidString())
|
||||
.build();
|
||||
return cmd;
|
||||
}
|
||||
|
||||
|
@ -566,10 +571,12 @@ public final class ContainerTestHelper {
|
|||
Preconditions.checkNotNull(pipeline);
|
||||
ContainerProtos.DeleteContainerRequestProto deleteRequest =
|
||||
ContainerProtos.DeleteContainerRequestProto.newBuilder().
|
||||
setContainerID(containerID).
|
||||
setForceDelete(forceDelete).build();
|
||||
return ContainerCommandRequestProto.newBuilder()
|
||||
.setCmdType(ContainerProtos.Type.DeleteContainer)
|
||||
.setContainerID(containerID)
|
||||
.setDeleteContainer(
|
||||
ContainerProtos.DeleteContainerRequestProto.getDefaultInstance())
|
||||
.setDeleteContainer(deleteRequest)
|
||||
.setTraceID(UUID.randomUUID().toString())
|
||||
.setDatanodeUuid(pipeline.getLeader().getUuidString())
|
||||
|
|
|
@ -108,16 +108,14 @@ public class TestCloseContainerHandler {
|
|||
|
||||
private long createContainer() {
|
||||
long testContainerId = ContainerTestHelper.getTestContainerID();
|
||||
ContainerProtos.CreateContainerRequestProto createReq =
|
||||
ContainerProtos.CreateContainerRequestProto.newBuilder()
|
||||
.setContainerID(testContainerId)
|
||||
.build();
|
||||
|
||||
ContainerProtos.ContainerCommandRequestProto request =
|
||||
ContainerProtos.ContainerCommandRequestProto.newBuilder()
|
||||
.setCmdType(ContainerProtos.Type.CreateContainer)
|
||||
.setContainerID(testContainerId)
|
||||
.setDatanodeUuid(DATANODE_UUID)
|
||||
.setCreateContainer(createReq)
|
||||
.setCreateContainer(ContainerProtos.CreateContainerRequestProto
|
||||
.getDefaultInstance())
|
||||
.build();
|
||||
|
||||
dispatcher.dispatch(request);
|
||||
|
@ -143,6 +141,7 @@ public class TestCloseContainerHandler {
|
|||
ContainerProtos.ContainerCommandRequestProto.Builder request =
|
||||
ContainerProtos.ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.WriteChunk);
|
||||
request.setContainerID(blockID.getContainerID());
|
||||
request.setWriteChunk(writeRequest);
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
|
@ -177,6 +176,7 @@ public class TestCloseContainerHandler {
|
|||
ContainerProtos.ContainerCommandRequestProto.Builder request =
|
||||
ContainerProtos.ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.PutKey);
|
||||
request.setContainerID(blockID.getContainerID());
|
||||
request.setPutKey(putKeyRequestProto);
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
|
@ -213,6 +213,7 @@ public class TestCloseContainerHandler {
|
|||
ContainerProtos.ContainerCommandRequestProto.Builder request =
|
||||
ContainerProtos.ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.DeleteChunk);
|
||||
request.setContainerID(blockID.getContainerID());
|
||||
request.setDeleteChunk(deleteChunkProto);
|
||||
request.setWriteChunk(writeRequest);
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
|
@ -242,13 +243,12 @@ public class TestCloseContainerHandler {
|
|||
.get(blockID.getLocalID()));
|
||||
Assert.assertTrue(
|
||||
keyData.getChunks().size() == chunkList.size());
|
||||
ContainerProtos.CloseContainerRequestProto.Builder closeContainerProto =
|
||||
ContainerProtos.CloseContainerRequestProto.newBuilder();
|
||||
closeContainerProto.setContainerID(blockID.getContainerID());
|
||||
ContainerProtos.ContainerCommandRequestProto.Builder request =
|
||||
ContainerProtos.ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.CloseContainer);
|
||||
request.setCloseContainer(closeContainerProto);
|
||||
request.setContainerID(blockID.getContainerID());
|
||||
request.setCloseContainer(
|
||||
ContainerProtos.CloseContainerRequestProto.getDefaultInstance());
|
||||
request.setTraceID(UUID.randomUUID().toString());
|
||||
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
|
||||
dispatcher.dispatch(request.build());
|
||||
|
|
|
@ -50,8 +50,6 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
.ContainerCommandRequestProto;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
.CreateContainerRequestProto;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
.ReadChunkRequestProto;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
|
@ -156,15 +154,14 @@ public class BenchMarkDatanodeDispatcher {
|
|||
FileUtils.deleteDirectory(new File(baseDir));
|
||||
}
|
||||
|
||||
private ContainerCommandRequestProto getCreateContainerCommand(long containerID) {
|
||||
CreateContainerRequestProto.Builder createRequest =
|
||||
CreateContainerRequestProto.newBuilder();
|
||||
createRequest.setContainerID(containerID).build();
|
||||
|
||||
private ContainerCommandRequestProto getCreateContainerCommand(
|
||||
long containerID) {
|
||||
ContainerCommandRequestProto.Builder request =
|
||||
ContainerCommandRequestProto.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.CreateContainer);
|
||||
request.setCreateContainer(createRequest);
|
||||
request.setContainerID(containerID);
|
||||
request.setCreateContainer(
|
||||
ContainerProtos.CreateContainerRequestProto.getDefaultInstance());
|
||||
request.setDatanodeUuid(datanodeUuid);
|
||||
request.setTraceID(containerID + "-trace");
|
||||
return request.build();
|
||||
|
@ -181,6 +178,7 @@ public class BenchMarkDatanodeDispatcher {
|
|||
ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto
|
||||
.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.WriteChunk)
|
||||
.setContainerID(blockID.getContainerID())
|
||||
.setTraceID(getBlockTraceID(blockID))
|
||||
.setDatanodeUuid(datanodeUuid)
|
||||
.setWriteChunk(writeChunkRequest);
|
||||
|
@ -193,9 +191,11 @@ public class BenchMarkDatanodeDispatcher {
|
|||
.newBuilder()
|
||||
.setBlockID(blockID.getDatanodeBlockIDProtobuf())
|
||||
.setChunkData(getChunkInfo(blockID, chunkName));
|
||||
|
||||
ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto
|
||||
.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.ReadChunk)
|
||||
.setContainerID(blockID.getContainerID())
|
||||
.setTraceID(getBlockTraceID(blockID))
|
||||
.setDatanodeUuid(datanodeUuid)
|
||||
.setReadChunk(readChunkRequest);
|
||||
|
@ -219,9 +219,11 @@ public class BenchMarkDatanodeDispatcher {
|
|||
PutKeyRequestProto.Builder putKeyRequest = PutKeyRequestProto
|
||||
.newBuilder()
|
||||
.setKeyData(getKeyData(blockID, chunkKey));
|
||||
|
||||
ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto
|
||||
.newBuilder();
|
||||
request.setCmdType(ContainerProtos.Type.PutKey)
|
||||
.setContainerID(blockID.getContainerID())
|
||||
.setTraceID(getBlockTraceID(blockID))
|
||||
.setDatanodeUuid(datanodeUuid)
|
||||
.setPutKey(putKeyRequest);
|
||||
|
@ -234,6 +236,7 @@ public class BenchMarkDatanodeDispatcher {
|
|||
ContainerCommandRequestProto.Builder request = ContainerCommandRequestProto
|
||||
.newBuilder()
|
||||
.setCmdType(ContainerProtos.Type.GetKey)
|
||||
.setContainerID(blockID.getContainerID())
|
||||
.setTraceID(getBlockTraceID(blockID))
|
||||
.setDatanodeUuid(datanodeUuid)
|
||||
.setGetKey(readKeyRequest);
|
||||
|
|
Loading…
Reference in New Issue