浏览代码

HDFS-12994. TestReconstructStripedFile.testNNSendsErasureCodingTasks fails due to socket timeout. (Contributed by Lei (Eddy) Xu)

Lei Xu 7 年之前
父节点
当前提交
47563d86fe

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java

@@ -37,6 +37,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
@@ -438,7 +439,7 @@ public class TestReconstructStripedFile {
   }
 
   // HDFS-12044
-  @Test(timeout = 60000)
+  @Test(timeout = 120000)
   public void testNNSendsErasureCodingTasks() throws Exception {
     testNNSendsErasureCodingTasks(1);
     testNNSendsErasureCodingTasks(2);
@@ -453,6 +454,9 @@ public class TestReconstructStripedFile {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 20);
     conf.setInt(DFSConfigKeys.DFS_DN_EC_RECONSTRUCTION_THREADS_KEY,
         2);
+    // Set shorter socket timeout, to allow the recovery task to be reschedule,
+    // if it is connecting to a dead DataNode.
+    conf.setInt(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 5 * 1000);
     cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(numDataNodes).build();
     cluster.waitActive();