[ https://issues.apache.org/jira/browse/HADOOP-14691?page=com.atlassian.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 > Original Estimate: 72h > Remaining Estimate: 72h > > 1. Bug description > Shell command “Hadoop fs -put” is a write operation. In this 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 “Hadoop fs -put”, for each created FSDataOutputStream > object, FSDataOutputStream.close() is called twice, which means the close > method, in the underlying distributed file system, is called twice. This is > the error, that’s 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. > 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. > For the command “Hadoop fs -put”, there are double close as follows: > a. The first close process: > at > org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.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.writeStreamToFile(CommandWithDestination.java:466) > at > org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(CommandWithDestination.java:391) > at > org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(CommandWithDestination.java:328) > at > org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:263) > at > org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:248) > at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:317) > at org.apache.hadoop.fs.shell.Command.processPathArgument(Command.java:289) > 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(CommandWithDestination.java:220) > at > org.apache.hadoop.fs.shell.CopyCommands$Put.processArguments(CopyCommands.java:267) > at org.apache.hadoop.fs.shell.Command.processRawArguments(Command.java:201) > 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. The second close process: > at > org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.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.writeStreamToFile(CommandWithDestination.java:468) > at > org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(CommandWithDestination.java:391) > at > org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(CommandWithDestination.java:328) > at > org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:263) > at > org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:248) > at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:317) > at org.apache.hadoop.fs.shell.Command.processPathArgument(Command.java:289) > 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(CommandWithDestination.java:220) > at > org.apache.hadoop.fs.shell.CopyCommands$Put.processArguments(CopyCommands.java:267) > at org.apache.hadoop.fs.shell.Command.processRawArguments(Command.java:201) > 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. Code analysis > void writeStreamToFile(InputStream in, PathData target, boolean lazyPersist) > throws IOException { > FSDataOutputStream out = null; > try { > out = this.create(target, lazyPersist); > IOUtils.copyBytes(in, out, this.getConf(), true); > } finally { > IOUtils.closeStream(out); > } > } > public static void copyBytes(InputStream in, OutputStream out, int buffSize, > boolean close) throws IOException { > try { > copyBytes(in, out, buffSize); > if(close) { > out.close(); > out = null; > in.close(); > in = null; > } > } finally { > if(close) { > closeStream(out); > closeStream(in); > } > } > } > The problem is caused by these two methods. > 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 “out = null” means > that within this API, the reference object out points to the address null, > but it doesn’t change the external reference object’s address value. As a > result, “out = null” is only valid for “closeStream(out)” in copyBytes(), so > for” IOUtils.closeStream(out)” in writeStreamToFile (), out is not null in > the normal process. That’s why there are double close calls. > 3. Solution > a. Suggest solution: > For the input object in an API, it is not suitable to set it as null within > this API. It should to be set in its defined API. And for this problem, the > parameter “Boolean close” seems unsuitable. The close operation should be > written in the definition API of “FSDataOutputStream out”. > This problem is also occurred for “FSDataInputStream in” which is defined in > copyFileToTarget(). > This modification solution could solve this problem thoroughly. And it makes > 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 “public static void copyBytes(InputStream in, > OutputStream out, int buffSize, boolean close)”, there are 16 calls of this > method. Considering the call path through “copyFileToTarget()” to > “copyBytes()” and related tests to submit the patch, it will be a huge work > for us. We use the above workaround solution to test our idea. > b. Our workaround solution: > Our solution is to add a return value to record the close status of out and > in. If it has been closed, it won’t be re-closed in the external API. > 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 “Hadoop fs -put” command with this solution. -- This message was sent by Atlassian JIRA (v6.4.14#64029) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org