[ 
https://issues.apache.org/jira/browse/HDFS-3696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13419179#comment-13419179
 ] 

Kihwal Lee commented on HDFS-3696:
----------------------------------

The following stack trace is from doing {{copyFromLocal}} with 140MB file. The 
map heap is 1G (-Xmx1000m) in the client side.

{noformat}
$ hadoop fs -copyFromLocal /tmp/xxx140m 
webhdfs://my.server.blah:50070/user/kihwal/xxx
Exception in thread "main" java.lang.OutOfMemoryError: Java heap space
        at java.util.Arrays.copyOf(Arrays.java:2786)
        at java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:94)
        at sun.net.www.http.PosterOutputStream.write(PosterOutputStream.java:61)
        at java.io.BufferedOutputStream.write(BufferedOutputStream.java:105)
        at 
org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:54)
        at java.io.DataOutputStream.write(DataOutputStream.java:90)
        at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:80)
        at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:52)
        at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:112)
        at 
org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(CommandWithDestination.java:240)
        at 
org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(CommandWithDestination.java:219)
        at 
org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:165)
        at 
org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:150)
        at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:306)
        at 
org.apache.hadoop.fs.shell.Command.processPathArgument(Command.java:278)
        at 
org.apache.hadoop.fs.shell.CommandWithDestination.processPathArgument(CommandWithDestination.java:145)
        at org.apache.hadoop.fs.shell.Command.processArgument(Command.java:260)
        at org.apache.hadoop.fs.shell.Command.processArguments(Command.java:244)
        at 
org.apache.hadoop.fs.shell.CommandWithDestination.processArguments(CommandWithDestination.java:122)
        at 
org.apache.hadoop.fs.shell.CopyCommands$Put.processArguments(CopyCommands.java:204)
        at 
org.apache.hadoop.fs.shell.Command.processRawArguments(Command.java:190)
        at org.apache.hadoop.fs.shell.Command.run(Command.java:154)
        at org.apache.hadoop.fs.FsShell.run(FsShell.java:254)
        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:304)
{noformat}
                
> FsShell put using WebHdfsFileSystem goes OOM when file size is big
> ------------------------------------------------------------------
>
>                 Key: HDFS-3696
>                 URL: https://issues.apache.org/jira/browse/HDFS-3696
>             Project: Hadoop HDFS
>          Issue Type: Bug
>    Affects Versions: 2.0.0-alpha
>            Reporter: Kihwal Lee
>            Priority: Critical
>             Fix For: 0.23.3, 3.0.0, 2.2.0-alpha
>
>
> When dong "fs -put" to a WebHdfsFileSystem (webhdfs://), the FsShell goes OOM 
> if the file size is large. When I tested, 20MB files were fine, but 200MB 
> didn't work.  
> I also tried reading a large file by issuing "-cat" and piping to a slow sink 
> in order to force buffering. The read path didn't have this problem. The 
> memory consumption stayed the same regardless of progress.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to