|
@@ -21,6 +21,8 @@ package org.apache.hadoop.ozone.container.common.transport.server.ratis;
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import org.apache.hadoop.hdds.HddsUtils;
|
|
|
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
|
|
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
|
|
|
import org.apache.ratis.protocol.RaftGroup;
|
|
|
import org.apache.ratis.protocol.RaftGroupId;
|
|
|
import org.apache.ratis.server.RaftServer;
|
|
@@ -506,17 +508,36 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|
|
// on a container
|
|
|
|
|
|
private CompletableFuture<Message> handlePutBlock(
|
|
|
- ContainerCommandRequestProto requestProto) {
|
|
|
+ ContainerCommandRequestProto requestProto, long index) {
|
|
|
List<CompletableFuture<Message>> futureList = new ArrayList<>();
|
|
|
- long localId =
|
|
|
- requestProto.getPutBlock().getBlockData().getBlockID().getLocalID();
|
|
|
+ BlockData blockData = null;
|
|
|
+ ContainerProtos.BlockData blockDataProto =
|
|
|
+ requestProto.getPutBlock().getBlockData();
|
|
|
+
|
|
|
+ // set the blockCommitSequenceId
|
|
|
+ try {
|
|
|
+ blockData = BlockData.getFromProtoBuf(blockDataProto);
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.error("unable to retrieve blockData info for Block {}",
|
|
|
+ blockDataProto.getBlockID());
|
|
|
+ return completeExceptionally(ioe);
|
|
|
+ }
|
|
|
+ blockData.setBlockCommitSequenceId(index);
|
|
|
+ final ContainerProtos.PutBlockRequestProto putBlockRequestProto =
|
|
|
+ ContainerProtos.PutBlockRequestProto
|
|
|
+ .newBuilder(requestProto.getPutBlock())
|
|
|
+ .setBlockData(blockData.getProtoBufMessage()).build();
|
|
|
+ ContainerCommandRequestProto containerCommandRequestProto =
|
|
|
+ ContainerCommandRequestProto.newBuilder(requestProto)
|
|
|
+ .setPutBlock(putBlockRequestProto).build();
|
|
|
+ long localId = blockDataProto.getBlockID().getLocalID();
|
|
|
// Need not wait for create container future here as it has already
|
|
|
// finished.
|
|
|
if (block2ChunkMap.get(localId) != null) {
|
|
|
futureList.addAll(block2ChunkMap.get(localId).getAll());
|
|
|
}
|
|
|
CompletableFuture<Message> effectiveFuture =
|
|
|
- runCommandAfterFutures(futureList, requestProto);
|
|
|
+ runCommandAfterFutures(futureList, containerCommandRequestProto);
|
|
|
|
|
|
CompletableFuture<Message> putBlockFuture =
|
|
|
effectiveFuture.thenApply(message -> {
|
|
@@ -616,7 +637,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|
|
case CloseContainer:
|
|
|
return handleCloseContainer(requestProto);
|
|
|
case PutBlock:
|
|
|
- return handlePutBlock(requestProto);
|
|
|
+ return handlePutBlock(requestProto, index);
|
|
|
case CreateContainer:
|
|
|
return handleCreateContainer(requestProto);
|
|
|
default:
|