|
@@ -21,20 +21,27 @@ import static org.junit.Assert.fail;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.io.OutputStream;
|
|
|
+import java.util.List;
|
|
|
+
|
|
|
+import junit.framework.Assert;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
|
|
|
+import org.apache.hadoop.util.ThreadUtil;
|
|
|
+
|
|
|
import org.junit.Test;
|
|
|
|
|
|
|
|
|
/**
|
|
|
- * These tests make sure that DFSClient retries fetching data from DFS
|
|
|
- * properly in case of errors.
|
|
|
+ * These tests make sure that DFSClient excludes writing data to
|
|
|
+ * a DN properly in case of errors.
|
|
|
*/
|
|
|
public class TestDFSClientExcludedNodes {
|
|
|
|
|
|
- @Test
|
|
|
+ @Test(timeout=10000)
|
|
|
public void testExcludedNodes() throws IOException {
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
|
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
|
|
@@ -43,14 +50,89 @@ public class TestDFSClientExcludedNodes {
|
|
|
|
|
|
// kill a datanode
|
|
|
cluster.stopDataNode(AppendTestUtil.nextInt(3));
|
|
|
- OutputStream out = fs.create(filePath, true, 4096);
|
|
|
+ OutputStream out = fs.create(
|
|
|
+ filePath,
|
|
|
+ true,
|
|
|
+ 4096,
|
|
|
+ (short) 3,
|
|
|
+ fs.getDefaultBlockSize(filePath)
|
|
|
+ );
|
|
|
out.write(20);
|
|
|
|
|
|
try {
|
|
|
out.close();
|
|
|
} catch (Exception e) {
|
|
|
- fail("DataNode failure should not result in a block abort: \n" + e.getMessage());
|
|
|
+ fail("Single DN failure should not result in a block abort: \n" +
|
|
|
+ e.getMessage());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test(timeout=10000)
|
|
|
+ public void testExcludedNodesForgiveness() throws IOException {
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ // Forgive nodes in under 1s for this test case.
|
|
|
+ conf.setLong(
|
|
|
+ DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL,
|
|
|
+ 1000);
|
|
|
+ // We'll be using a 512 bytes block size just for tests
|
|
|
+ // so making sure the checksum bytes too match it.
|
|
|
+ conf.setInt("io.bytes.per.checksum", 512);
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
|
|
|
+ List<DataNodeProperties> props = cluster.dataNodes;
|
|
|
+ FileSystem fs = cluster.getFileSystem();
|
|
|
+ Path filePath = new Path("/testForgivingExcludedNodes");
|
|
|
+
|
|
|
+ // 256 bytes data chunk for writes
|
|
|
+ byte[] bytes = new byte[256];
|
|
|
+ for (int index=0; index<bytes.length; index++) {
|
|
|
+ bytes[index] = '0';
|
|
|
+ }
|
|
|
+
|
|
|
+ // File with a 512 bytes block size
|
|
|
+ FSDataOutputStream out = fs.create(filePath, true, 4096, (short) 3, 512);
|
|
|
+
|
|
|
+ // Write a block to all 3 DNs (2x256bytes).
|
|
|
+ out.write(bytes);
|
|
|
+ out.write(bytes);
|
|
|
+ out.hflush();
|
|
|
+
|
|
|
+ // Remove two DNs, to put them into the exclude list.
|
|
|
+ DataNodeProperties two = cluster.stopDataNode(2);
|
|
|
+ DataNodeProperties one = cluster.stopDataNode(1);
|
|
|
+
|
|
|
+ // Write another block.
|
|
|
+ // At this point, we have two nodes already in excluded list.
|
|
|
+ out.write(bytes);
|
|
|
+ out.write(bytes);
|
|
|
+ out.hflush();
|
|
|
+
|
|
|
+ // Bring back the older DNs, since they are gonna be forgiven only
|
|
|
+ // afterwards of this previous block write.
|
|
|
+ Assert.assertEquals(true, cluster.restartDataNode(one, true));
|
|
|
+ Assert.assertEquals(true, cluster.restartDataNode(two, true));
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ // Sleep for 2s, to let the excluded nodes be expired
|
|
|
+ // from the excludes list (i.e. forgiven after the configured wait period).
|
|
|
+ // [Sleeping just in case the restart of the DNs completed < 2s cause
|
|
|
+ // otherwise, we'll end up quickly excluding those again.]
|
|
|
+ ThreadUtil.sleepAtLeastIgnoreInterrupts(2000);
|
|
|
+
|
|
|
+ // Terminate the last good DN, to assert that there's no
|
|
|
+ // single-DN-available scenario, caused by not forgiving the other
|
|
|
+ // two by now.
|
|
|
+ cluster.stopDataNode(0);
|
|
|
+
|
|
|
+ try {
|
|
|
+ // Attempt writing another block, which should still pass
|
|
|
+ // cause the previous two should have been forgiven by now,
|
|
|
+ // while the last good DN added to excludes this time.
|
|
|
+ out.write(bytes);
|
|
|
+ out.hflush();
|
|
|
+ out.close();
|
|
|
+ } catch (Exception e) {
|
|
|
+ fail("Excluded DataNodes should be forgiven after a while and " +
|
|
|
+ "not cause file writing exception of: '" + e.getMessage() + "'");
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
}
|