It is now in trunk (future 0.11)
Julien

On Jun 19, 2012, at 9:35 AM, Thejas Nair wrote:

> Yes, pig should cache the results of getschema(). that is someting to be 
> fixed.
> Julien is working on some patches to fix the issues related to udfcontext, 
> including reducing the number of times load function gets instantiated.
> 
> -Thejas
> 
> 
> On 6/12/12 4:16 PM, Alex Rovner wrote:
>> Thejas,
>> 
>> Why can't Pig cache results of getschema instead of repeatedly calling the 
>> load function? This is causing us to put in lots of unnecessary workarounds 
>> in place since during those calls one cant rely on the udf context as it 
>> gets cleared somewhere inbetween those calls?
>> 
>> Sent from my iPhone
>> 
>> On Jun 1, 2012, at 12:45 PM, Thejas Nair<[email protected]>  wrote:
>> 
>>> Can you do a jstack<pid>  on the pig client process id a few times and see 
>>> what it is doing when it is taking so long and send the results ?
>>> With that we should be able to easily identify what might be happening and 
>>> suggest a fix.
>>> 
>>> I think pig calls getSchema() on load functions more times than actually 
>>> necessary during the query optimization phase (each time after a 
>>> transform). One theory I have is that your avro load function does a stat 
>>> on all the 50k files each time to determine the common schema. This might 
>>> be causing the delay to generate query plan.
>>> 
>>> About the second large lag before queue seems to be time spend in input 
>>> split calculations, not within pig logic.
>>> 
>>> These are just theories I have. The jstack output would help determine if 
>>> this is actually the case.
>>> 
>>> Also, I think this delay is likely to be caused by the number of input 
>>> files, and not the actual data size. You might also want to look at ways to 
>>> reduce the actual number of input files. Is it generated by a previous MR 
>>> job ? If yes, given your cluster size, it works against you to have so many 
>>> maps or reducers in the previous MR job. If pig is generating the data,  
>>> you should consider setting the parallelism (reducers) or the number of 
>>> maps (see http://pig.apache.org/docs/r0.10.0/perf.html#combine-files) .
>>> While reducing number of input files will most likely solve your problem, I 
>>> am still interested in knowing where the delays are coming from and if we 
>>> can fix something in pig to improve the situation.
>>> 
>>> Thanks,
>>> Thejas
>>> 
>>> 
>>> On 6/1/12 6:34 AM, Markus Resch wrote:
>>>> Hi Prashant, Hi Thejas,
>>>> 
>>>> thanks for your very quick answer.
>>>> No, this is not a typo. Those time stamps are true and as I said the
>>>> machines are not very busy during this time.
>>>> 
>>>> As this is our test cluster I am sure I am the only one who is running
>>>> jobs on it. Another issue we have is that we are currently only able to
>>>> run one job at a time but this shouldn't be the topic of this request.
>>>> We even have no continuous input stream to that cluster but copied a
>>>> bunch of data to it some time ago.
>>>> From my perspective the 464 GB of input data you are mentioned is the
>>>> uncompressed amount of the 160GByte compressed files. Which I get when I
>>>> use hadoop -f dus on that folder.
>>>> 
>>>> Another interesting fact for you could be that we're running the
>>>> cloudera CDH3 Update 3 version on our systems.
>>>> 
>>>> I suspect this could be due to some fancy avro schema validation
>>>> implicitly executed by the avro storage? If so, can this be avoided?
>>>> 
>>>> Sadly I'm currently not able to provide you the actual script currently
>>>> as it contains confidential information but I will try to provide you a
>>>> version as soon as possible. But I'd rather think of a configuration
>>>> problem to the hadoop or pig anyways as the script works fine with a
>>>> smaller amount of input data
>>>> 
>>>> I would ask the hadoop mailing list if this issue would occur during the
>>>> actual mapred run but as this occur even before a single mapred job is
>>>> launched I suspect pig to have a problem.
>>>> 
>>>> Thanks
>>>> Markus
>>>> 
>>>> This is the full log until the main work job starts:
>>>> mapred@ournamenode$ pig OurScript.pig
>>>> 2012-05-30 15:27:21,052 [main] INFO  org.apache.pig.Main - Logging error
>>>> messages to: /tmp/pig_1338384441037.log
>>>> 2012-05-30 15:27:21,368 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.HExecutionEngine -
>>>> Connecting to hadoop file system at: hdfs://OurNamenode:9000
>>>> 2012-05-30 15:27:21,609 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.HExecutionEngine -
>>>> Connecting to map-reduce job tracker at:
>>>> dev-jobtracker001.eu-fra.adtech.com:54311
>>>> 2012-05-30 15:57:27,814 [main] WARN  org.apache.pig.PigServer -
>>>> Encountered Warning IMPLICIT_CAST_TO_LONG 1 time(s).
>>>> 2012-05-30 15:57:27,816 [main] INFO
>>>> org.apache.pig.tools.pigstats.ScriptState - Pig features used in the
>>>> script: REPLICATED_JOIN,COGROUP,GROUP_BY,FILTER
>>>> 2012-05-30 15:57:27,816 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.HExecutionEngine -
>>>> pig.usenewlogicalplan is set to true. New logical plan will be used.
>>>> 2012-05-30 16:06:55,304 [main] INFO
>>>> org.apache.pig.newplan.logical.rules.ColumnPruneVisitor - Columns pruned
>>>> for CampaignInfo: $0, $1, $2, $4, $5, $6, $8, $9
>>>> 2012-05-30 16:06:55,308 [main] INFO
>>>> org.apache.pig.newplan.logical.rules.ColumnPruneVisitor - Columns pruned
>>>> for dataImport: $2, $3, $4
>>>> 2012-05-30 16:06:55,441 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.HExecutionEngine - (Name:
>>>> OutputData1:
>>>> Store(SomeOutputFile1.csv:org.apache.pig.builtin.PigStorage) - scope-521
>>>> Operator Key: scope-521)
>>>> 2012-05-30 16:06:55,441 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.HExecutionEngine - (Name:
>>>> OutputData2:
>>>> Store(/SomeOutputFile2.csv:org.apache.pig.builtin.PigStorage) -
>>>> scope-524 Operator Key: scope-524)
>>>> 2012-05-30 16:06:55,441 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.HExecutionEngine - (Name:
>>>> OutputData2:
>>>> Store(/SomeOutputFile3.csv:org.apache.pig.builtin.PigStorage) -
>>>> scope-483 Operator Key: scope-483)
>>>> 2012-05-30 16:06:55,453 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler
>>>> - File concatenation threshold: 100 optimistic? false
>>>> 2012-05-30 16:06:55,467 [main] INFO
>>>> org.apache.hadoop.mapreduce.lib.input.FileInputFormat - Total input
>>>> paths to process : 1
>>>> 2012-05-30 16:06:55,471 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil - Total
>>>> input paths to process : 1
>>>> 2012-05-30 16:06:55,483 [main] WARN
>>>> org.apache.hadoop.io.compress.snappy.LoadSnappy - Snappy native library
>>>> is available
>>>> 2012-05-30 16:06:55,484 [main] INFO
>>>> org.apache.hadoop.util.NativeCodeLoader - Loaded the native-hadoop
>>>> library
>>>> 2012-05-30 16:06:55,484 [main] INFO
>>>> org.apache.hadoop.io.compress.snappy.LoadSnappy - Snappy native library
>>>> loaded
>>>> 2012-05-30 16:06:55,486 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil - Total
>>>> input paths (combined) to process : 1
>>>> 2012-05-30 16:06:55,486 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler
>>>> - number of input files: 1
>>>> 2012-05-30 16:06:55,516 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MultiQueryOptimizer
>>>>  - MR plan size before optimization: 7
>>>> 2012-05-30 16:06:55,516 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MultiQueryOptimizer
>>>>  - Merged 2 map-only splittees.
>>>> 2012-05-30 16:06:55,516 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MultiQueryOptimizer
>>>>  - Merged 2 out of total 3 MR operators.
>>>> 2012-05-30 16:06:55,517 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MultiQueryOptimizer
>>>>  - Merged 2 map-only splittees.
>>>> 2012-05-30 16:06:55,517 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MultiQueryOptimizer
>>>>  - Merged 2 out of total 3 MR operators.
>>>> 2012-05-30 16:06:55,517 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MultiQueryOptimizer
>>>>  - Merged 0 out of total 2 MR operators.
>>>> 2012-05-30 16:06:55,517 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MultiQueryOptimizer
>>>>  - MR plan size after optimization: 3
>>>> 2012-05-30 16:06:56,131 [main] INFO
>>>> org.apache.pig.tools.pigstats.ScriptState - Pig script settings are
>>>> added to the job
>>>> 2012-05-30 16:06:56,138 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler
>>>>  - mapred.job.reduce.markreset.buffer.percent is not set, set to default 
>>>> 0.3
>>>> 2012-05-30 16:07:00,432 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler
>>>>  - Setting up single store job
>>>> 2012-05-30 16:07:00,468 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher
>>>>  - 1 map-reduce job(s) waiting for submission
>>>> 2012-05-30 16:07:00,969 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher
>>>>  - 0% complete
>>>> 2012-05-30 16:07:01,460 [Thread-4] INFO
>>>> org.apache.hadoop.mapreduce.lib.input.FileInputFormat - Total input
>>>> paths to process : 1
>>>> 2012-05-30 16:07:01,460 [Thread-4] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil - Total
>>>> input paths to process : 1
>>>> 2012-05-30 16:07:01,462 [Thread-4] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil - Total
>>>> input paths (combined) to process : 1
>>>> 2012-05-30 16:07:02,353 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher
>>>>  - HadoopJobId: job_201205080931_0178
>>>> 2012-05-30 16:07:02,353 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher
>>>>  - More information at: 
>>>> http://ourJobtracker:50030/jobdetails.jsp?jobid=job_201205080931_0178
>>>> 2012-05-30 16:07:12,421 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher
>>>>  - 16% complete
>>>> 2012-05-30 16:07:13,929 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher
>>>>  - 33% complete
>>>> 2012-05-30 16:07:16,969 [main] INFO
>>>> org.apache.pig.tools.pigstats.ScriptState - Pig script settings are
>>>> added to the job
>>>> 2012-05-30 16:07:16,970 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler
>>>>  - mapred.job.reduce.markreset.buffer.percent is not set, set to default 
>>>> 0.3
>>>> 2012-05-30 16:07:26,888 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler
>>>>  - Setting up single store job
>>>> 2012-05-30 16:07:30,886 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler
>>>>  - BytesPerReducer=1000000000 maxReducers=999 
>>>> totalInputFileSize=463325937621
>>>> 2012-05-30 16:07:30,886 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler
>>>>  - Neither PARALLEL nor default parallelism is set for this job. Setting 
>>>> number of reducers to 464
>>>> 2012-05-30 16:07:30,947 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher
>>>>  - 1 map-reduce job(s) waiting for submission.
>>>> 2012-05-30 16:15:38,022 [Thread-16] INFO
>>>> org.apache.hadoop.mapreduce.lib.input.FileInputFormat - Total input
>>>> paths to process : 50353
>>>> 2012-05-30 16:15:54,421 [Thread-16] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil - Total
>>>> input paths (combined) to process : 6899
>>>> 2012-05-30 16:15:57,844 [main] INFO
>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher
>>>>  - HadoopJobId: job_201205080931_0179
>>>> 
>>>> 
>>>> 
>>>> 
>>>> Am Donnerstag, den 31.05.2012, 02:57 -0700 schrieb Prashant Kommireddi:
>>>>> Hi Markus,
>>>>> 
>>>>> Have you checked the JobTracker at the time of launching the job that Map
>>>>> slots were available?
>>>>> 
>>>>> Looks like the input dataset size is ~464 GB. Since you mentioned 10 GB
>>>>> jobs are running fine, there should be no reason a larger dataset should 
>>>>> be
>>>>> stuck, atleast not on Pig side. I can't think of a good reason why the job
>>>>> does not take off other than the fact that cluster was busy running some
>>>>> other job.
>>>>> 
>>>>> I see that the number of files being processed is large, 50353. That could
>>>>> be a reason for slowness, but ~8 minutes as shown in the logs seems to be
>>>>> on the higher end for that.
>>>>> 
>>>>> May be also post your script here.
>>>>> 
>>>>> On Thu, May 31, 2012 at 2:38 AM, Markus 
>>>>> Resch<[email protected]>wrote:
>>>>> 
>>>>>> Hi all,
>>>>>> 
>>>>>> when we're running a pig job for aggregating some amount of slightly
>>>>>> compressed avro data (~160GByte), the time until the first actual mapred
>>>>>> job starts takes ages:
>>>>>> 15:27:21,052 [main] INFO  org.apache.pig.Main - Logging error messages
>>>>>> to:
>>>>>> [...]
>>>>>> 15:57:27,816 [main] INFO  org.apache.pig.tools.pigstats.ScriptState -
>>>>>> Pig features used in the script:
>>>>>> [...]
>>>>>> 16:07:00,969 [main] INFO
>>>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher
>>>>>> - 0% complete
>>>>>> [...]
>>>>>> 16:07:30,886 [main] INFO
>>>>>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler
>>>>>> - BytesPerReducer=1000000000 maxReducers=999 
>>>>>> totalInputFileSize=463325937621
>>>>>> [...]
>>>>>> 16:15:38,022 [Thread-16] INFO
>>>>>> org.apache.hadoop.mapreduce.lib.input.FileInputFormat - Total input
>>>>>> paths to process : 50353
>>>>>> 
>>>>>> This log messages are from our test cluster which has a dedicated
>>>>>> jobtracker and namenode each and 5 data nodes with a map task capacity
>>>>>> of 15 and a reduce task capacity of 10. There were 6899 map tasks and
>>>>>> 464 reduce tasks set up.
>>>>>> 
>>>>>> During the initialisation phase we were observing the work load and
>>>>>> memory usage of jobtracker, namenode and some data nodes using top.
>>>>>> Those were nearly all the time kind of bored (e.g. 30% cpu load on the
>>>>>> namenode, total idle on he data nodes). When the jobs were running most
>>>>>> of the tasks where in "waiting for IO" most of the time. It seemed there
>>>>>> was some swapping space reserved but rarely used in those times.
>>>>>> 
>>>>>> In our eyes it looks like a hadoop config issue but we have no idea what
>>>>>> it exaclty could be. Jobs with about 10GBytes of input data were running
>>>>>> quite well.
>>>>>> 
>>>>>> Any hint where to tweak will be appreciated
>>>>>> 
>>>>>> Thanks
>>>>>> Markus
>>>>>> 
>>>>>> 
>>>> 
>>> 
> 

Reply via email to