Return-Path: X-Original-To: apmail-hadoop-hdfs-dev-archive@minotaur.apache.org Delivered-To: apmail-hadoop-hdfs-dev-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id F3AF71172A for ; Mon, 9 Jun 2014 20:47:16 +0000 (UTC) Received: (qmail 44500 invoked by uid 500); 9 Jun 2014 20:47:16 -0000 Delivered-To: apmail-hadoop-hdfs-dev-archive@hadoop.apache.org Received: (qmail 44401 invoked by uid 500); 9 Jun 2014 20:47:16 -0000 Mailing-List: contact hdfs-dev-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-dev@hadoop.apache.org Received: (qmail 44390 invoked by uid 99); 9 Jun 2014 20:47:16 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 09 Jun 2014 20:47:16 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of yuzhihong@gmail.com designates 209.85.213.41 as permitted sender) Received: from [209.85.213.41] (HELO mail-yh0-f41.google.com) (209.85.213.41) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 09 Jun 2014 20:47:11 +0000 Received: by mail-yh0-f41.google.com with SMTP id f73so3055014yha.14 for ; Mon, 09 Jun 2014 13:46:51 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; bh=YsdIPGTE3HZO2WV8uA6QMeZvVHhFeYpJs1BqjvS2m3Y=; b=u1AiQOXunmz//EUm171n2WKf2EHpNL09PSAmu0CP4HEsfV+6oUvE1vCWbe3H/VtyHX T/HyUASySpludH8Gn9g7nP7kF8Nk/y4unAqhF6ax6f5/sQefBR+jg4VmzXyhvR6vnjaN m9wuCQmE2hJjbrFzTzOFN4YFJF4Bm/iJm4GFibpkj7UaNdeYtwvN1nWs38Ea1Nidx8d2 lRiAD8oKhYlnekakQYILZdUJpM0rMk0rHxV9bY62wzIpRk3FMSHa0GCuEIWwmcIqbvc3 0oLMyzGkJLkAfQCINZvPkJxASyyUfuUFH3CRmpiJRwgfP+0YHFsdC3qXSN7L/+pfsIKT hoNQ== MIME-Version: 1.0 X-Received: by 10.236.45.10 with SMTP id o10mr16868086yhb.49.1402346811022; Mon, 09 Jun 2014 13:46:51 -0700 (PDT) Received: by 10.170.55.213 with HTTP; Mon, 9 Jun 2014 13:46:50 -0700 (PDT) In-Reply-To: References: Date: Mon, 9 Jun 2014 13:46:50 -0700 Message-ID: Subject: Re: hedged read bug From: Ted Yu To: "hdfs-dev@hadoop.apache.org" Content-Type: multipart/alternative; boundary=089e011615fc85e3ce04fb6d4f83 X-Virus-Checked: Checked by ClamAV on apache.org --089e011615fc85e3ce04fb6d4f83 Content-Type: text/plain; charset=UTF-8 Lei: If you can attach the test code from your first email to HDFS-6494, that would help us know the scenario you were referring to. Cheers On Mon, Jun 9, 2014 at 12:06 PM, Chris Nauroth wrote: > Hi Lei, > > I just reviewed this code path on trunk again, and I couldn't find a > problem. It appears to me that if one future fails, then the exception > handling logic will allow the other future to proceed without canceling. > Also, I haven't been able to reproduce the infinite loop that you reported > with the test case that you gave. > > However, if you're still seeing a bug on your side, then I recommend filing > a new jira issue with a full description. We can continue troubleshooting > there. > > Chris Nauroth > Hortonworks > http://hortonworks.com/ > > > > On Sun, Jun 8, 2014 at 8:16 PM, lei liu wrote: > > > Hi Chris, > > > > I review the patch, I think there is problem in the patch. > > > > Example there are two futures, if the first return futrue is failure > and > > then the the second future will be cancled. > > > > > > 2014-06-07 3:44 GMT+08:00 Chris Nauroth : > > > > > Hello Lei, > > > > > > There is a known bug in 2.4.0 that can cause hedged reads to hang. I > > fixed > > > it in HDFS-6231: > > > > > > https://issues.apache.org/jira/browse/HDFS-6231 > > > > > > This patch will be included in the forthcoming 2.4.1 release. I'm > > curious > > > to see if applying this patch fixes the problem for you. Can you try > it > > > and let us know? Thank you! > > > > > > Chris Nauroth > > > Hortonworks > > > http://hortonworks.com/ > > > > > > > > > > > > On Thu, Jun 5, 2014 at 8:34 PM, lei liu wrote: > > > > > > > I use hadoop2.4. > > > > > > > > When I use "hedged read", If there is only one live datanode, the > > reading > > > > from the datanode throw TimeoutException and ChecksumException., the > > > > Client will infinite wait. > > > > > > > > Example below test case: > > > > @Test > > > > public void testException() throws IOException, > InterruptedException, > > > > ExecutionException { > > > > Configuration conf = new Configuration(); > > > > int numHedgedReadPoolThreads = 5; > > > > final int hedgedReadTimeoutMillis = 50; > > > > > > conf.setInt(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE, > > > > numHedgedReadPoolThreads); > > > > > > > conf.setLong(DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS, > > > > hedgedReadTimeoutMillis); > > > > // Set up the InjectionHandler > > > > DFSClientFaultInjector.instance = > > > > Mockito.mock(DFSClientFaultInjector.class); > > > > DFSClientFaultInjector injector = > DFSClientFaultInjector.instance; > > > > // make preads ChecksumException > > > > Mockito.doAnswer(new Answer() { > > > > @Override > > > > public Void answer(InvocationOnMock invocation) throws > Throwable > > { > > > > if(true) { > > > > Thread.sleep(hedgedReadTimeoutMillis + 10); > > > > throw new ChecksumException("test", 100); > > > > } > > > > return null; > > > > } > > > > }*).when(injector).fetchFromDatanodeException();* > > > > > > > > MiniDFSCluster cluster = new > > > > MiniDFSCluster.Builder(conf).numDataNodes(3).format(true).build(); > > > > DistributedFileSystem fileSys = cluster.getFileSystem(); > > > > DFSClient dfsClient = fileSys.getClient(); > > > > DFSHedgedReadMetrics metrics = dfsClient.getHedgedReadMetrics(); > > > > > > > > try { > > > > Path file = new Path("/hedgedReadException.dat"); > > > > FSDataOutputStream output = fileSys.create(file,(short)1); > > > > byte[] data = new byte[64 * 1024]; > > > > output.write(data); > > > > output.flush(); > > > > output.write(data); > > > > output.flush(); > > > > output.write(data); > > > > output.flush(); > > > > output.close(); > > > > byte[] buffer = new byte[64 * 1024]; > > > > FSDataInputStream input = fileSys.open(file); > > > > input.read(0, buffer, 0, 1024); > > > > input.close(); > > > > assertTrue(metrics.getHedgedReadOps() == 1); > > > > assertTrue(metrics.getHedgedReadWins() == 1); > > > > } finally { > > > > fileSys.close(); > > > > cluster.shutdown(); > > > > Mockito.reset(injector); > > > > } > > > > } > > > > > > > > > > > > *The code of actualGetFromOneDataNode() method call > > > > **fetchFromDatanodeException() > > > > method as below:* > > > > try { > > > > *DFSClientFaultInjector.get().fetchFromDatanodeException();* > > > > Token blockToken = > block.getBlockToken(); > > > > int len = (int) (end - start + 1); > > > > reader = new BlockReaderFactory(dfsClient.getConf()). > > > > setInetSocketAddress(targetAddr). > > > > setRemotePeerFactory(dfsClient). > > > > setDatanodeInfo(chosenNode). > > > > setFileName(src). > > > > setBlock(block.getBlock()). > > > > setBlockToken(blockToken). > > > > setStartOffset(start). > > > > setVerifyChecksum(verifyChecksum). > > > > setClientName(dfsClient.clientName). > > > > setLength(len). > > > > setCachingStrategy(curCachingStrategy). > > > > > > setAllowShortCircuitLocalReads(allowShortCircuitLocalReads). > > > > setClientCacheContext(dfsClient.getClientContext()). > > > > setUserGroupInformation(dfsClient.ugi). > > > > setConfiguration(dfsClient.getConfiguration()). > > > > build(); > > > > int nread = reader.readAll(buf, offset, len); > > > > if (nread != len) { > > > > throw new IOException("truncated return from reader.read(): > > " + > > > > "excpected " + len + ", got " + > nread); > > > > } > > > > return; > > > > } catch (ChecksumException e) { > > > > String msg = "fetchBlockByteRange(). Got a checksum exception > > > for " > > > > + src + " at " + block.getBlock() + ":" + e.getPos() + " > > > from " > > > > + chosenNode; > > > > DFSClient.LOG.warn(msg); > > > > // we want to remember what we have tried > > > > addIntoCorruptedBlockMap(block.getBlock(), chosenNode, > > > > corruptedBlockMap); > > > > addToDeadNodes(chosenNode); > > > > throw new IOException(msg); > > > > } > > > > > > > > > > -- > > > CONFIDENTIALITY NOTICE > > > NOTICE: This message is intended for the use of the individual or > entity > > to > > > which it is addressed and may contain information that is confidential, > > > privileged and exempt from disclosure under applicable law. If the > reader > > > of this message is not the intended recipient, you are hereby notified > > that > > > any printing, copying, dissemination, distribution, disclosure or > > > forwarding of this communication is strictly prohibited. If you have > > > received this communication in error, please contact the sender > > immediately > > > and delete it from your system. Thank You. > > > > > > > -- > CONFIDENTIALITY NOTICE > NOTICE: This message is intended for the use of the individual or entity to > which it is addressed and may contain information that is confidential, > privileged and exempt from disclosure under applicable law. If the reader > of this message is not the intended recipient, you are hereby notified that > any printing, copying, dissemination, distribution, disclosure or > forwarding of this communication is strictly prohibited. If you have > received this communication in error, please contact the sender immediately > and delete it from your system. Thank You. > --089e011615fc85e3ce04fb6d4f83--