Explorar el Código

HDFS-6231. DFSClient hangs infinitely if using hedged reads and all eligible datanodes die. Contributed by Chris Nauroth.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1586551 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth hace 11 años
padre
commit
f8904ad299

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -372,6 +372,9 @@ Release 2.4.1 - UNRELEASED
 
     HDFS-6208. DataNode caching can leak file descriptors. (cnauroth)
 
+    HDFS-6231. DFSClient hangs infinitely if using hedged reads and all eligible
+    datanodes die. (cnauroth)
+
 Release 2.4.0 - 2014-04-07 
 
   INCOMPATIBLE CHANGES

+ 11 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -983,12 +983,15 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     return new Callable<ByteBuffer>() {
       @Override
       public ByteBuffer call() throws Exception {
-        byte[] buf = bb.array();
-        int offset = bb.position();
-        actualGetFromOneDataNode(datanode, block, start, end, buf, offset,
-            corruptedBlockMap);
-        latch.countDown();
-        return bb;
+        try {
+          byte[] buf = bb.array();
+          int offset = bb.position();
+          actualGetFromOneDataNode(datanode, block, start, end, buf, offset,
+              corruptedBlockMap);
+          return bb;
+        } finally {
+          latch.countDown();
+        }
       }
     };
   }
@@ -1101,7 +1104,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       long end, byte[] buf, int offset,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
-    ArrayList<Future<ByteBuffer>> futures = null;
+    ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
     ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
     ByteBuffer bb = null;
     int len = (int) (end - start + 1);
@@ -1112,7 +1115,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       DNAddrPair chosenNode = null;
       Future<ByteBuffer> future = null;
       // futures is null if there is no request already executing.
-      if (futures == null) {
+      if (futures.isEmpty()) {
         // chooseDataNode is a commitment.  If no node, we go to
         // the NN to reget block locations.  Only go here on first read.
         chosenNode = chooseDataNode(block, ignored);
@@ -1130,7 +1133,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // Ignore this node on next go around.
           ignored.add(chosenNode.info);
           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
-          futures = new ArrayList<Future<ByteBuffer>>();
           futures.add(future);
           continue; // no need to refresh block locations
         } catch (InterruptedException e) {

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

@@ -237,7 +237,7 @@ public class TestPread {
   public void testHedgedPreadDFSBasic() throws IOException {
     Configuration conf = new Configuration();
     conf.setInt(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE, 5);
-    conf.setLong(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS, 100);
+    conf.setLong(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS, 1);
     dfsPreadTest(conf, false, true); // normal pread
     dfsPreadTest(conf, true, true); // trigger read code path without
                                     // transferTo.
@@ -273,6 +273,10 @@ public class TestPread {
     DistributedFileSystem fileSys = cluster.getFileSystem();
     DFSClient dfsClient = fileSys.getClient();
     DFSHedgedReadMetrics metrics = dfsClient.getHedgedReadMetrics();
+    // Metrics instance is static, so we need to reset counts from prior tests.
+    metrics.hedgedReadOps.set(0);
+    metrics.hedgedReadOpsWin.set(0);
+    metrics.hedgedReadOpsInCurThread.set(0);
 
     try {
       Path file1 = new Path("hedgedReadMaxOut.dat");