Browse Source

HDDS-185: TestCloseContainerByPipeline#testCloseContainerViaRatis fail intermittently. Contributed by Shashikant Banerjee.

Nanda kumar 7 years ago
parent
commit
852ca6b432

+ 58 - 7
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestCloseContainerByPipeline.java

@@ -23,6 +23,8 @@ import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
@@ -33,6 +35,7 @@ import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientFactory;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
+import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyArgs;
 import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
@@ -45,6 +48,7 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Random;
 import java.util.concurrent.TimeoutException;
 
 public class TestCloseContainerByPipeline {
@@ -88,6 +92,59 @@ public class TestCloseContainerByPipeline {
     }
   }
 
+  @Test
+  public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception {
+    OzoneOutputStream key = objectStore.getVolume("test").getBucket("test")
+        .createKey("testCloseContainer", 1024, ReplicationType.STAND_ALONE,
+            ReplicationFactor.ONE);
+    key.write("standalone".getBytes());
+    key.close();
+
+    //get the name of a valid container
+    KsmKeyArgs keyArgs =
+        new KsmKeyArgs.Builder().setVolumeName("test").setBucketName("test")
+            .setType(HddsProtos.ReplicationType.STAND_ALONE)
+            .setFactor(HddsProtos.ReplicationFactor.ONE).setDataSize(1024)
+            .setKeyName("testCloseContainer").build();
+
+    KsmKeyLocationInfo ksmKeyLocationInfo =
+        cluster.getKeySpaceManager().lookupKey(keyArgs).getKeyLocationVersions()
+            .get(0).getBlocksLatestVersionOnly().get(0);
+
+    long containerID = ksmKeyLocationInfo.getContainerID();
+    List<DatanodeDetails> datanodes =
+        cluster.getStorageContainerManager().getContainerInfo(containerID)
+            .getPipeline().getMachines();
+    Assert.assertTrue(datanodes.size() == 1);
+
+    DatanodeDetails datanodeDetails = datanodes.get(0);
+    HddsDatanodeService datanodeService = null;
+    Assert
+        .assertFalse(isContainerClosed(cluster, containerID, datanodeDetails));
+    for (HddsDatanodeService datanodeServiceItr : cluster.getHddsDatanodes()) {
+      if (datanodeDetails.equals(datanodeServiceItr.getDatanodeDetails())) {
+        datanodeService = datanodeServiceItr;
+        break;
+      }
+    }
+    CommandHandler closeContainerHandler =
+        datanodeService.getDatanodeStateMachine().getCommandDispatcher()
+            .getCloseContainerHandler();
+    int lastInvocationCount = closeContainerHandler.getInvocationCount();
+    //send the order to close the container
+    cluster.getStorageContainerManager().getScmNodeManager()
+        .addDatanodeCommand(datanodeDetails.getUuid(),
+            new CloseContainerCommand(containerID,
+                HddsProtos.ReplicationType.STAND_ALONE));
+    GenericTestUtils
+        .waitFor(() -> isContainerClosed(cluster, containerID, datanodeDetails),
+            500, 5 * 1000);
+    // Make sure the closeContainerCommandHandler is Invoked
+    Assert.assertTrue(
+        closeContainerHandler.getInvocationCount() > lastInvocationCount);
+
+  }
+
   @Test
   public void testCloseContainerViaStandaAlone()
       throws IOException, TimeoutException, InterruptedException {
@@ -205,13 +262,7 @@ public class TestCloseContainerByPipeline {
           containerData =
               datanodeService.getDatanodeStateMachine().getContainer()
                   .getContainerManager().readContainer(containerID);
-          if (!containerData.isOpen()) {
-            // make sure the closeContainerHandler on the Datanode is invoked
-            Assert.assertTrue(
-                datanodeService.getDatanodeStateMachine().getCommandDispatcher()
-                    .getCloseContainerHandler().getInvocationCount() > 0);
-            return true;
-          }
+          return !containerData.isOpen();
         }
     } catch (StorageContainerException e) {
       throw new AssertionError(e);