[jira] [Commented] (HADOOP-14691) Shell command "hadoop fs -put" multiple close problem
[ https://issues.apache.org/jira/browse/HADOOP-14691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16119833#comment-16119833 ] Andras Bokor commented on HADOOP-14691: --- [~jzhuge], I think we should eliminate the multiple close calls instead of handling it. {code:title=CommandWithDestination#writeStreamToFile} try { out = create(target, lazyPersist, direct); IOUtils.copyBytes(in, out, getConf(), true); // 1st close call } finally { IOUtils.closeStream(out); // second close call } {code} I beleive the original author assumed that after calling {{IOUtils.copyBytes}} with true the out will be null so the second close call has no effect. One possible solution is to call {{IOUtils.copyBytes}} with false and close the stream with try-catch-resource: {code} void writeStreamToFile(InputStream in, PathData target, boolean lazyPersist, boolean direct) throws IOException { try (FSDataOutputStream out = create(target, lazyPersist, direct)) { IOUtils.copyBytes(in, out, getConf(), false); } } {code} But I agree with the reporter about {{IOUtils.copyBytes}} is misleading and should be changed. If we want to fix this specific double call I suggest go with the code few lines above. But I still suggest fixing HADOOP-5943 since there are more double call in the code which shows the current API is misleading. Please check {{FileContext.Util#copy}} for example. > 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 >Assignee: Eric Lei > Labels: close, filesystem, hadoop, multi > Attachments: CommandWithDestination.patch, > hadoop_common_unit_test_result_after_modification.docx, > hadoop_common_unit_test_result_before_modification.docx, IOUtils.patch > > 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
[jira] [Commented] (HADOOP-14691) Shell command "hadoop fs -put" multiple close problem
[ https://issues.apache.org/jira/browse/HADOOP-14691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16119087#comment-16119087 ] John Zhuge commented on HADOOP-14691: - Can we just add {{out = null}} after {{out.close()}} to ensure no-op to close a closed stream? {code:java|title=FSDataOutputStream.PositionCache#close} @Override public void close() throws IOException { // ensure close works even if a null reference was passed in if (out != null) { out.close(); out = null; <== } } {code} {code:java|title=FSDataOutputStream#close} public void close() throws IOException { if (out != null) { = out.close(); // This invokes PositionCache.close() out = null; == } } {code} > 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 >Assignee: Eric Lei > Labels: close, filesystem, hadoop, multi > Attachments: CommandWithDestination.patch, > hadoop_common_unit_test_result_after_modification.docx, > hadoop_common_unit_test_result_before_modification.docx, IOUtils.patch > > 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 >
[jira] [Commented] (HADOOP-14691) Shell command "hadoop fs -put" multiple close problem
[ https://issues.apache.org/jira/browse/HADOOP-14691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16118303#comment-16118303 ] Andras Bokor commented on HADOOP-14691: --- It's a very good catch but the solution seems makes the things more complicated and it is not backward-compatible since we change the signature of a method. Instead, I suggest fixing HADOOP-5943. Using try-with-resources at the same place where the resource is created seems a better practice and used widely in Java world. We can introduce the new methods without closing ability and keep the old ones as deprecated to keep the compatibility. I am happy to send a patch for HADOOP-5943. Thoughts? P.s.: I removing the linked issue since it is not related to the exception in HDFS-10429. > 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 >Assignee: Eric Lei > Labels: close, filesystem, hadoop, multi > Attachments: CommandWithDestination.patch, > hadoop_common_unit_test_result_after_modification.docx, > hadoop_common_unit_test_result_before_modification.docx, IOUtils.patch > > 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)
[jira] [Commented] (HADOOP-14691) Shell command "hadoop fs -put" multiple close problem
[ https://issues.apache.org/jira/browse/HADOOP-14691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16115457#comment-16115457 ] Hadoop QA commented on HADOOP-14691: | (x) *{color:red}-1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 0m 0s{color} | {color:blue} Docker mode activated. {color} | | {color:red}-1{color} | {color:red} patch {color} | {color:red} 0m 4s{color} | {color:red} HADOOP-14691 does not apply to trunk. Rebase required? Wrong Branch? See https://wiki.apache.org/hadoop/HowToContribute for help. {color} | \\ \\ || Subsystem || Report/Notes || | JIRA Issue | HADOOP-14691 | | JIRA Patch URL | https://issues.apache.org/jira/secure/attachment/12880516/CommandWithDestination.patch | | Console output | https://builds.apache.org/job/PreCommit-HADOOP-Build/12964/console | | Powered by | Apache Yetus 0.6.0-SNAPSHOT http://yetus.apache.org | This message was automatically generated. > 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 >Assignee: Eric Lei > Labels: close, filesystem, hadoop, multi > Attachments: CommandWithDestination.patch, > hadoop_common_unit_test_result_after_modification.docx, > hadoop_common_unit_test_result_before_modification.docx, IOUtils.patch > > 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
[jira] [Commented] (HADOOP-14691) Shell command "hadoop fs -put" multiple close problem
[ https://issues.apache.org/jira/browse/HADOOP-14691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16115306#comment-16115306 ] John Zhuge commented on HADOOP-14691: - [~Eric88] Thanks for reporting the issue and working on patch! I added you as a Hadoop Contributor and assigned the JIRA to you. Please read https://wiki.apache.org/hadoop/HowToContribute and name the patch properly. > 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 >Assignee: Eric Lei > Labels: close, filesystem, hadoop, multi > Attachments: CommandWithDestination.patch, > hadoop_common_unit_test_result_after_modification.docx, > hadoop_common_unit_test_result_before_modification.docx, IOUtils.patch > > 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 >
[jira] [Commented] (HADOOP-14691) Shell command "hadoop fs -put" multiple close problem
[ https://issues.apache.org/jira/browse/HADOOP-14691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16115304#comment-16115304 ] Eric Lei commented on HADOOP-14691: --- Hi Wei-Chiu Chuang, Thanks for your comments very much! Please help to review the new patches. > 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: CommandWithDestination.patch, > hadoop_common_unit_test_result_after_modification.docx, > hadoop_common_unit_test_result_before_modification.docx, IOUtils.patch > > 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
[jira] [Commented] (HADOOP-14691) Shell command "hadoop fs -put" multiple close problem
[ https://issues.apache.org/jira/browse/HADOOP-14691?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16103829#comment-16103829 ] Wei-Chiu Chuang commented on HADOOP-14691: -- Hi [~Eric88] thanks for the detailed report! It seems what you discovered is similar to HDFS-10429. I have not yet reviewed the patch in depth, but it looks like your patch contains unrelevant stuff. Could you remove them and rebase against trunk? Thanks. > 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_result_after_modification.docx, > hadoop_common_unit_test_result_before_modification.docx > > 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 >