Return-Path: X-Original-To: apmail-hadoop-hdfs-commits-archive@minotaur.apache.org Delivered-To: apmail-hadoop-hdfs-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 723B110DD5 for ; Fri, 11 Apr 2014 03:48:58 +0000 (UTC) Received: (qmail 84776 invoked by uid 500); 11 Apr 2014 03:48:57 -0000 Delivered-To: apmail-hadoop-hdfs-commits-archive@hadoop.apache.org Received: (qmail 84497 invoked by uid 500); 11 Apr 2014 03:48:55 -0000 Mailing-List: contact hdfs-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hdfs-dev@hadoop.apache.org Delivered-To: mailing list hdfs-commits@hadoop.apache.org Received: (qmail 84489 invoked by uid 99); 11 Apr 2014 03:48:54 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 11 Apr 2014 03:48:54 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 11 Apr 2014 03:48:52 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id A98D5238889B; Fri, 11 Apr 2014 03:48:29 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1586551 - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: CHANGES.txt src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java src/test/java/org/apache/hadoop/hdfs/TestPread.java Date: Fri, 11 Apr 2014 03:48:29 -0000 To: hdfs-commits@hadoop.apache.org From: cnauroth@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20140411034829.A98D5238889B@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: cnauroth Date: Fri Apr 11 03:48:29 2014 New Revision: 1586551 URL: http://svn.apache.org/r1586551 Log: HDFS-6231. DFSClient hangs infinitely if using hedged reads and all eligible datanodes die. Contributed by Chris Nauroth. Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1586551&r1=1586550&r2=1586551&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Fri Apr 11 03:48:29 2014 @@ -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 Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java?rev=1586551&r1=1586550&r2=1586551&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java (original) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java Fri Apr 11 03:48:29 2014 @@ -983,12 +983,15 @@ implements ByteBufferReadable, CanSetDro return new Callable() { @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, CanSetDro long end, byte[] buf, int offset, Map> corruptedBlockMap) throws IOException { - ArrayList> futures = null; + ArrayList> futures = new ArrayList>(); ArrayList ignored = new ArrayList(); ByteBuffer bb = null; int len = (int) (end - start + 1); @@ -1112,7 +1115,7 @@ implements ByteBufferReadable, CanSetDro DNAddrPair chosenNode = null; Future 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, CanSetDro // Ignore this node on next go around. ignored.add(chosenNode.info); dfsClient.getHedgedReadMetrics().incHedgedReadOps(); - futures = new ArrayList>(); futures.add(future); continue; // no need to refresh block locations } catch (InterruptedException e) { Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java?rev=1586551&r1=1586550&r2=1586551&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java (original) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java Fri Apr 11 03:48:29 2014 @@ -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");