Bläddra i källkod

HDDS-847. TestBlockDeletion is failing.
Contributed by Lokesh Jain.

Nanda kumar 6 år sedan
förälder
incheckning
93666087bc

+ 23 - 27
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java

@@ -23,11 +23,11 @@ import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.ratis.util.CheckedConsumer;
 import org.junit.Assert;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.function.Consumer;
 
 /**
  * Helper class for Tests.
@@ -48,23 +48,27 @@ public final class OzoneTestUtils {
    * @return true if close containers is successful.
    * @throws IOException
    */
-  public static boolean closeContainers(
+  public static void closeContainers(
       List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups,
-      StorageContainerManager scm) throws IOException {
-    return performOperationOnKeyContainers((blockID) -> {
-      try {
+      StorageContainerManager scm) throws Exception {
+    performOperationOnKeyContainers((blockID) -> {
+      if (scm.getContainerManager()
+          .getContainer(ContainerID.valueof(blockID.getContainerID()))
+          .getState() == HddsProtos.LifeCycleState.OPEN) {
         scm.getContainerManager()
             .updateContainerState(ContainerID.valueof(blockID.getContainerID()),
                 HddsProtos.LifeCycleEvent.FINALIZE);
+      }
+      if (scm.getContainerManager()
+          .getContainer(ContainerID.valueof(blockID.getContainerID()))
+          .getState() == HddsProtos.LifeCycleState.CLOSING) {
         scm.getContainerManager()
             .updateContainerState(ContainerID.valueof(blockID.getContainerID()),
                 HddsProtos.LifeCycleEvent.CLOSE);
-        Assert.assertFalse(scm.getContainerManager()
-            .getContainer(ContainerID.valueof(
-                blockID.getContainerID())).isOpen());
-      } catch (IOException e) {
-        throw new AssertionError("Failed to close the container", e);
       }
+      Assert.assertFalse(scm.getContainerManager()
+          .getContainer(ContainerID.valueof(blockID.getContainerID()))
+          .isOpen());
     }, omKeyLocationInfoGroups);
   }
 
@@ -74,28 +78,20 @@ public final class OzoneTestUtils {
    *
    * @param consumer Consumer which accepts BlockID as argument.
    * @param omKeyLocationInfoGroups locationInfos for a key.
-   * @return true if consumer is successful.
    * @throws IOException
    */
-  public static boolean performOperationOnKeyContainers(
-      Consumer<BlockID> consumer,
-      List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups) throws IOException {
+  public static void performOperationOnKeyContainers(
+      CheckedConsumer<BlockID, Exception> consumer,
+      List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups) throws Exception {
 
-    try {
-      for (OmKeyLocationInfoGroup omKeyLocationInfoGroup :
-          omKeyLocationInfoGroups) {
-        List<OmKeyLocationInfo> omKeyLocationInfos =
-            omKeyLocationInfoGroup.getLocationList();
-        for (OmKeyLocationInfo omKeyLocationInfo : omKeyLocationInfos) {
-          BlockID blockID = omKeyLocationInfo.getBlockID();
-          consumer.accept(blockID);
-        }
+    for (OmKeyLocationInfoGroup omKeyLocationInfoGroup : omKeyLocationInfoGroups) {
+      List<OmKeyLocationInfo> omKeyLocationInfos =
+          omKeyLocationInfoGroup.getLocationList();
+      for (OmKeyLocationInfo omKeyLocationInfo : omKeyLocationInfos) {
+        BlockID blockID = omKeyLocationInfo.getBlockID();
+        consumer.accept(blockID);
       }
-    } catch (Error e) {
-      e.printStackTrace();
-      return false;
     }
-    return true;
   }
 
 }

+ 28 - 37
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/TestBlockDeletion.java

@@ -120,8 +120,7 @@ public class TestBlockDeletion {
   }
 
   @Test
-  public void testBlockDeletion()
-      throws IOException, InterruptedException, TimeoutException {
+  public void testBlockDeletion() throws Exception {
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
 
@@ -148,7 +147,7 @@ public class TestBlockDeletion {
         om.lookupKey(keyArgs).getKeyLocationVersions();
 
     // verify key blocks were created in DN.
-    Assert.assertTrue(verifyBlocksCreated(omKeyLocationInfoGroupList));
+    verifyBlocksCreated(omKeyLocationInfoGroupList);
     // No containers with deleted blocks
     Assert.assertTrue(containerIdsWithDeletedBlocks.isEmpty());
     // Delete transactionIds for the containers should be 0.
@@ -158,17 +157,21 @@ public class TestBlockDeletion {
     om.deleteKey(keyArgs);
     Thread.sleep(5000);
     // The blocks should not be deleted in the DN as the container is open
-    Assert.assertTrue(!verifyBlocksDeleted(omKeyLocationInfoGroupList));
+    try {
+      verifyBlocksDeleted(omKeyLocationInfoGroupList);
+      Assert.fail("Blocks should not have been deleted");
+    } catch (Throwable e) {
+      Assert.assertTrue(e.getMessage().contains("expected null, but was"));
+      Assert.assertEquals(e.getClass(), AssertionError.class);
+    }
 
     // close the containers which hold the blocks for the key
-    Assert
-        .assertTrue(
-            OzoneTestUtils.closeContainers(omKeyLocationInfoGroupList, scm));
+    OzoneTestUtils.closeContainers(omKeyLocationInfoGroupList, scm);
 
     waitForDatanodeCommandRetry();
     waitForDatanodeBlockDeletionStart();
     // The blocks should be deleted in the DN.
-    Assert.assertTrue(verifyBlocksDeleted(omKeyLocationInfoGroupList));
+    verifyBlocksDeleted(omKeyLocationInfoGroupList);
 
     // Few containers with deleted blocks
     Assert.assertTrue(!containerIdsWithDeletedBlocks.isEmpty());
@@ -288,44 +291,32 @@ public class TestBlockDeletion {
     }
   }
 
-  private boolean verifyBlocksCreated(
-      List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups)
-      throws IOException {
+  private void verifyBlocksCreated(
+      List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups) throws Exception {
     ContainerSet dnContainerSet =
         cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
             .getContainer().getContainerSet();
-    return OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
-      try {
-        MetadataStore db = BlockUtils.getDB((KeyValueContainerData)
-                dnContainerSet.getContainer(blockID.getContainerID())
-                    .getContainerData(), conf);
-        Assert.assertNotNull(db.get(Longs.toByteArray(blockID.getLocalID())));
-      } catch (IOException e) {
-        e.printStackTrace();
-      }
+    OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
+      MetadataStore db = BlockUtils.getDB((KeyValueContainerData) dnContainerSet
+          .getContainer(blockID.getContainerID()).getContainerData(), conf);
+      Assert.assertNotNull(db.get(Longs.toByteArray(blockID.getLocalID())));
     }, omKeyLocationInfoGroups);
   }
 
-  private boolean verifyBlocksDeleted(
-      List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups)
-      throws IOException {
+  private void verifyBlocksDeleted(
+      List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups) throws Exception {
     ContainerSet dnContainerSet =
         cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
             .getContainer().getContainerSet();
-    return OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
-      try {
-        MetadataStore db = BlockUtils.getDB((KeyValueContainerData)
-            dnContainerSet.getContainer(blockID.getContainerID())
-                .getContainerData(), conf);
-        Assert.assertNull(db.get(Longs.toByteArray(blockID.getLocalID())));
-        Assert.assertNull(db.get(DFSUtil.string2Bytes(
-            OzoneConsts.DELETING_KEY_PREFIX + blockID.getLocalID())));
-        Assert.assertNotNull(DFSUtil.string2Bytes(
-            OzoneConsts.DELETED_KEY_PREFIX + blockID.getLocalID()));
-        containerIdsWithDeletedBlocks.add(blockID.getContainerID());
-      } catch (IOException e) {
-        e.printStackTrace();
-      }
+    OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
+      MetadataStore db = BlockUtils.getDB((KeyValueContainerData) dnContainerSet
+          .getContainer(blockID.getContainerID()).getContainerData(), conf);
+      Assert.assertNull(db.get(Longs.toByteArray(blockID.getLocalID())));
+      Assert.assertNull(db.get(DFSUtil.string2Bytes(
+          OzoneConsts.DELETING_KEY_PREFIX + blockID.getLocalID())));
+      Assert.assertNotNull(DFSUtil
+          .string2Bytes(OzoneConsts.DELETED_KEY_PREFIX + blockID.getLocalID()));
+      containerIdsWithDeletedBlocks.add(blockID.getContainerID());
     }, omKeyLocationInfoGroups);
   }
 }