/** Test replace datanode on failure. */ @Test public void testReplaceDatanodeOnFailure() throws Exception { final Configuration conf = new HdfsConfiguration(); // do not consider load factor when selecting a data node conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false); // always replace a datanode ReplaceDatanodeOnFailure.write(Policy.ALWAYS, true, conf); final String[] racks = new String[REPLICATION]; Arrays.fill(racks, RACK0); final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).racks(racks).numDataNodes(REPLICATION).build(); try { cluster.waitActive(); final DistributedFileSystem fs = cluster.getFileSystem(); final Path dir = new Path(DIR); final int NUM_WRITERS = 10; final int FIRST_BATCH = 5; final SlowWriter[] slowwriters = new SlowWriter[NUM_WRITERS]; for (int i = 1; i <= slowwriters.length; i++) { // create slow writers in different speed slowwriters[i - 1] = new SlowWriter(fs, new Path(dir, "file" + i), i * 200L); } for (int i = 0; i < FIRST_BATCH; i++) { slowwriters[i].start(); } // Let slow writers write something. // Some of them are too slow and will be not yet started. sleepSeconds(3); // start new datanodes cluster.startDataNodes(conf, 2, true, null, new String[] {RACK1, RACK1}); cluster.waitActive(); // wait for first block reports for up to 10 seconds cluster.waitFirstBRCompleted(0, 10000); // stop an old datanode MiniDFSCluster.DataNodeProperties dnprop = cluster.stopDataNode(AppendTestUtil.nextInt(REPLICATION)); for (int i = FIRST_BATCH; i < slowwriters.length; i++) { slowwriters[i].start(); } waitForBlockReplication(slowwriters); // check replication and interrupt. for (SlowWriter s : slowwriters) { s.checkReplication(); s.interruptRunning(); } // close files for (SlowWriter s : slowwriters) { s.joinAndClose(); } // Verify the file LOG.info("Verify the file"); for (int i = 0; i < slowwriters.length; i++) { LOG.info( slowwriters[i].filepath + ": length=" + fs.getFileStatus(slowwriters[i].filepath).getLen()); FSDataInputStream in = null; try { in = fs.open(slowwriters[i].filepath); for (int j = 0, x; (x = in.read()) != -1; j++) { Assert.assertEquals(j, x); } } finally { IOUtils.closeStream(in); } } } finally { if (cluster != null) { cluster.shutdown(); } } }