소스 검색

HDFS-17453. IncrementalBlockReport can have race condition with Edit Log Tailer (#6708)

dannytbecker 1 년 전
부모
커밋
05964ad07a

+ 19 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingDataNodeMessages.java

@@ -95,16 +95,30 @@ class PendingDataNodeMessages {
   
   void enqueueReportedBlock(DatanodeStorageInfo storageInfo, Block block,
       ReplicaState reportedState) {
+    if (storageInfo == null || block == null) {
+      return;
+    }
+    block = new Block(block);
+    long genStamp = block.getGenerationStamp();
+    Queue<ReportedBlockInfo> queue = null;
     if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
       Block blkId = new Block(BlockIdManager.convertToStripedID(block
           .getBlockId()));
-      getBlockQueue(blkId).add(
-          new ReportedBlockInfo(storageInfo, new Block(block), reportedState));
+      queue = getBlockQueue(blkId);
     } else {
-      block = new Block(block);
-      getBlockQueue(block).add(
-          new ReportedBlockInfo(storageInfo, block, reportedState));
+      queue = getBlockQueue(block);
+    }
+    // We only want the latest non-future reported block to be queued for each
+    // DataNode. Otherwise, there can be a race condition that causes an old
+    // reported block to be kept in the queue until the SNN switches to ANN and
+    // the old reported block will be processed and marked as corrupt by the ANN.
+    // See HDFS-17453
+    int size = queue.size();
+    if (queue.removeIf(rbi -> storageInfo.equals(rbi.storageInfo) &&
+        rbi.block.getGenerationStamp() <= genStamp)) {
+      count -= (size - queue.size());
     }
+    queue.add(new ReportedBlockInfo(storageInfo, block, reportedState));
     count++;
   }
   

+ 18 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingDataNodeMessages.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
+import java.util.Arrays;
+import java.util.List;
 import java.util.Queue;
 
 import org.apache.hadoop.conf.Configuration;
@@ -52,11 +54,21 @@ public class TestPendingDataNodeMessages {
 
   @Test
   public void testQueues() {
-    DatanodeDescriptor fakeDN = DFSTestUtil.getLocalDatanodeDescriptor();
-    DatanodeStorage storage = new DatanodeStorage("STORAGE_ID");
-    DatanodeStorageInfo storageInfo = new DatanodeStorageInfo(fakeDN, storage);
-    msgs.enqueueReportedBlock(storageInfo, block1Gs1, ReplicaState.FINALIZED);
-    msgs.enqueueReportedBlock(storageInfo, block1Gs2, ReplicaState.FINALIZED);
+    DatanodeDescriptor fakeDN1 = DFSTestUtil.getDatanodeDescriptor(
+        "localhost", 8898, "/default-rack");
+    DatanodeDescriptor fakeDN2 = DFSTestUtil.getDatanodeDescriptor(
+        "localhost", 8899, "/default-rack");
+    DatanodeStorage storage1 = new DatanodeStorage("STORAGE_ID_1");
+    DatanodeStorage storage2 = new DatanodeStorage("STORAGE_ID_2");
+    DatanodeStorageInfo storageInfo1 = new DatanodeStorageInfo(fakeDN1, storage1);
+    DatanodeStorageInfo storageInfo2 = new DatanodeStorageInfo(fakeDN2, storage2);
+    msgs.enqueueReportedBlock(storageInfo1, block1Gs1, ReplicaState.FINALIZED);
+    msgs.enqueueReportedBlock(storageInfo2, block1Gs1, ReplicaState.FINALIZED);
+    msgs.enqueueReportedBlock(storageInfo1, block1Gs2, ReplicaState.FINALIZED);
+    msgs.enqueueReportedBlock(storageInfo2, block1Gs2, ReplicaState.FINALIZED);
+    List<ReportedBlockInfo> rbis = Arrays.asList(
+        new ReportedBlockInfo(storageInfo1, block1Gs2, ReplicaState.FINALIZED),
+        new ReportedBlockInfo(storageInfo2, block1Gs2, ReplicaState.FINALIZED));
 
     assertEquals(2, msgs.count());
     
@@ -66,9 +78,7 @@ public class TestPendingDataNodeMessages {
     
     Queue<ReportedBlockInfo> q =
       msgs.takeBlockQueue(block1Gs2DifferentInstance);
-    assertEquals(
-        "ReportedBlockInfo [block=blk_1_1, dn=127.0.0.1:9866, reportedState=FINALIZED]," +
-        "ReportedBlockInfo [block=blk_1_2, dn=127.0.0.1:9866, reportedState=FINALIZED]",
+    assertEquals(Joiner.on(",").join(rbis),
         Joiner.on(",").join(q));
     assertEquals(0, msgs.count());
     

+ 110 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBlockReports.java

@@ -17,14 +17,25 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyString;
 import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.times;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Phaser;
+import java.util.concurrent.TimeUnit;
 
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.mockito.invocation.InvocationOnMock;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -55,6 +66,9 @@ public class TestIncrementalBlockReports {
   private static final long DUMMY_BLOCK_ID = 5678;
   private static final long DUMMY_BLOCK_LENGTH = 1024 * 1024;
   private static final long DUMMY_BLOCK_GENSTAMP = 1000;
+  private static final String TEST_FILE_DATA = "hello world";
+  private static final String TEST_FILE = "/TestStandbyBlockManagement";
+  private static final Path TEST_FILE_PATH = new Path(TEST_FILE);
 
   private MiniDFSCluster cluster = null;
   private Configuration conf;
@@ -215,4 +229,100 @@ public class TestIncrementalBlockReports {
       cluster = null;
     }
   }
+
+  @Test
+  public void testIBRRaceCondition() throws Exception {
+    cluster.shutdown();
+    conf = new Configuration();
+    HAUtil.setAllowStandbyReads(conf, true);
+    conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology())
+        .numDataNodes(3)
+        .build();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+
+      NameNode nn1 = cluster.getNameNode(0);
+      NameNode nn2 = cluster.getNameNode(1);
+      FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+      List<InvocationOnMock> ibrsToStandby = new ArrayList<>();
+      List<DatanodeProtocolClientSideTranslatorPB> spies = new ArrayList<>();
+      Phaser ibrPhaser = new Phaser(1);
+      for (DataNode dn : cluster.getDataNodes()) {
+        DatanodeProtocolClientSideTranslatorPB nnSpy =
+            InternalDataNodeTestUtils.spyOnBposToNN(dn, nn2);
+        doAnswer((inv) -> {
+          for (StorageReceivedDeletedBlocks srdb :
+              inv.getArgument(2, StorageReceivedDeletedBlocks[].class)) {
+            for (ReceivedDeletedBlockInfo block : srdb.getBlocks()) {
+              if (block.getStatus().equals(BlockStatus.RECEIVED_BLOCK)) {
+                ibrPhaser.arriveAndDeregister();
+              }
+            }
+          }
+          ibrsToStandby.add(inv);
+          return null;
+        }).when(nnSpy).blockReceivedAndDeleted(
+            any(DatanodeRegistration.class),
+            anyString(),
+            any(StorageReceivedDeletedBlocks[].class));
+        spies.add(nnSpy);
+      }
+
+      LOG.info("==================================");
+      // Force the DNs to delay report to the SNN
+      ibrPhaser.bulkRegister(9);
+      DFSTestUtil.writeFile(fs, TEST_FILE_PATH, TEST_FILE_DATA);
+      DFSTestUtil.appendFile(fs, TEST_FILE_PATH, TEST_FILE_DATA);
+      DFSTestUtil.appendFile(fs, TEST_FILE_PATH, TEST_FILE_DATA);
+      HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
+      // SNN has caught up to the latest edit log so we send the IBRs to SNN
+      int phase = ibrPhaser.arrive();
+      ibrPhaser.awaitAdvanceInterruptibly(phase, 60, TimeUnit.SECONDS);
+      for (InvocationOnMock sendIBRs : ibrsToStandby) {
+        try {
+          sendIBRs.callRealMethod();
+        } catch (Throwable t) {
+          LOG.error("Exception thrown while calling sendIBRs: ", t);
+        }
+      }
+
+      assertEquals("There should be 3 pending messages from DNs", 3,
+          nn2.getNamesystem().getBlockManager().getPendingDataNodeMessageCount());
+      ibrsToStandby.clear();
+      // We need to trigger another edit log roll so that the pendingDNMessages
+      // are processed.
+      ibrPhaser.bulkRegister(6);
+      DFSTestUtil.appendFile(fs, TEST_FILE_PATH, TEST_FILE_DATA);
+      DFSTestUtil.appendFile(fs, TEST_FILE_PATH, TEST_FILE_DATA);
+      phase = ibrPhaser.arrive();
+      ibrPhaser.awaitAdvanceInterruptibly(phase, 60, TimeUnit.SECONDS);
+      for (InvocationOnMock sendIBRs : ibrsToStandby) {
+        try {
+          sendIBRs.callRealMethod();
+        } catch (Throwable t) {
+          LOG.error("Exception thrown while calling sendIBRs: ", t);
+        }
+      }
+      ibrsToStandby.clear();
+      ibrPhaser.arriveAndDeregister();
+      ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, TEST_FILE_PATH);
+      HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
+      LOG.info("==================================");
+
+      // Trigger an active switch to force SNN to mark blocks as corrupt if they
+      // have a bad genstamp in the pendingDNMessages queue.
+      cluster.transitionToStandby(0);
+      cluster.transitionToActive(1);
+      cluster.waitActive(1);
+
+      assertEquals("There should not be any corrupt replicas", 0,
+          nn2.getNamesystem().getBlockManager()
+              .numCorruptReplicas(block.getLocalBlock()));
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }

+ 6 - 20
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java

@@ -421,9 +421,6 @@ public class TestDNFencing {
    */
   @Test
   public void testQueueingWithAppend() throws Exception {
-    int numQueued = 0;
-    int numDN = cluster.getDataNodes().size();
-    
     // case 1: create file and call hflush after write
     FSDataOutputStream out = fs.create(TEST_FILE_PATH);
     try {
@@ -436,20 +433,16 @@ public class TestDNFencing {
       // Apply cluster.triggerBlockReports() to trigger the reporting sooner.
       //
       cluster.triggerBlockReports();
-      numQueued += numDN; // RBW messages
 
       // The cluster.triggerBlockReports() call above does a full 
       // block report that incurs 3 extra RBW messages
-      numQueued += numDN; // RBW messages      
     } finally {
       IOUtils.closeStream(out);
-      numQueued += numDN; // blockReceived messages
     }
 
     cluster.triggerBlockReports();
-    numQueued += numDN;
-    assertEquals(numQueued, cluster.getNameNode(1).getNamesystem().
-        getPendingDataNodeMessageCount());
+    assertEquals("The queue should only have the latest report for each DN",
+        3, nn2.getNamesystem().getPendingDataNodeMessageCount());
 
     // case 2: append to file and call hflush after write
     try {
@@ -457,14 +450,12 @@ public class TestDNFencing {
       AppendTestUtil.write(out, 10, 10);
       out.hflush();
       cluster.triggerBlockReports();
-      numQueued += numDN * 2; // RBW messages, see comments in case 1
     } finally {
       IOUtils.closeStream(out);
       cluster.triggerHeartbeats();
-      numQueued += numDN; // blockReceived
     }
-    assertEquals(numQueued, cluster.getNameNode(1).getNamesystem().
-        getPendingDataNodeMessageCount());
+    assertEquals("The queue should only have the latest report for each DN",
+        3, nn2.getNamesystem().getPendingDataNodeMessageCount());
 
     // case 3: similar to case 2, except no hflush is called.
     try {
@@ -483,17 +474,12 @@ public class TestDNFencing {
       //    BPServiceActor#addPendingReplicationBlockInfo 
       //
       IOUtils.closeStream(out);
-      numQueued += numDN; // blockReceived
     }
 
     cluster.triggerBlockReports();
-    numQueued += numDN;
-
-    LOG.info("Expect " + numQueued + " and got: " + cluster.getNameNode(1).getNamesystem().
-        getPendingDataNodeMessageCount());      
 
-    assertEquals(numQueued, cluster.getNameNode(1).getNamesystem().
-        getPendingDataNodeMessageCount());
+    assertEquals("The queue should only have the latest report for each DN",
+        3, nn2.getNamesystem().getPendingDataNodeMessageCount());
 
     cluster.transitionToStandby(0);
     cluster.transitionToActive(1);