crunch-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Everett Anderson <ever...@nuna.com>
Subject Re: Crunch Planner Hint to Not Combine Tasks
Date Tue, 24 Nov 2015 18:30:04 GMT
On Tue, Nov 24, 2015 at 9:55 AM, Robinson, Landon - Landon <
landon.t.robinson@lowes.com> wrote:

> Jeff,
>
> Thanks for that awesome set of tips. Building on your solution, here’s
> some information about ours:
> Our program:
>
>    - Starts the MR pipeline
>    - Does some DoFn ParallelDos
>    - Calls pipeline.execute
>
> We never leveraged pipeline.run(). Going with your suggestion, would my
> course of action be:
>

>    - Start the MR pipeline
>    - Do some of the dofn parallelDos
>    - Call pipeline.write
>    - Call pipeline.run
>    - Ingest that file into a new Pcollection
>
>
You don't have to explicitly do anything to re-ingest, here -- you keep
using the same PCollection references that you checkpointed. Under the
covers, however, they'll be backed by sequence files that survive across
the call to run().

We have a utility method for this that uses crunch.tmp.dir with a unique
path name per table and calls

      pCollection.write(To.sequenceFile(path), Target.WriteMode.CHECKPOINT);





>
>    - Call another (or more) Dofns
>    - Call pipeline.execute
>
> Does that seem in line with your recommendation, Jeff? Let me know if my
> logic needs adjusting…
> - Landon
> ---------------------------------------------------------------------------
> Landon Robinson
> Big Data/Hadoop Engineer
> ---------------------------------------------------------------------------
>
> From: Jeff Quinn <jeff@nuna.com>
> Reply-To: Apache Crunch Mailing List <user@crunch.apache.org>
> Date: Tuesday, November 24, 2015 at 12:00 PM
> To: Apache Crunch Mailing List <user@crunch.apache.org>
> Subject: Re: Crunch Planner Hint to Not Combine Tasks
>
> Hey Landon,
>
> Our team has dealt with this exact problem before. Our solution was to
> call PCollection#write after each DoFn or string of DoFns then call
> Pipeline#run, which will do an iteration of crunch planning / job
> submission, then use that same PCollection object for the next round of
> DoFns, call #write, and then #call run again, etc.
>
> #parallelDo -> #write -> #run -> repeat
>
> The importance of calling #write is that it the crunch planner will not
> actually do any work unless it has seen that there is at least one
> materialization of the data. This technique allows you to guarantee your
> DoFns are segmented into different MR jobs, as DoFns cannot be combined if
> they are already completed.
>
> Hope this helps,
>
> Jeff
>
> On Tue, Nov 24, 2015 at 6:14 AM, Ron Hashimshony <
> ron.hashimshony@myheritage.com> wrote:
>
>> Try set mapreduce.input.fileinputformat.split.minsize
>> & mapreduce.input.fileinputformat.split.maxsize to a lower number from the
>> default (usually 64 MB).
>> If you know of a specific DoFn in which this is required, better put it
>> there in its configure function.
>>
>> On Tue, Nov 24, 2015 at 3:28 PM Robinson, Landon - Landon <
>> landon.t.robinson@lowes.com> wrote:
>>
>>> Hi all,
>>>
>>> I have a Crunch job that tries to combine the last four tasks of my
>>> program into one M/R job.
>>> That’s normally not a problem, but my data *starts small and grows
>>> exponentially* in the most major of those DoFn tasks, resulting in
>>> spills to disk (local, not HDFS).
>>>
>>> I’ve already:
>>>
>>>    - Implemented scaleFactor on the DoFn where the data will emit back
>>>    more records than it consumed, which is 40.0f
>>>    - Set io.sort.mb parameter to cluster setting, which is 1792
>>>    - Implemented map-side compression with snappy
>>>
>>> Data set I’m ingesting is from a previous map-reduce job, which comes
>>> out to 19 files of 10mb size (which in Crunch comes to 2 splits).
>>> Help?
>>>
>>> ---------------------------------------------------------------------------
>>> Landon Robinson
>>> Big Data/Hadoop Engineer
>>>
>>> ---------------------------------------------------------------------------
>>> NOTICE: All information in and attached to the e-mails below may be
>>> proprietary, confidential, privileged and otherwise protected from improper
>>> or erroneous disclosure. If you are not the sender's intended recipient,
>>> you are not authorized to intercept, read, print, retain, copy, forward, or
>>> disseminate this message. If you have erroneously received this
>>> communication, please notify the sender immediately by phone
>>> (704-758-1000) or by e-mail and destroy all copies of this message
>>> electronic, paper, or otherwise.
>>>
>>> *By transmitting documents via this email: Users, Customers, Suppliers
>>> and Vendors collectively acknowledge and agree the transmittal of
>>> information via email is voluntary, is offered as a convenience, and is not
>>> a secured method of communication; Not to transmit any payment information
>>> E.G. credit card, debit card, checking account, wire transfer information,
>>> passwords, or sensitive and personal information E.G. Driver's license,
>>> DOB, social security, or any other information the user wishes to remain
>>> confidential; To transmit only non-confidential information such as plans,
>>> pictures and drawings and to assume all risk and liability for and
>>> indemnify Lowe's from any claims, losses or damages that may arise from the
>>> transmittal of documents or including non-confidential information in the
>>> body of an email transmittal. Thank you. *
>>>
>>
>
> *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.
> NOTICE: All information in and attached to the e-mails below may be
> proprietary, confidential, privileged and otherwise protected from improper
> or erroneous disclosure. If you are not the sender's intended recipient,
> you are not authorized to intercept, read, print, retain, copy, forward, or
> disseminate this message. If you have erroneously received this
> communication, please notify the sender immediately by phone (704-758-1000)
> or by e-mail and destroy all copies of this message electronic, paper, or
> otherwise.
>
> *By transmitting documents via this email: Users, Customers, Suppliers and
> Vendors collectively acknowledge and agree the transmittal of information
> via email is voluntary, is offered as a convenience, and is not a secured
> method of communication; Not to transmit any payment information E.G.
> credit card, debit card, checking account, wire transfer information,
> passwords, or sensitive and personal information E.G. Driver's license,
> DOB, social security, or any other information the user wishes to remain
> confidential; To transmit only non-confidential information such as plans,
> pictures and drawings and to assume all risk and liability for and
> indemnify Lowe's from any claims, losses or damages that may arise from the
> transmittal of documents or including non-confidential information in the
> body of an email transmittal. Thank you. *
>

-- 
*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.

Mime
View raw message