Return-Path: Delivered-To: apmail-hadoop-core-dev-archive@www.apache.org Received: (qmail 48202 invoked from network); 31 Jan 2008 04:57:56 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 31 Jan 2008 04:57:56 -0000 Received: (qmail 33288 invoked by uid 500); 31 Jan 2008 04:57:46 -0000 Delivered-To: apmail-hadoop-core-dev-archive@hadoop.apache.org Received: (qmail 33270 invoked by uid 500); 31 Jan 2008 04:57:46 -0000 Mailing-List: contact core-dev-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: core-dev@hadoop.apache.org Delivered-To: mailing list core-dev@hadoop.apache.org Received: (qmail 33261 invoked by uid 99); 31 Jan 2008 04:57:46 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 30 Jan 2008 20:57:46 -0800 X-ASF-Spam-Status: No, hits=-100.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO brutus.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 31 Jan 2008 04:57:27 +0000 Received: from brutus (localhost [127.0.0.1]) by brutus.apache.org (Postfix) with ESMTP id 66EF5714204 for ; Wed, 30 Jan 2008 20:57:34 -0800 (PST) Message-ID: <8580913.1201755454380.JavaMail.jira@brutus> Date: Wed, 30 Jan 2008 20:57:34 -0800 (PST) From: "lohit vijayarenu (JIRA)" To: core-dev@hadoop.apache.org Subject: [jira] Commented: (HADOOP-2647) dfs -put hangs MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-Virus-Checked: Checked by ClamAV on apache.org [ https://issues.apache.org/jira/browse/HADOOP-2647?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12564294#action_12564294 ] lohit vijayarenu commented on HADOOP-2647: ------------------------------------------ +1 looks like this will handle the failure and client will not hang Meanwhile {code} if (closed) { - throw new IOException("Stream closed"); + return; // closing multiple times is ok. } {code} in close(), can stream be closed due to other reason? > dfs -put hangs > -------------- > > Key: HADOOP-2647 > URL: https://issues.apache.org/jira/browse/HADOOP-2647 > Project: Hadoop Core > Issue Type: Bug > Components: dfs > Affects Versions: 0.15.1 > Environment: LINUX > Reporter: lohit vijayarenu > Assignee: Raghu Angadi > Attachments: HADOOP-2647.patch > > > We saw a case where dfs -put hung while copying a 2GB file for over 20 hours. > When we took a look at the stack trace of process the main thread was waiting for confirmation from namenode for complete status. > only 4 blocks were copied and 5th block was missing when we ran fsck on the partially transfered file. > There are 2 problems we saw here. > 1. DFS client hung without a timeout when there is no response from namenode. > 2. In IOUtils::copyBytes(InputStream in, OutputStream out, int buffSize, boolean close) > During copy, if there is an exception, the out.close() is called. Exception is not caught. Which is why we see a close call in the stack trace. > When we checked for block IDs in namenode log. For the block which was missing, there was only one response to namenode instead of three. > This close state coupled with namenode not reporting the error back might have cause the whole process to hang. > Opening this JIRA to see if we could add checks to the two problems mentioned above. > > "main" prio=10 tid=0x0805a000 nid=0x5b53 waiting on condition [0xf7e64000..0xf7e65288] java.lang.Thread.State: TIMED_WAITING (sleeping) > at java.lang.Thread.sleep(Native Method) > at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.close(DFSClient.java:1751) - locked <0x77d593a0> (a org.apache.hadoop.dfs.DFSClient$DFSOutputStream) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:49) > at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:64) at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:55) > at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:83) at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:140) > at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:826) > at org.apache.hadoop.fs.FsShell.copyFromLocal(FsShell.java:114) > at org.apache.hadoop.fs.FsShell.run(FsShell.java:1354) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:65) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:79) > at org.apache.hadoop.fs.FsShell.main(FsShell.java:1472) > -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.