diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index e2416c2b616..1622ddbf054 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -87,7 +87,7 @@ public class XceiverClientGrpc extends XceiverClientSpi { } LOG.debug("Connecting to server Port : " + leader.getIpAddress()); channel = NettyChannelBuilder.forAddress(leader.getIpAddress(), port) - .usePlaintext(true) + .usePlaintext() .maxInboundMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE) .build(); asyncStub = XceiverClientProtocolServiceGrpc.newStub(channel); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java index f4f3f6f0d60..4dc232db7ba 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java @@ -77,7 +77,7 @@ public final class XceiverServerGrpc implements XceiverServerSpi { datanodeDetails.setPort( DatanodeDetails.newPort(DatanodeDetails.Port.Name.STANDALONE, port)); server = ((NettyServerBuilder) ServerBuilder.forPort(port)) - .maxMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE) + .maxInboundMessageSize(OzoneConfigKeys.DFS_CONTAINER_CHUNK_MAX_SIZE) .addService(new GrpcXceiverService(dispatcher)) .build(); storageContainer = dispatcher; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java index ede87f48400..68d6d5bb144 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java @@ -59,6 +59,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadPoolExecutor; +import java.util.stream.Collectors; /** A {@link org.apache.ratis.statemachine.StateMachine} for containers. * @@ -316,6 +317,23 @@ public class ContainerStateMachine extends BaseStateMachine { return LogEntryProto.newBuilder().setSmLogEntry(log).build(); } + /** + * Returns the combined future of all the writeChunks till the given log + * index. The Raft log worker will wait for the stateMachineData to complete + * flush as well. + * + * @param index log index till which the stateMachine data needs to be flushed + * @return Combined future of all writeChunks till the log index given. + */ + @Override + public CompletableFuture flushStateMachineData(long index) { + List> futureList = + writeChunkFutureMap.entrySet().stream().filter(x -> x.getKey() <= index) + .map(x -> x.getValue()).collect(Collectors.toList()); + CompletableFuture combinedFuture = CompletableFuture.allOf( + futureList.toArray(new CompletableFuture[futureList.size()])); + return combinedFuture; + } /* * This api is used by the leader while appending logs to the follower * This allows the leader to read the state machine data from the diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index b45b4954f6c..18b907fdd31 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -97,7 +97,7 @@ 1.0.0-M33 - 0.3.0-e4a016f-SNAPSHOT + 0.3.0-e6fd494-SNAPSHOT 1.0-alpha-1 3.3.1 2.4.12