Prechádzať zdrojové kódy

HDDS-1126. Datanode is trying to qausi-close a container which is already closed.

Signed-off-by: Nanda kumar <nanda@apache.org>
Nanda kumar 6 rokov pred
rodič
commit
2bc3cfe28f

+ 5 - 0
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CloseContainerCommandHandler.java

@@ -86,6 +86,11 @@ public class CloseContainerCommandHandler implements CommandHandler {
         return;
       }
 
+      if (container.getContainerState() == ContainerProtos.ContainerDataProto.State.CLOSED) {
+        // Closing a container is an idempotent operation.
+        return;
+      }
+
       // Move the container to CLOSING state
       controller.markContainerForClose(containerId);
 

+ 41 - 0
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerCommandHandler.java

@@ -215,6 +215,47 @@ public class TestCloseContainerCommandHandler {
     }
   }
 
+  @Test
+  public void testQuasiCloseClosedContainer()
+      throws Exception {
+    final OzoneConfiguration conf = new OzoneConfiguration();
+    final DatanodeDetails datanodeDetails = randomDatanodeDetails();
+    final OzoneContainer ozoneContainer = getOzoneContainer(conf, datanodeDetails);
+    ozoneContainer.start();
+    try {
+      final Container container = createContainer(conf, datanodeDetails, ozoneContainer);
+      Mockito.verify(context.getParent(),
+          Mockito.times(1)).triggerHeartbeat();
+      final long containerId = container.getContainerData().getContainerID();
+      final PipelineID pipelineId = PipelineID.valueOf(UUID.fromString(
+          container.getContainerData().getOriginPipelineId()));
+
+      final CloseContainerCommandHandler closeHandler = new CloseContainerCommandHandler();
+      final CloseContainerCommand closeCommand = new CloseContainerCommand(
+          containerId, pipelineId);
+
+      closeHandler.handle(closeCommand, ozoneContainer, context, null);
+
+      Assert.assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
+          ozoneContainer.getContainerSet().getContainer(containerId)
+              .getContainerState());
+
+      // The container is closed, now we send close command with pipeline id which doesn't exist.
+      // This should cause the datanode to trigger quasi close, since the container is already
+      // closed, this should do nothing. The command should not fail either.
+      final PipelineID randomPipeline = PipelineID.randomId();
+      final CloseContainerCommand quasiCloseCommand = new CloseContainerCommand(
+          containerId, randomPipeline);
+      closeHandler.handle(quasiCloseCommand, ozoneContainer, context, null);
+
+      Assert.assertEquals(ContainerProtos.ContainerDataProto.State.CLOSED,
+          ozoneContainer.getContainerSet().getContainer(containerId)
+              .getContainerState());
+    } finally {
+      ozoneContainer.stop();
+    }
+  }
+
   private OzoneContainer getOzoneContainer(final OzoneConfiguration conf,
       final DatanodeDetails datanodeDetails) throws IOException {
     testDir = GenericTestUtils.getTestDir(