hadoop-mapreduce-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Aaron Kimball (JIRA)" <j...@apache.org>
Subject [jira] Updated: (MAPREDUCE-968) NPE in distcp encountered when placing _logs directory on S3FileSystem
Date Thu, 10 Sep 2009 21:36:57 GMT

     [ https://issues.apache.org/jira/browse/MAPREDUCE-968?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Aaron Kimball updated MAPREDUCE-968:
------------------------------------

    Attachment: MAPREDUCE-968.patch

This patch fixes the issue. If the destination directory is '/' and doesn't exist, it will
fall into a case where Path.getParent() is used to compute the {{\_logs}} target directory
name. This returns null if the Path is '/'. In this special case, the '/' directory needs
to be created by distcp too.

No unit test because this requires creating S3 buckets. I manually tested this by creating
an empty S3 bucket and running:

{code}
bin/hadoop distcp some-hdfs-dir s3://<access-key>:<secret-key>@my-new-bucket/
{code}

This failed with the NPE. After the patch, this succeeded. Confirmed that file uploads worked
via

{code}
bin/hadoop fs -ls s3://<access-key>:<secret-key>@my-new-bucket/
{code}



> NPE in distcp encountered when placing _logs directory on S3FileSystem
> ----------------------------------------------------------------------
>
>                 Key: MAPREDUCE-968
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-968
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: distcp
>    Affects Versions: 0.20.1
>            Reporter: Aaron Kimball
>            Assignee: Aaron Kimball
>         Attachments: MAPREDUCE-968.patch
>
>
> If distcp is pointed to an empty S3 bucket as the destination for an s3:// filesystem
transfer, it will fail with the following exception
> Copy failed: java.lang.NullPointerException
> at org.apache.hadoop.fs.s3.S3FileSystem.makeAbsolute(S3FileSystem.java:121)
> at org.apache.hadoop.fs.s3.S3FileSystem.getFileStatus(S3FileSystem.java:332)
> at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:633)
> at org.apache.hadoop.tools.DistCp.setup(DistCp.java:1005)
> at org.apache.hadoop.tools.DistCp.copy(DistCp.java:650)
> at org.apache.hadoop.tools.DistCp.run(DistCp.java:857)
> at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:65)
> at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:79)
> at org.apache.hadoop.tools.DistCp.main(DistCp.java:884) 

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


Mime
View raw message