You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ms...@apache.org on 2018/09/12 13:02:22 UTC

hadoop git commit: HDDS-433. ContainerStateMachine#readStateMachineData should properly build LogEntryProto. Contributed by Lokesh Jain.

Repository: hadoop
Updated Branches:
  refs/heads/trunk 5e64e62de -> 47299ce23


HDDS-433. ContainerStateMachine#readStateMachineData should properly build LogEntryProto. Contributed by Lokesh Jain.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/47299ce2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/47299ce2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/47299ce2

Branch: refs/heads/trunk
Commit: 47299ce23fa0f327b8c532ca96191a571c92d244
Parents: 5e64e62
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Wed Sep 12 18:30:53 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Wed Sep 12 18:31:03 2018 +0530

----------------------------------------------------------------------
 .../transport/server/ratis/ContainerStateMachine.java | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/47299ce2/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java
----------------------------------------------------------------------
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 1636f24..b84db66 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
@@ -268,7 +268,7 @@ public class ContainerStateMachine extends BaseStateMachine {
     }
   }
 
-  private LogEntryProto readStateMachineData(SMLogEntryProto smLogEntryProto,
+  private LogEntryProto readStateMachineData(LogEntryProto entry,
       ContainerCommandRequestProto requestProto) {
     WriteChunkRequestProto writeChunkRequestProto =
         requestProto.getWriteChunk();
@@ -307,18 +307,18 @@ public class ContainerStateMachine extends BaseStateMachine {
         ContainerCommandRequestProto.newBuilder(requestProto)
             .setWriteChunk(dataWriteChunkProto);
 
-    return recreateLogEntryProto(smLogEntryProto,
+    return recreateLogEntryProto(entry,
         newStateMachineProto.build().toByteString());
   }
 
-  private LogEntryProto recreateLogEntryProto(SMLogEntryProto smLogEntryProto,
+  private LogEntryProto recreateLogEntryProto(LogEntryProto entry,
       ByteString stateMachineData) {
     // recreate the log entry
     final SMLogEntryProto log =
-        SMLogEntryProto.newBuilder(smLogEntryProto)
+        SMLogEntryProto.newBuilder(entry.getSmLogEntry())
             .setStateMachineData(stateMachineData)
             .build();
-    return LogEntryProto.newBuilder().setSmLogEntry(log).build();
+    return LogEntryProto.newBuilder(entry).setSmLogEntry(log).build();
   }
 
   /**
@@ -360,11 +360,11 @@ public class ContainerStateMachine extends BaseStateMachine {
 
       if (requestProto.getCmdType() == Type.WriteChunk) {
         return CompletableFuture.supplyAsync(() ->
-                readStateMachineData(smLogEntryProto, requestProto),
+                readStateMachineData(entry, requestProto),
             chunkExecutor);
       } else if (requestProto.getCmdType() == Type.CreateContainer) {
         LogEntryProto log =
-            recreateLogEntryProto(smLogEntryProto, requestProto.toByteString());
+            recreateLogEntryProto(entry, requestProto.toByteString());
         return CompletableFuture.completedFuture(log);
       } else {
         throw new IllegalStateException("Cmd type:" + requestProto.getCmdType()


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org