Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id C8F6A200CD2 for ; Thu, 27 Jul 2017 11:14:07 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id C76E616A58F; Thu, 27 Jul 2017 09:14:07 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id EF21F16A55D for ; Thu, 27 Jul 2017 11:14:06 +0200 (CEST) Received: (qmail 25394 invoked by uid 500); 27 Jul 2017 09:14:06 -0000 Mailing-List: contact common-issues-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list common-issues@hadoop.apache.org Received: (qmail 25383 invoked by uid 99); 27 Jul 2017 09:14:06 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 27 Jul 2017 09:14:06 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 88264C29AC for ; Thu, 27 Jul 2017 09:14:05 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -100.002 X-Spam-Level: X-Spam-Status: No, score=-100.002 tagged_above=-999 required=6.31 tests=[RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id DupS5XRWWMiW for ; Thu, 27 Jul 2017 09:14:04 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 6E6F46111B for ; Thu, 27 Jul 2017 09:14:02 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id B8B87E0D39 for ; Thu, 27 Jul 2017 09:14:01 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 8D3A624D2D for ; Thu, 27 Jul 2017 09:14:00 +0000 (UTC) Date: Thu, 27 Jul 2017 09:14:00 +0000 (UTC) From: "Eric Lei (JIRA)" To: common-issues@hadoop.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (HADOOP-14691) Shell command "hadoop fs -put" multiple close problem MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Thu, 27 Jul 2017 09:14:08 -0000 [ https://issues.apache.org/jira/browse/HADOOP-14691?page=3Dcom.atlass= ian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eric Lei updated HADOOP-14691: ------------------------------ Status: Patch Available (was: Open) > Shell command "hadoop fs -put" multiple close problem > ----------------------------------------------------- > > Key: HADOOP-14691 > URL: https://issues.apache.org/jira/browse/HADOOP-14691 > Project: Hadoop Common > Issue Type: Bug > Components: common > Affects Versions: 2.7.3 > Environment: CentOS7.0 > JDK1.8.0_121 > hadoop2.7.3 > Reporter: Eric Lei > Labels: close, filesystem, hadoop, multi > Attachments: hadoop-2.7.3-src.patch, hadoop_common_unit_test_resu= lt_after_modification.docx, hadoop_common_unit_test_result_before_modificat= ion.docx > > Original Estimate: 72h > Remaining Estimate: 72h > > 1.=09Bug description > Shell command =E2=80=9CHadoop fs -put=E2=80=9D is a write operation. In t= his process, FSDataOutputStream is new created and closed lastly. Finally, = the FSDataOutputStream.close() calls the close method in HDFS to end up the= communication of this write process between the server and client. > With the command =E2=80=9CHadoop fs -put=E2=80=9D, for each created FSDat= aOutputStream object, FSDataOutputStream.close() is called twice, which mea= ns the close method, in the underlying distributed file system, is called t= wice. This is the error, that=E2=80=99s because the communication process, = for example socket, might be repeated shut down. Unfortunately, if there is= no error protection for the socket, there might be error for the socket in= the second close.=20 > Further, we think a correct upper file system design should keep the one = time close principle. It means that each creation of underlying distributed= file system object should correspond with close only once.=20 > For the command =E2=80=9CHadoop fs -put=E2=80=9D, there are double close = as follows: > a.=09The first close process: > at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutp= utStream.java:72) > at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:= 106) > at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:61) > at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:119) > at org.apache.hadoop.fs.shell.CommandWithDestination$TargetFileSystem.wri= teStreamToFile(CommandWithDestination.java:466) > at org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(C= ommandWithDestination.java:391) > at org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(Com= mandWithDestination.java:328) > at org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandW= ithDestination.java:263) > at org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandW= ithDestination.java:248) > at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:317) > at org.apache.hadoop.fs.shell.Command.processPathArgument(Command.java:28= 9) > at org.apache.hadoop.fs.shell.CommandWithDestination.processPathArgument(= CommandWithDestination.java:243) > at org.apache.hadoop.fs.shell.Command.processArgument(Command.java:271) > at org.apache.hadoop.fs.shell.Command.processArguments(Command.java:255) > at org.apache.hadoop.fs.shell.CommandWithDestination.processArguments(Com= mandWithDestination.java:220) > at org.apache.hadoop.fs.shell.CopyCommands$Put.processArguments(CopyComma= nds.java:267) > at org.apache.hadoop.fs.shell.Command.processRawArguments(Command.java:20= 1) > at org.apache.hadoop.fs.shell.Command.run(Command.java:165) > at org.apache.hadoop.fs.FsShell.run(FsShell.java:287) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:84) > at org.apache.hadoop.fs.FsShell.main(FsShell.java:340) > b.=09The second close process: > at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutp= utStream.java:72) > at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:= 106) > at org.apache.hadoop.io.IOUtils.cleanup(IOUtils.java:244) > at org.apache.hadoop.io.IOUtils.closeStream(IOUtils.java:261) > at org.apache.hadoop.fs.shell.CommandWithDestination$TargetFileSystem.wri= teStreamToFile(CommandWithDestination.java:468) > at org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(C= ommandWithDestination.java:391) > at org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(Com= mandWithDestination.java:328) > at org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandW= ithDestination.java:263) > at org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandW= ithDestination.java:248) > at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:317) > at org.apache.hadoop.fs.shell.Command.processPathArgument(Command.java:28= 9) > at org.apache.hadoop.fs.shell.CommandWithDestination.processPathArgument(= CommandWithDestination.java:243) > at org.apache.hadoop.fs.shell.Command.processArgument(Command.java:271) > at org.apache.hadoop.fs.shell.Command.processArguments(Command.java:255) > at org.apache.hadoop.fs.shell.CommandWithDestination.processArguments(Com= mandWithDestination.java:220) > at org.apache.hadoop.fs.shell.CopyCommands$Put.processArguments(CopyComma= nds.java:267) > at org.apache.hadoop.fs.shell.Command.processRawArguments(Command.java:20= 1) > at org.apache.hadoop.fs.shell.Command.run(Command.java:165) > at org.apache.hadoop.fs.FsShell.run(FsShell.java:287) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:70) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:84) > at org.apache.hadoop.fs.FsShell.main(FsShell.java:340) > 2.=09Code analysis > void writeStreamToFile(InputStream in, PathData target, boolean lazyPersi= st) throws IOException { > FSDataOutputStream out =3D null; > try { > out =3D this.create(target, lazyPersist); > IOUtils.copyBytes(in, out, this.getConf(), true); > } finally { > IOUtils.closeStream(out); > } > } > public static void copyBytes(InputStream in, OutputStream out, int buffSi= ze, boolean close) throws IOException { > try { > copyBytes(in, out, buffSize); > if(close) { > out.close(); > out =3D null; > in.close(); > in =3D null; > } > } finally { > if(close) { > closeStream(out); > closeStream(in); > } > } > } > The problem is caused by these two methods.=20 > FSDataOutputStream out is defined in the method writeStreamToFile, and it= is a input parameter of copyBytes. When an object is as an input parameter= of a API, it equals to a new reference in this API. It points to the same = address with the external API. In the API copyBytes(), the code =E2=80=9Cou= t =3D null=E2=80=9D means that within this API, the reference object out po= ints to the address null, but it doesn=E2=80=99t change the external refere= nce object=E2=80=99s address value. As a result, =E2=80=9Cout =3D null=E2= =80=9D is only valid for =E2=80=9CcloseStream(out)=E2=80=9D in copyBytes(),= so for=E2=80=9D IOUtils.closeStream(out)=E2=80=9D in writeStreamToFile ()= , out is not null in the normal process. That=E2=80=99s why there are doubl= e close calls. > 3.=09Solution > a.=09Suggest solution: > For the input object in an API, it is not suitable to set it as null with= in this API. It should to be set in its defined API. And for this problem, = the parameter =E2=80=9CBoolean close=E2=80=9D seems unsuitable. The close o= peration should be written in the definition API of =E2=80=9CFSDataOutputSt= ream out=E2=80=9D.=20 > This problem is also occurred for =E2=80=9CFSDataInputStream in=E2=80=9D = which is defined in copyFileToTarget(). > This modification solution could solve this problem thoroughly. And it ma= kes the design to be more suitable for the file system. > However, with this modification solution, there are many related code for= this process. Only the =E2=80=9Cpublic static void copyBytes(InputStream i= n, OutputStream out, int buffSize, boolean close)=E2=80=9D, there are 16 ca= lls of this method. Considering the call path through =E2=80=9CcopyFileToTa= rget()=E2=80=9D to =E2=80=9CcopyBytes()=E2=80=9D and related tests to submi= t the patch, it will be a huge work for us. We use the above workaround sol= ution to test our idea. > b.=09Our workaround solution: > Our solution is to add a return value to record the close status of out a= nd in. If it has been closed, it won=E2=80=99t be re-closed in the external= API.=20 > This solution could make sure the out and in are closed only once easily. > The details could refer to the patch. It shows the least modification for= this =E2=80=9CHadoop fs -put=E2=80=9D command with this solution. -- This message was sent by Atlassian JIRA (v6.4.14#64029) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org For additional commands, e-mail: common-issues-help@hadoop.apache.org