hadoop-common-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Ken Krugler <kkrugler_li...@transpac.com>
Subject Re: HDFS to S3 copy problems
Date Wed, 13 May 2009 20:58:36 GMT
Hi Tom,

>Ian - Thanks for the detailed analysis. It was these issues that lead
>me to create a temporary file in NativeS3FileSystem in the first
>place. I think we can get NativeS3FileSystem to report progress
>though, see https://issues.apache.org/jira/browse/HADOOP-5814.
>
>Ken - I can't see why you would be getting that error. Does it work
>with hadoop fs, but not hadoop distcp?

Neither worked, unfortunately.

See my original email re my first attempt using hadoop fs - the 
result was similar to when I tried hadoop distcp. Seemed to be 
working, then hung.

Eventually I copied the files to my master node's local disk, then 
our ops guy used s3cmd to push the files to S3.

-- Ken


>On Sat, May 9, 2009 at 6:48 AM, Nowland, Ian <nowland@amazon.com> wrote:
>>  Hi Tom,
>>
>>  Not creating a temp file is the ideal as it saves you from having 
>>to "waste" using the local hard disk by writing an output file just 
>>before uploading same to Amazon S3. There are a few problems though:
>>
>>  1) Amazon S3 PUTs need the file length up front. You could use a 
>>chunked POST, but then you have the disadvantage of having to 
>>Base64 encode all your data, increasing bandwidth usage, and also 
>>you still have the next problems;
>>
>>  2) You would still want to have MD5 checking. In Amazon S3 both 
>>PUT and POST require the MD5 to be supplied before the contents. To 
>>work around this then you would have to upload the object without 
>>MD5, then check its metadata to make sure the MD5 is correct, then 
>>delete it if it is not. This is all possible, but would be 
>>difficult to make bulletproof, whereas in the current version, if 
>>the MD5 is different the PUT fails atomically and you can easily 
>>just retry.
>>
>>  3) Finally, you would have to be careful in reducers that output 
>>only very rarely. If there is too big a gap between data being 
>>uploaded through the socket, then S3 may determine the connection 
>>has timed out, closing the connection and meaning your task has to 
>>rerun (perhaps just to hit the same problem again).
>>
>>  All of this means that the current solution may be best for now as 
>>far as general upload. The best I think we can so is fix the fact 
>>that the task is not progressed in close(). The best way I can see 
>>to do this is introducing a new interface say called 
>>ExtendedClosable which defines a close(Progressable p) method. 
>>Then, have the various clients of FileSystem output streams (e.g. 
>>Distcp, TextOutputFormat) test if their DataOutputStream supports 
>>the interface, and if so call this in preference to the default. In 
>>the case of NativeS3FileSystem then, this method spins up a thread 
>>to keep the Progressable updated as the upload progresses.
>>
>>  As an additional optimization to Distcp, where the source file 
>>already exists we could have some extended interface say 
>>ExtendedWriteFileSystem that has a create() method that takes the 
>>MD5 and the file size, then test for this interface in the Distcp 
>>mapper call the extended method. The trade off here is the fact 
>>that the checksum HDFS stored is not the MD5 needed by S3, and so 
>>two (perhaps distributed) reads would be needed so the tradeoff is 
>>these two distributed reads vs a distributed read and a local write 
>>then local read.
>>
>>  What do you think?
>>
>>  Cheers,
>>  Ian Nowland
>>  Amazon.com
>>
>>  -----Original Message-----
>>  From: Tom White [mailto:tom@cloudera.com]
>>  Sent: Friday, May 08, 2009 1:36 AM
>>  To: core-user@hadoop.apache.org
>>  Subject: Re: HDFS to S3 copy problems
>>
>>  Perhaps we should revisit the implementation of NativeS3FileSystem so
>>  that it doesn't always buffer the file on the client. We could have an
>>  option to make it write directly to S3. Thoughts?
>>
>>  Regarding the problem with HADOOP-3733, you can work around it by
>>  setting fs.s3.awsAccessKeyId and fs.s3.awsSecretAccessKey in your
>>  hadoop-site.xml.
>>
>>  Cheers,
>>  Tom
>>
>>  On Fri, May 8, 2009 at 1:17 AM, Andrew Hitchcock <adpowers@gmail.com> wrote:
>>>  Hi Ken,
>>>
>>>  S3N doesn't work that well with large files. When uploading a file to
>  >> S3, S3N saves it to local disk during write() and then uploads to S3
>>>  during the close(). Close can take a long time for large files and it
>>>  doesn't report progress, so the call can time out.
>>>
>>>  As a work around, I'd recommend either increasing the timeout or
>>>  uploading the files by hand. Since you only have a few large files,
>>>  you might want to copy the files to local disk and then use something
>>>  like s3cmd to upload them to S3.
>>>
>>>  Regards,
>>>  Andrew
>>>
>>>  On Thu, May 7, 2009 at 4:42 PM, Ken Krugler 
>>><kkrugler_lists@transpac.com> wrote:
>>>>  Hi all,
>>>>
>>>>  I have a few large files (4 that are 1.8GB+) I'm trying to copy 
>>>>from HDFS to
>>>>  S3. My micro EC2 cluster is running Hadoop 0.19.1, and has one master/two
>>>>  slaves.
>>>>
>  >>> I first tried using the hadoop fs -cp command, as in:
>>>>
>>>>  hadoop fs -cp output/<dir>/ s3n://<bucket>/<dir>/
>>>>
>>>>  This seemed to be working, as I could walk the network traffic spike, and
>>>>  temp files were being created in S3 (as seen with CyberDuck).
>>>>
>>>>  But then it seemed to hang. Nothing happened for 30 minutes, so 
>>>>I killed the
>>>>  command.
>  >>>
>>>>  Then I tried using the hadoop distcp command, as in:
>>>>
>>>>  hadoop distcp hdfs://<host>:50001/<path>/<dir>/ s3://<public
key>:<private
>>>>  key>@<bucket>/<dir2>/
>>>>
>>>>  This failed, because my secret key has a '/' in it
>>>>  (http://issues.apache.org/jira/browse/HADOOP-3733)
>>>>
>>>>  Then I tried using hadoop distcp with the s3n URI syntax:
>>>>
>>>>  hadoop distcp hdfs://<host>:50001/<path>/<dir>/ s3n://<bucket>/<dir2>/
>>>>
>>>>  Similar to my first attempt, it seemed to work. Lots of network activity,
>>>>  temp files being created, and in the terminal I got:
>>>>
>>>>  09/05/07 18:36:11 INFO mapred.JobClient: Running job: 
>>>>job_200905071339_0004
>>>>  09/05/07 18:36:12 INFO mapred.JobClient:  map 0% reduce 0%
>>>>  09/05/07 18:36:30 INFO mapred.JobClient:  map 9% reduce 0%
>>>>  09/05/07 18:36:35 INFO mapred.JobClient:  map 14% reduce 0%
>>>>  09/05/07 18:36:38 INFO mapred.JobClient:  map 20% reduce 0%
>>>>
>>>>  But again it hung. No network traffic, and eventually it dumped out:
>>>>
>>>>  09/05/07 18:52:34 INFO mapred.JobClient: Task Id :
>>>>  attempt_200905071339_0004_m_000001_0, Status : FAILED
>>>>  Task attempt_200905071339_0004_m_000001_0 failed to report status for 601
>>>>  seconds. Killing!
>>>>  09/05/07 18:53:02 INFO mapred.JobClient: Task Id :
>>>>  attempt_200905071339_0004_m_000004_0, Status : FAILED
>>>>  Task attempt_200905071339_0004_m_000004_0 failed to report status for 602
>>>>  seconds. Killing!
>>>>  09/05/07 18:53:06 INFO mapred.JobClient: Task Id :
>>>>  attempt_200905071339_0004_m_000002_0, Status : FAILED
>>>>  Task attempt_200905071339_0004_m_000002_0 failed to report status for 602
>>>>  seconds. Killing!
>>>>  09/05/07 18:53:09 INFO mapred.JobClient: Task Id :
>>>>  attempt_200905071339_0004_m_000003_0, Status : FAILED
>>>>  Task attempt_200905071339_0004_m_000003_0 failed to report status for 601
>>>>  seconds. Killing!
>>>>
>  >>> In the task GUI, I can see the same tasks failing, and being 
>restarted. But
>>>>  the restarted tasks seem to be just hanging w/o doing anything.
>>>>
>>>>  Eventually one of the tasks made a bit more progress, but then it finally
>>>>  died with:
>>>>
>  >>> Copy failed: java.io.IOException: Job failed!
>>>>         at org.apache.hadoop.mapred.JobClient.runJob(JobClient.java:1232)
>>>>         at org.apache.hadoop.tools.DistCp.copy(DistCp.java:647)
>>>>         at org.apache.hadoop.tools.DistCp.run(DistCp.java:844)
>>>>         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:871)
>>>>
>>>>  So - any thoughts on what's going wrong?
>>>>
>>>>  Thanks,
>>>>
>>>>  -- Ken
>>>>  --
>>>>  Ken Krugler
>>>>  +1 530-210-6378
>>>>
>>>
>>


-- 
Ken Krugler
+1 530-210-6378

Mime
View raw message