|
@@ -33,13 +33,16 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.AppendTestUtil;
|
|
|
+import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.HAUtil;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNodeAdapter;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
@@ -131,6 +134,81 @@ public class TestStandbyIsHot {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Regression test for HDFS-2795:
|
|
|
+ * - Start an HA cluster with a DN.
|
|
|
+ * - Write several blocks to the FS with replication 1.
|
|
|
+ * - Shutdown the DN
|
|
|
+ * - Wait for the NNs to declare the DN dead. All blocks will be under-replicated.
|
|
|
+ * - Restart the DN.
|
|
|
+ * In the bug, the standby node would only very slowly notice the blocks returning
|
|
|
+ * to the cluster.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testDatanodeRestarts() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024);
|
|
|
+ // We read from the standby to watch block locations
|
|
|
+ HAUtil.setAllowStandbyReads(conf, true);
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .nnTopology(MiniDFSNNTopology.simpleHATopology())
|
|
|
+ .numDataNodes(1)
|
|
|
+ .build();
|
|
|
+ try {
|
|
|
+ NameNode nn0 = cluster.getNameNode(0);
|
|
|
+ NameNode nn1 = cluster.getNameNode(1);
|
|
|
+ nn1.getNamesystem().getEditLogTailer().setSleepTime(250);
|
|
|
+ nn1.getNamesystem().getEditLogTailer().interrupt();
|
|
|
+
|
|
|
+ cluster.transitionToActive(0);
|
|
|
+
|
|
|
+ // Create 5 blocks.
|
|
|
+ DFSTestUtil.createFile(cluster.getFileSystem(0),
|
|
|
+ TEST_FILE_PATH, 5*1024, (short)1, 1L);
|
|
|
+
|
|
|
+ HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
|
|
|
+
|
|
|
+ // Stop the DN.
|
|
|
+ DataNode dn = cluster.getDataNodes().get(0);
|
|
|
+ String dnName = dn.getDatanodeId().getName();
|
|
|
+ DataNodeProperties dnProps = cluster.stopDataNode(0);
|
|
|
+
|
|
|
+ // Make sure both NNs register it as dead.
|
|
|
+ BlockManagerTestUtil.noticeDeadDatanode(nn0, dnName);
|
|
|
+ BlockManagerTestUtil.noticeDeadDatanode(nn1, dnName);
|
|
|
+
|
|
|
+ BlockManagerTestUtil.updateState(nn0.getNamesystem().getBlockManager());
|
|
|
+ BlockManagerTestUtil.updateState(nn1.getNamesystem().getBlockManager());
|
|
|
+ assertEquals(5, nn0.getNamesystem().getUnderReplicatedBlocks());
|
|
|
+
|
|
|
+ // The SBN will not have any blocks in its neededReplication queue
|
|
|
+ // since the SBN doesn't process replication.
|
|
|
+ assertEquals(0, nn1.getNamesystem().getUnderReplicatedBlocks());
|
|
|
+
|
|
|
+ LocatedBlocks locs = nn1.getRpcServer().getBlockLocations(
|
|
|
+ TEST_FILE, 0, 1);
|
|
|
+ assertEquals("Standby should have registered that the block has no replicas",
|
|
|
+ 0, locs.get(0).getLocations().length);
|
|
|
+
|
|
|
+ cluster.restartDataNode(dnProps);
|
|
|
+ // Wait for both NNs to re-register the DN.
|
|
|
+ cluster.waitActive(0);
|
|
|
+ cluster.waitActive(1);
|
|
|
+
|
|
|
+ BlockManagerTestUtil.updateState(nn0.getNamesystem().getBlockManager());
|
|
|
+ BlockManagerTestUtil.updateState(nn1.getNamesystem().getBlockManager());
|
|
|
+ assertEquals(0, nn0.getNamesystem().getUnderReplicatedBlocks());
|
|
|
+ assertEquals(0, nn1.getNamesystem().getUnderReplicatedBlocks());
|
|
|
+
|
|
|
+ locs = nn1.getRpcServer().getBlockLocations(
|
|
|
+ TEST_FILE, 0, 1);
|
|
|
+ assertEquals("Standby should have registered that the block has replicas again",
|
|
|
+ 1, locs.get(0).getLocations().length);
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
static void waitForBlockLocations(final MiniDFSCluster cluster,
|
|
|
final NameNode nn,
|