|
@@ -27,14 +27,18 @@ import static org.mockito.Mockito.times;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.LinkedHashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.Phaser;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
|
|
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.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -246,6 +250,7 @@ public class TestIncrementalBlockReports {
|
|
|
|
|
|
NameNode nn1 = cluster.getNameNode(0);
|
|
|
NameNode nn2 = cluster.getNameNode(1);
|
|
|
+ BlockManager bm2 = nn2.getNamesystem().getBlockManager();
|
|
|
FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
|
|
|
List<InvocationOnMock> ibrsToStandby = new ArrayList<>();
|
|
|
List<DatanodeProtocolClientSideTranslatorPB> spies = new ArrayList<>();
|
|
@@ -262,7 +267,6 @@ public class TestIncrementalBlockReports {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
- ibrsToStandby.add(inv);
|
|
|
return null;
|
|
|
}).when(nnSpy).blockReceivedAndDeleted(
|
|
|
any(DatanodeRegistration.class),
|
|
@@ -289,8 +293,9 @@ public class TestIncrementalBlockReports {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- assertEquals("There should be 3 pending messages from DNs", 3,
|
|
|
- nn2.getNamesystem().getBlockManager().getPendingDataNodeMessageCount());
|
|
|
+ GenericTestUtils.waitFor(() -> bm2.getPendingDataNodeMessageCount() == 0,
|
|
|
+ 1000, 30000,
|
|
|
+ "There should be 0 pending DN messages");
|
|
|
ibrsToStandby.clear();
|
|
|
// We need to trigger another edit log roll so that the pendingDNMessages
|
|
|
// are processed.
|
|
@@ -308,6 +313,9 @@ public class TestIncrementalBlockReports {
|
|
|
}
|
|
|
ibrsToStandby.clear();
|
|
|
ibrPhaser.arriveAndDeregister();
|
|
|
+ GenericTestUtils.waitFor(() -> bm2.getPendingDataNodeMessageCount() == 0,
|
|
|
+ 1000, 30000,
|
|
|
+ "There should be 0 pending DN messages");
|
|
|
ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, TEST_FILE_PATH);
|
|
|
HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
|
|
|
LOG.info("==================================");
|
|
@@ -325,4 +333,192 @@ public class TestIncrementalBlockReports {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testIBRRaceCondition2() throws Exception {
|
|
|
+ cluster.shutdown();
|
|
|
+ Configuration 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);
|
|
|
+ BlockManager bm2 = nn2.getNamesystem().getBlockManager();
|
|
|
+ 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)) {
|
|
|
+ ibrsToStandby.add(inv);
|
|
|
+ ibrPhaser.arriveAndDeregister();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ 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);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(() -> bm2.getPendingDataNodeMessageCount() == 0,
|
|
|
+ 1000, 30000,
|
|
|
+ "There should be 0 pending DN messages");
|
|
|
+ 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();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testIBRRaceCondition3() throws Exception {
|
|
|
+ cluster.shutdown();
|
|
|
+ Configuration 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);
|
|
|
+ BlockManager bm2 = nn2.getNamesystem().getBlockManager();
|
|
|
+ FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
|
|
|
+ LinkedHashMap<Long, List<InvocationOnMock>> ibrsToStandby =
|
|
|
+ new LinkedHashMap<>();
|
|
|
+ AtomicLong lowestGenStamp = new AtomicLong(Long.MAX_VALUE);
|
|
|
+ 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)) {
|
|
|
+ long genStamp = block.getBlock().getGenerationStamp();
|
|
|
+ ibrsToStandby.putIfAbsent(genStamp, new ArrayList<>());
|
|
|
+ ibrsToStandby.get(genStamp).add(inv);
|
|
|
+ lowestGenStamp.getAndUpdate((prev) -> Math.min(prev, genStamp));
|
|
|
+ ibrPhaser.arriveAndDeregister();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ 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);
|
|
|
+ ibrsToStandby.forEach((genStamp, ibrs) -> {
|
|
|
+ if (lowestGenStamp.get() != genStamp) {
|
|
|
+ ibrs.removeIf(inv -> {
|
|
|
+ try {
|
|
|
+ inv.callRealMethod();
|
|
|
+ } catch (Throwable t) {
|
|
|
+ LOG.error("Exception thrown while calling sendIBRs: ", t);
|
|
|
+ }
|
|
|
+ return true;
|
|
|
+ });
|
|
|
+ }
|
|
|
+ });
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(() -> bm2.getPendingDataNodeMessageCount() == 0,
|
|
|
+ 1000, 30000,
|
|
|
+ "There should be 0 pending DN messages");
|
|
|
+ ibrPhaser.arriveAndDeregister();
|
|
|
+ ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, TEST_FILE_PATH);
|
|
|
+ HATestUtil.waitForStandbyToCatchUp(nn1, nn2);
|
|
|
+
|
|
|
+ // Send old ibrs to simulate actual stale or corrupt DNs
|
|
|
+ for (InvocationOnMock sendIBR : ibrsToStandby.get(lowestGenStamp.get())) {
|
|
|
+ try {
|
|
|
+ sendIBR.callRealMethod();
|
|
|
+ } catch (Throwable t) {
|
|
|
+ LOG.error("Exception thrown while calling sendIBRs: ", t);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ GenericTestUtils.waitFor(() -> bm2.getPendingDataNodeMessageCount() == 3,
|
|
|
+ 1000, 30000,
|
|
|
+ "There should be 0 pending DN messages");
|
|
|
+ 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 be 1 corrupt replica", 1,
|
|
|
+ nn2.getNamesystem().getBlockManager()
|
|
|
+ .numCorruptReplicas(block.getLocalBlock()));
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|