Просмотр исходного кода

HDFS-13846. Safe blocks counter is not decremented correctly if the block is striped
(Contributed by Kitti Nanasi via Daniel Templeton)

Change-Id: Id41747a67dc946fdf0dbde90643bb1ea7e9e0f70

Daniel Templeton 6 лет назад
Родитель
Сommit
78bd3b1db9

+ 13 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerSafeMode.java

@@ -418,8 +418,10 @@ class BlockManagerSafeMode {
   }
 
   /**
-   * Increment number of safe blocks if current block has reached minimal
-   * replication.
+   * Increment number of safe blocks if the current block is contiguous
+   * and it has reached minimal replication or
+   * if the current block is striped and the number of its actual data blocks
+   * reaches the number of data units specified by the erasure coding policy.
    * If safe mode is not currently on, this is a no-op.
    * @param storageNum  current number of replicas or number of internal blocks
    *                    of a striped block group
@@ -433,9 +435,9 @@ class BlockManagerSafeMode {
       return;
     }
 
-    final int safe = storedBlock.isStriped() ?
+    final int safeNumberOfNodes = storedBlock.isStriped() ?
         ((BlockInfoStriped)storedBlock).getRealDataBlockNum() : safeReplication;
-    if (storageNum == safe) {
+    if (storageNum == safeNumberOfNodes) {
       this.blockSafe++;
 
       // Report startup progress only if we haven't completed startup yet.
@@ -453,8 +455,10 @@ class BlockManagerSafeMode {
   }
 
   /**
-   * Decrement number of safe blocks if current block has fallen below minimal
-   * replication.
+   * Decrement number of safe blocks if the current block is contiguous
+   * and it has just fallen below minimal replication or
+   * if the current block is striped and its actual data blocks has just fallen
+   * below the number of data units specified by erasure coding policy.
    * If safe mode is not currently on, this is a no-op.
    */
   synchronized void decrementSafeBlockCount(BlockInfo b) {
@@ -463,9 +467,11 @@ class BlockManagerSafeMode {
       return;
     }
 
+    final int safeNumberOfNodes = b.isStriped() ?
+        ((BlockInfoStriped)b).getRealDataBlockNum() : safeReplication;
     BlockInfo storedBlock = blockManager.getStoredBlock(b);
     if (storedBlock.isComplete() &&
-        blockManager.countNodes(b).liveReplicas() == safeReplication - 1) {
+        blockManager.countNodes(b).liveReplicas() == safeNumberOfNodes - 1) {
       this.blockSafe--;
       assert blockSafe >= 0;
       checkSafeMode();

+ 63 - 18
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManagerSafeMode.java

@@ -237,15 +237,7 @@ public class TestBlockManagerSafeMode {
       BlockInfo blockInfo = mock(BlockInfo.class);
       doReturn(false).when(blockInfo).isStriped();
       bmSafeMode.incrementSafeBlockCount(1, blockInfo);
-      if (i < BLOCK_THRESHOLD) {
-        assertEquals(i, getblockSafe());
-        assertTrue(bmSafeMode.isInSafeMode());
-      } else {
-        // block manager leaves safe mode if block threshold is met
-        assertFalse(bmSafeMode.isInSafeMode());
-        // the increment will be a no-op if safe mode is OFF
-        assertEquals(BLOCK_THRESHOLD, getblockSafe());
-      }
+      assertSafeModeIsLeftAtThreshold(i);
     }
   }
 
@@ -314,14 +306,35 @@ public class TestBlockManagerSafeMode {
       bmSafeMode.decrementSafeBlockCount(blockInfo);
       bmSafeMode.incrementSafeBlockCount(1, blockInfo);
 
-      if (i < BLOCK_THRESHOLD) {
-        assertEquals(i, getblockSafe());
-        assertTrue(bmSafeMode.isInSafeMode());
-      } else {
-        // block manager leaves safe mode if block threshold is met
-        assertEquals(BLOCK_THRESHOLD, getblockSafe());
-        assertFalse(bmSafeMode.isInSafeMode());
-      }
+      assertSafeModeIsLeftAtThreshold(i);
+    }
+  }
+
+  /**
+   * Test when the block safe increment and decrement interleave
+   * for striped blocks.
+   *
+   * Both the increment and decrement will be a no-op if the safe mode is OFF.
+   * The safe mode status lifecycle: OFF -> PENDING_THRESHOLD -> OFF
+   */
+  @Test(timeout = 30000)
+  public void testIncrementAndDecrementStripedSafeBlockCount() {
+    bmSafeMode.activate(BLOCK_TOTAL);
+    Whitebox.setInternalState(bmSafeMode, "extension", 0);
+
+    // this number is used only by the decrementSafeBlockCount method
+    final int liveReplicasWhenDecrementing = 1;
+    final short realDataBlockNum = 2;
+    mockBlockManagerForStripedBlockSafeDecrement(liveReplicasWhenDecrementing);
+    for (long i = 1; i <= BLOCK_TOTAL; i++) {
+      BlockInfoStriped blockInfo = mock(BlockInfoStriped.class);
+      when(blockInfo.getRealDataBlockNum()).thenReturn(realDataBlockNum);
+
+      bmSafeMode.incrementSafeBlockCount(realDataBlockNum, blockInfo);
+      bmSafeMode.decrementSafeBlockCount(blockInfo);
+      bmSafeMode.incrementSafeBlockCount(realDataBlockNum, blockInfo);
+
+      assertSafeModeIsLeftAtThreshold(i);
     }
   }
 
@@ -508,10 +521,27 @@ public class TestBlockManagerSafeMode {
    */
   private void mockBlockManagerForBlockSafeDecrement() {
     BlockInfo storedBlock = mock(BlockInfo.class);
+    mockBlockManagerForBlockSafeDecrement(storedBlock, 0);
+  }
+
+  /**
+   * Mock block manager internal state for decrement safe block
+   * in case of striped block.
+   */
+  private void mockBlockManagerForStripedBlockSafeDecrement(int liveReplicas) {
+    BlockInfo storedBlock = mock(BlockInfoStriped.class);
+    mockBlockManagerForBlockSafeDecrement(storedBlock, liveReplicas);
+  }
+
+  /**
+   * Mock block manager internal state for decrement safe block.
+   */
+  private void mockBlockManagerForBlockSafeDecrement(BlockInfo storedBlock,
+        int liveReplicas) {
     when(storedBlock.isComplete()).thenReturn(true);
     doReturn(storedBlock).when(bm).getStoredBlock(any(Block.class));
     NumberReplicas numberReplicas = mock(NumberReplicas.class);
-    when(numberReplicas.liveReplicas()).thenReturn(0);
+    when(numberReplicas.liveReplicas()).thenReturn(liveReplicas);
     doReturn(numberReplicas).when(bm).countNodes(any(BlockInfo.class));
   }
 
@@ -552,4 +582,19 @@ public class TestBlockManagerSafeMode {
   private long getblockSafe() {
     return (long)Whitebox.getInternalState(bmSafeMode, "blockSafe");
   }
+
+  private void assertSafeModeIsLeftAtThreshold(long blockIndex) {
+    if (blockIndex < BLOCK_THRESHOLD) {
+      assertEquals("Current block index should be equal to " +
+          "the safe block counter.", blockIndex, getblockSafe());
+      assertTrue("Block Manager should stay in safe mode until " +
+          "the safe block threshold is reached.", bmSafeMode.isInSafeMode());
+    } else {
+      assertEquals("If safe block threshold is reached, safe block " +
+          "counter should not increase further.",
+          BLOCK_THRESHOLD, getblockSafe());
+      assertFalse("Block manager leaves safe mode if block " +
+          "threshold is met.", bmSafeMode.isInSafeMode());
+    }
+  }
 }