|
@@ -17,6 +17,9 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
|
|
|
+import org.apache.hadoop.fs.BlockLocation;
|
|
|
+import org.apache.hadoop.fs.FileStatus;
|
|
|
+import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
|
|
|
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
|
|
|
import org.apache.hadoop.thirdparty.com.google.common.collect.LinkedListMultimap;
|
|
@@ -2066,4 +2069,56 @@ public class TestBlockManager {
|
|
|
// validateReconstructionWork return false, need to perform resetTargets().
|
|
|
assertNull(work.getTargets());
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test whether the first block report after DataNode restart is completely
|
|
|
+ * processed.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testBlockReportAfterDataNodeRestart() throws Exception {
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .numDataNodes(3).storagesPerDatanode(1).build()) {
|
|
|
+ cluster.waitActive();
|
|
|
+ BlockManager blockManager = cluster.getNamesystem().getBlockManager();
|
|
|
+ DistributedFileSystem fs = cluster.getFileSystem();
|
|
|
+ final Path filePath = new Path("/tmp.txt");
|
|
|
+ final long fileLen = 1L;
|
|
|
+ DFSTestUtil.createFile(fs, filePath, fileLen, (short) 3, 1L);
|
|
|
+ DFSTestUtil.waitForReplication(fs, filePath, (short) 3, 60000);
|
|
|
+ ArrayList<DataNode> datanodes = cluster.getDataNodes();
|
|
|
+ assertEquals(datanodes.size(), 3);
|
|
|
+
|
|
|
+ // Stop RedundancyMonitor.
|
|
|
+ blockManager.setInitializedReplQueues(false);
|
|
|
+
|
|
|
+ // Delete the replica on the first datanode.
|
|
|
+ DataNode dn = datanodes.get(0);
|
|
|
+ int dnIpcPort = dn.getIpcPort();
|
|
|
+ File dnDir = dn.getFSDataset().getVolumeList().get(0).getCurrentDir();
|
|
|
+ String[] children = FileUtil.list(dnDir);
|
|
|
+ for (String s : children) {
|
|
|
+ if (!s.equals("VERSION")) {
|
|
|
+ FileUtil.fullyDeleteContents(new File(dnDir, s));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // The number of replicas is still 3 because the datanode has not sent
|
|
|
+ // a new block report.
|
|
|
+ FileStatus stat = fs.getFileStatus(filePath);
|
|
|
+ BlockLocation[] locs = fs.getFileBlockLocations(stat, 0, stat.getLen());
|
|
|
+ assertEquals(3, locs[0].getHosts().length);
|
|
|
+
|
|
|
+ // Restart the first datanode.
|
|
|
+ cluster.restartDataNode(0, true);
|
|
|
+
|
|
|
+ // Wait for the block report to be processed.
|
|
|
+ cluster.waitDatanodeFullyStarted(cluster.getDataNode(dnIpcPort), 10000);
|
|
|
+ cluster.waitFirstBRCompleted(0, 10000);
|
|
|
+
|
|
|
+ // The replica num should be 2.
|
|
|
+ locs = fs.getFileBlockLocations(stat, 0, stat.getLen());
|
|
|
+ assertEquals(2, locs[0].getHosts().length);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|