You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@crunch.apache.org by Jeff Quinn <je...@nuna.com> on 2015/12/05 02:28:12 UTC

Re: CrunchJobHooks.CompletionHook Inefficiency on S3NativeFileSystem

I have created a JIRA issue and a patch for this issue:
https://issues.apache.org/jira/browse/CRUNCH-580

It turns out the #getSize issue I mentioned in my last email is not really
as much as a bottle neck as I thought (the new EMR Consistency Filesystem
allows for quick listing and metadata gathering of objects on S3), so I
left that unaddressed here.

Thanks!

Jeff

On Mon, Nov 30, 2015 at 12:54 PM, Jeff Quinn <je...@nuna.com> wrote:

> We currently use S3DistCp, but that workflow for us assumes that all of
> our input data can fit on HDFS for the cluster we spin up, which might
> cause some problems at some point, so we are trying to see if we can get
> direct to S3 io working.
>
> I tried an initial run using a threadpool to do the copy/deletes and it
> was much faster/didn't crash with 32 threads, but it hit another bottle
> neck in org.apache.crunch.io.impl.FileSourceImpl#getSize as it tries to get
> the size of each output file for planning the next job. Going to try
> threadpooling that operation as well and see if anything else comes up.
>
> On Tue, Nov 24, 2015 at 7:07 AM, Ron Hashimshony <
> ron.hashimshony@myheritage.com> wrote:
>
>> We just use S3DistCp, works pretty well (we usually use EMR, but DistCp
>> does the same in our Cloudera cluster).
>>
>>
>> On Tue, Nov 24, 2015 at 9:46 AM Josh Wills <jo...@gmail.com> wrote:
>>
>>> Sounds good to me; either a PR or patch on the JIRA works as a way to
>>> get it committed.
>>> On Mon, Nov 23, 2015 at 11:16 PM Jeff Quinn <je...@nuna.com> wrote:
>>>
>>>> So far the only solution I can imagine is using a thread pool to make
>>>> all the copy/delete requests. Reading this relevant blog post gives me some
>>>> confidence that a thread pool could work and not be horribly slow:
>>>> http://shlomoswidler.com/2010/05/how-i-moved-5-of-all-objects-in-s3-with-jets3t.html
>>>>
>>>> If we wanted to submit a patch what would be a good approach? I was
>>>> thinking maybe org.apache.crunch.io.PathTarget#handleOutputs could return a
>>>> ListenableFuture that executes the rename and CrunchJobHooks.
>>>> CompletionHook#handleMultiPaths could use a threadpool executor, with
>>>> a configuration property controlling number of threads..
>>>>
>>>> On Mon, Nov 23, 2015 at 7:47 PM, Josh Wills <jo...@gmail.com>
>>>> wrote:
>>>>
>>>>> No, just moving to Slack from Cloudera, my data team is all of two
>>>>> people* right now, and a dedicated Hadoop ops person doesn't make sense yet.
>>>>>
>>>>> * But of course, I'm hiring. :)
>>>>>
>>>>> On Mon, Nov 23, 2015 at 6:43 PM Everett Anderson <ev...@nuna.com>
>>>>> wrote:
>>>>>
>>>>>> Josh, not to steal the thread, but I'm quite curious -- did something
>>>>>> drive you to using S3 instead of HDFS?
>>>>>>
>>>>>> For me, I've been surprised how brittle HDFS seems out of the box in
>>>>>> the face of even mild load. :( We've spent a lot of time turning knobs to
>>>>>> make our data nodes stay responsive.
>>>>>>
>>>>>>
>>>>>> On Mon, Nov 23, 2015 at 5:45 PM, Josh Wills <jo...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> (I don't know the answer to this, but as I also now run Crunch on
>>>>>>> top of S3, I'm interested in a solution.)
>>>>>>>
>>>>>>> On Mon, Nov 23, 2015 at 5:22 PM, Jeff Quinn <je...@nuna.com> wrote:
>>>>>>>
>>>>>>>> Hey All,
>>>>>>>>
>>>>>>>> We have run in to a pretty frustrating inefficiency inside of
>>>>>>>> the CrunchJobHooks.CompletionHook#handleMultiPaths.
>>>>>>>>
>>>>>>>> This method loops over all of the partial output files and moves
>>>>>>>> them to their ultimate destination directories,
>>>>>>>> calling org.apache.hadoop.fs.FileSystem#rename(org.apache.hadoop.fs.Path,
>>>>>>>> org.apache.hadoop.fs.Path) on each partial output in a loop.
>>>>>>>>
>>>>>>>> This is no problem when the org.apache.hadoop.fs.FileSystem in
>>>>>>>> question is HDFS where #rename is a cheap operation, but when an
>>>>>>>> implementation such as S3NativeFileSystem is used it is extremely
>>>>>>>> inefficient, as each iteration through the loop makes a single blocking S3
>>>>>>>> API call, and this loop can be extremely long when there are many thousands
>>>>>>>> of partial output files.
>>>>>>>>
>>>>>>>> Has anyone dealt with this before / have any ideas to work around?
>>>>>>>>
>>>>>>>> Thanks!
>>>>>>>>
>>>>>>>> Jeff
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> *DISCLAIMER:* The contents of this email, including any
>>>>>>>> attachments, may contain information that is confidential, proprietary in
>>>>>>>> nature, protected health information (PHI), or otherwise protected by law
>>>>>>>> from disclosure, and is solely for the use of the intended recipient(s). If
>>>>>>>> you are not the intended recipient, you are hereby notified that any use,
>>>>>>>> disclosure or copying of this email, including any attachments, is
>>>>>>>> unauthorized and strictly prohibited. If you have received this email in
>>>>>>>> error, please notify the sender of this email. Please delete this and all
>>>>>>>> copies of this email from your system. Any opinions either expressed or
>>>>>>>> implied in this email and all attachments, are those of its author only,
>>>>>>>> and do not necessarily reflect those of Nuna Health, Inc.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>> *DISCLAIMER:* The contents of this email, including any attachments,
>>>>>> may contain information that is confidential, proprietary in nature,
>>>>>> protected health information (PHI), or otherwise protected by law from
>>>>>> disclosure, and is solely for the use of the intended recipient(s). If you
>>>>>> are not the intended recipient, you are hereby notified that any use,
>>>>>> disclosure or copying of this email, including any attachments, is
>>>>>> unauthorized and strictly prohibited. If you have received this email in
>>>>>> error, please notify the sender of this email. Please delete this and all
>>>>>> copies of this email from your system. Any opinions either expressed or
>>>>>> implied in this email and all attachments, are those of its author only,
>>>>>> and do not necessarily reflect those of Nuna Health, Inc.
>>>>>
>>>>>
>>>>
>>>> *DISCLAIMER:* The contents of this email, including any attachments,
>>>> may contain information that is confidential, proprietary in nature,
>>>> protected health information (PHI), or otherwise protected by law from
>>>> disclosure, and is solely for the use of the intended recipient(s). If you
>>>> are not the intended recipient, you are hereby notified that any use,
>>>> disclosure or copying of this email, including any attachments, is
>>>> unauthorized and strictly prohibited. If you have received this email in
>>>> error, please notify the sender of this email. Please delete this and all
>>>> copies of this email from your system. Any opinions either expressed or
>>>> implied in this email and all attachments, are those of its author only,
>>>> and do not necessarily reflect those of Nuna Health, Inc.
>>>
>>>
>

-- 
*DISCLAIMER:* The contents of this email, including any attachments, may 
contain information that is confidential, proprietary in nature, protected 
health information (PHI), or otherwise protected by law from disclosure, 
and is solely for the use of the intended recipient(s). If you are not the 
intended recipient, you are hereby notified that any use, disclosure or 
copying of this email, including any attachments, is unauthorized and 
strictly prohibited. If you have received this email in error, please 
notify the sender of this email. Please delete this and all copies of this 
email from your system. Any opinions either expressed or implied in this 
email and all attachments, are those of its author only, and do not 
necessarily reflect those of Nuna Health, Inc.