You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pig.apache.org by Suhas Satish <su...@gmail.com> on 2014/03/07 00:05:21 UTC

pig12 job stuck in infinite loop

Hi
I launched the attached pig job on pig-12 with hadoop MRv1 with the
attached data, but the FILTER function causes the job to get stuck in an
infinite loop.

pig -p pPeriod=201312 -f test.pig

The thread in question seems to be stuck forever inside while loop of
runPipeline method.

stack trace:
-----------

"main" prio=10 tid=0x00007fd74800b000 nid=0x2f63 runnable
[0x00007fd750d50000]
   java.lang.Thread.State: RUNNABLE
    at
org.apache.pig.backend.hadoop.executionengine.physicalLayer.
relationalOperators.POForEach.getNextTuple(POForEach.java:217)
    at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
PigGenericMapBase.runPipeline(PigGenericMapBase.java:282)
    at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
PigGenericMapBase.map(PigGenericMapBase.java:277)
    at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
PigGenericMapBase.map(PigGenericMapBase.java:64)
    at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
    at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:680)
    at org.apache.hadoop.mapred.MapTask.run(MapTask.java:346)
    at org.apache.hadoop.mapred.Child$4.run(Child.java:282)
    at java.security.AccessController.doPrivileged(Native Method)
    at javax.security.auth.Subject.doAs(Subject.java:415)
    at
org.apache.hadoop.security.UserGroupInformation.doAs(
UserGroupInformation.java:1117)
    at org.apache.hadoop.mapred.Child.main(Child.java:271)




org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
PigGenericMapBase.java:

protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
InterruptedException {
        while(true){
            Result res = leaf.getNext(DUMMYTUPLE);
            if(res.returnStatus==POStatus.STATUS_OK){
                collect(outputCollector,(Tuple)res.result);
                continue;
            }
....



Whats the suggested code fix here?


Thanks,
Suhas.

Re: pig12 job stuck in infinite loop

Posted by Suhas Satish <su...@gmail.com>.
I was able to reproduce the issue on apache hadoop with latest apache pig
trunk, so I dont think its a MapR distribution -specific issue.

The infinite looping is through this section where only empty tuples () are
being generated with STATUS_OK, without a terminating condition being set.
I could not find a related JIRA, shall I open a new one for this?


protected void runPipeline(PhysicalOperator leaf) throws IOException,
InterruptedException {

     while(true){
            Result res = leaf.getNextTuple();
            log.debug(res.toString());
            log.debug("Suhas result's return status="+ res.returnStatus);
           * if(res.returnStatus==POStatus.STATUS_OK){*
*                collect(outputCollector,(Tuple)res.result);*
*                continue;*
*            }*

            if(res.returnStatus==POStatus.STATUS_EOP) {
                return;
            }

            if(res.returnStatus==POStatus.STATUS_NULL)
                continue;

            if(res.returnStatus==POStatus.STATUS_ERR){
                // remember that we had an issue so that in
                // close() we can do the right thing
                errorInMap  = true;
                // if there is an errmessage use it
                String errMsg;
                if(res.result != null) {
                    errMsg = "Received Error while " +
                    "processing the map plan: " + res.result;
                } else {
                    errMsg = "Received Error while " +
                    "processing the map plan.";
                }

                int errCode = 2055;
                ExecException ee = new ExecException(errMsg, errCode,
PigException.BUG);
                throw ee;
            }
        }

    }
=================================================================
How to interpret this physical plan from the leaf?

2014-03-12 11:32:36,609 DEBUG
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapOnly$Map:
*gpWeekRanked:
New For Each()[bag] *- scope-29
|
|---New For Each(false,true)[tuple] - scope-28
    |   |
    |   Project[long][0] - scope-26
    |   |
    |   Project[bag][2] - scope-27
    |
    |---pWeekRanked: PORank[tuple] - scope-25
        |   |
        |   Project[int][0] - scope-22
2014-03-12 11:32:36,610 INFO
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapOnly$Map:
Aliases being processed per job phase*(AliasName[line,offset]):* *M:
gpWeekRanked[11,15] *C:  R:
=================================================================
pig script -

tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS (WEEK:int,
DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);

gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;

pWeek = FILTER gTWeek BY (PERIOD == 201312);

pWeekRanked = RANK pWeek BY WEEK ASC DENSE;

gpWeekRanked = FOREACH pWeekRanked GENERATE $0;

store gpWeekRanked INTO 'gpWeekRanked2';

describe gpWeekRanked2;
=================================================================



Cheers,
Suhas.


On Fri, Mar 7, 2014 at 10:20 AM, Cheolsoo Park <pi...@gmail.com> wrote:

> I just ran your script in local mode with -Dhadoopversion=20 (Hadoop 1)
> using Apache Pig.
>
> 1) Branch 0.11 and 0.12 fail with NPE
>
> Caused by: java.lang.NullPointerException
>         at
> org.apache.hadoop.mapred.Counters.getShortName(Counters.java:669)
>         at org.apache.hadoop.mapred.Counters.getGroup(Counters.java:405)
>         at
>
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler.saveCounters(JobControlCompiler.java:360)
>         ... 12 more
>
> This is fixed in trunk.
>
> 2) Trunk works fine.
>
> So I cannot reproduce your problem. If you're using a specific distribution
> (such as MapR), it should be a distribution specific issue.
>
> Thanks,
> Cheolsoo
>
>
> On Thu, Mar 6, 2014 at 7:38 PM, Suhas Satish <su...@gmail.com>
> wrote:
>
> > The example that reproduces the issue along with data is attached in the
> > very first email on this thread
> >
> > On Thursday, March 6, 2014, Cheolsoo Park <pi...@gmail.com> wrote:
> >
> > > So that's backend. It has nothing to do with the filter extractor. The
> > > filter extractor is for predicate push down on the frontend.
> > >
> > > The code that you're showing is the entry point where Pig mapper
> begins.
> > So
> > > it doesn't tell us much. The mapper is given a segment of physical plan
> > > (pipeline), and the getNext() call pulls records from roots to leaves
> one
> > > by one.
> > >
> > > You need to find where time is spent in the pipeline. If you're
> > suspecting
> > > Filter By is slow, then it should be POFilter. Please take thread dump
> > > multiple times and see the stack traces. Unless you provide an example
> > that
> > > reproduces the error, I cannot help you more.
> > >
> > >
> > >
> > > On Thu, Mar 6, 2014 at 6:03 PM, Suhas Satish <suhas.satish@gmail.com
> > <javascript:;>>
> > > wrote:
> > >
> > > > Hi Cheolsoo,
> > > > This is where its hanging -
> > > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > > >
> > > > org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> > > > PigGenericMapBase.java:
> > > >
> > > > protected void *runPipeline*(PhysicalOperator leaf) throws
> IOException,
> > > > InterruptedException {
> > > >         while(true){
> > > >             Result res = leaf.getNext(DUMMYTUPLE);
> > > >             if(res.returnStatus==POStatus.STATUS_OK){
> > > >                 collect(outputCollector,(Tuple)res.result);
> > > >                 continue;
> > > >             }
> > > > ....
> > > >
> > > > Cheers,
> > > > Suhas.
> > > >
> > > >
> > > > On Thu, Mar 6, 2014 at 5:56 PM, Cheolsoo Park <pi...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Suhas,
> > > > >
> > > > > No. The issue with PIG-3461 is that Pig hangs at the query
> > compilation
> > > > with
> > > > > a big filter expression before the job is submitted.
> > > > > In addition, the filter extractor was totally rewritten in 0.12.
> > > > > https://issues.apache.org/jira/browse/PIG-3461
> > > > >
> > > > > Where exactly is your job hanging? Backend or frontend? Are you
> > running
> > > > it
> > > > > in local mode or remote mode?
> > > > >
> > > > > Thanks,
> > > > > Cheolsoo
> > > > >
> > > > > p.s.
> > > > > There are two known issues with the new filter extractor in 0.12.0
> > > > although
> > > > > these are probably not related to your issue-
> > > > > https://issues.apache.org/jira/browse/PIG-3510
> > > > > https://issues.apache.org/jira/browse/PIG-3657
> > > > >
> > > > >
> > > > > On Thu, Mar 6, 2014 at 5:30 PM, Suhas Satish <
> suhas.satish@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > I seem to be hitting this issue in pig-0.12 although it claims to
> > be
> > > > > fixed
> > > > > > in pig-0.12
> > > > > > https://issues.apache.org/jira/browse/PIG-3395
> > > > > > Large filter expression makes Pig hang
> > > > > >
> > > > > > Cheers,
> > > > > > Suhas.
> > > > > >
> > > > > >
> > > > > > On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <
> > suhas.satish@gmail.com
> > > >
> > > > > > wrote:
> > > > > >
> > > > > > > This is the pig script -
> > > > > > >
> > > > > > > %default previousPeriod $pPeriod
> > > > > > >
> > > > > > > tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS
> > > > (WEEK:int,
> > > > > > > DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
> > > > > > >
> > > > > > > gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
> > > > > > >
> > > > > > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > > > > > >
> > > > > > > pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
> > > > > > >
> > > > > > > gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> > > > > > > store gpWeekRanked INTO 'gpWeekRanked';
> > > > > > > describe gpWeekRanked;
> > > > > > >
> > > > > > >
> > > > > > > Without the filter statement, the code runs without hanging.
> > > > > > >
> > > > > > > Cheers,
> > > > > > > Suhas.
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <
> > > suhas.satish@gmail.com
> > > > > > >wrote:
> > > > > > >
> > > > > > >> Hi
> > > > > > >> I launched the attached pig job on pig-12 with hadoop MRv1
> with
> > > the
> > > > > > >> attached data, but the FILTER function causes the job to get
> > stuck
> > > > in
> > > > > an
> > > > > > >> infinite loop.
> > > > > > >>
> > > > > > >> pig -p pPeriod=201312 -f test.pig
> > > > > > >>
> > > > > > >> The thread in question seems to be stuck forever inside while
> > loop
> > > > of
> > > > > > >> runPipel
> >
> >
> >
> > --
> > Cheers,
> > Suhas.
> >
>

Re: pig12 job stuck in infinite loop

Posted by Suhas Satish <su...@gmail.com>.
I was able to reproduce the issue on apache hadoop with latest apache pig
trunk, so I dont think its a MapR distribution -specific issue.

The infinite looping is through this section where only empty tuples () are
being generated with STATUS_OK, without a terminating condition being set.
I could not find a related JIRA, shall I open a new one for this?


protected void runPipeline(PhysicalOperator leaf) throws IOException,
InterruptedException {

     while(true){
            Result res = leaf.getNextTuple();
            log.debug(res.toString());
            log.debug("Suhas result's return status="+ res.returnStatus);
           * if(res.returnStatus==POStatus.STATUS_OK){*
*                collect(outputCollector,(Tuple)res.result);*
*                continue;*
*            }*

            if(res.returnStatus==POStatus.STATUS_EOP) {
                return;
            }

            if(res.returnStatus==POStatus.STATUS_NULL)
                continue;

            if(res.returnStatus==POStatus.STATUS_ERR){
                // remember that we had an issue so that in
                // close() we can do the right thing
                errorInMap  = true;
                // if there is an errmessage use it
                String errMsg;
                if(res.result != null) {
                    errMsg = "Received Error while " +
                    "processing the map plan: " + res.result;
                } else {
                    errMsg = "Received Error while " +
                    "processing the map plan.";
                }

                int errCode = 2055;
                ExecException ee = new ExecException(errMsg, errCode,
PigException.BUG);
                throw ee;
            }
        }

    }
=================================================================
How to interpret this physical plan from the leaf?

2014-03-12 11:32:36,609 DEBUG
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapOnly$Map:
*gpWeekRanked:
New For Each()[bag] *- scope-29
|
|---New For Each(false,true)[tuple] - scope-28
    |   |
    |   Project[long][0] - scope-26
    |   |
    |   Project[bag][2] - scope-27
    |
    |---pWeekRanked: PORank[tuple] - scope-25
        |   |
        |   Project[int][0] - scope-22
2014-03-12 11:32:36,610 INFO
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapOnly$Map:
Aliases being processed per job phase*(AliasName[line,offset]):* *M:
gpWeekRanked[11,15] *C:  R:
=================================================================
pig script -

tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS (WEEK:int,
DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);

gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;

pWeek = FILTER gTWeek BY (PERIOD == 201312);

pWeekRanked = RANK pWeek BY WEEK ASC DENSE;

gpWeekRanked = FOREACH pWeekRanked GENERATE $0;

store gpWeekRanked INTO 'gpWeekRanked2';

describe gpWeekRanked2;
=================================================================



Cheers,
Suhas.


On Fri, Mar 7, 2014 at 10:20 AM, Cheolsoo Park <pi...@gmail.com> wrote:

> I just ran your script in local mode with -Dhadoopversion=20 (Hadoop 1)
> using Apache Pig.
>
> 1) Branch 0.11 and 0.12 fail with NPE
>
> Caused by: java.lang.NullPointerException
>         at
> org.apache.hadoop.mapred.Counters.getShortName(Counters.java:669)
>         at org.apache.hadoop.mapred.Counters.getGroup(Counters.java:405)
>         at
>
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler.saveCounters(JobControlCompiler.java:360)
>         ... 12 more
>
> This is fixed in trunk.
>
> 2) Trunk works fine.
>
> So I cannot reproduce your problem. If you're using a specific distribution
> (such as MapR), it should be a distribution specific issue.
>
> Thanks,
> Cheolsoo
>
>
> On Thu, Mar 6, 2014 at 7:38 PM, Suhas Satish <su...@gmail.com>
> wrote:
>
> > The example that reproduces the issue along with data is attached in the
> > very first email on this thread
> >
> > On Thursday, March 6, 2014, Cheolsoo Park <pi...@gmail.com> wrote:
> >
> > > So that's backend. It has nothing to do with the filter extractor. The
> > > filter extractor is for predicate push down on the frontend.
> > >
> > > The code that you're showing is the entry point where Pig mapper
> begins.
> > So
> > > it doesn't tell us much. The mapper is given a segment of physical plan
> > > (pipeline), and the getNext() call pulls records from roots to leaves
> one
> > > by one.
> > >
> > > You need to find where time is spent in the pipeline. If you're
> > suspecting
> > > Filter By is slow, then it should be POFilter. Please take thread dump
> > > multiple times and see the stack traces. Unless you provide an example
> > that
> > > reproduces the error, I cannot help you more.
> > >
> > >
> > >
> > > On Thu, Mar 6, 2014 at 6:03 PM, Suhas Satish <suhas.satish@gmail.com
> > <javascript:;>>
> > > wrote:
> > >
> > > > Hi Cheolsoo,
> > > > This is where its hanging -
> > > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > > >
> > > > org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> > > > PigGenericMapBase.java:
> > > >
> > > > protected void *runPipeline*(PhysicalOperator leaf) throws
> IOException,
> > > > InterruptedException {
> > > >         while(true){
> > > >             Result res = leaf.getNext(DUMMYTUPLE);
> > > >             if(res.returnStatus==POStatus.STATUS_OK){
> > > >                 collect(outputCollector,(Tuple)res.result);
> > > >                 continue;
> > > >             }
> > > > ....
> > > >
> > > > Cheers,
> > > > Suhas.
> > > >
> > > >
> > > > On Thu, Mar 6, 2014 at 5:56 PM, Cheolsoo Park <pi...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Suhas,
> > > > >
> > > > > No. The issue with PIG-3461 is that Pig hangs at the query
> > compilation
> > > > with
> > > > > a big filter expression before the job is submitted.
> > > > > In addition, the filter extractor was totally rewritten in 0.12.
> > > > > https://issues.apache.org/jira/browse/PIG-3461
> > > > >
> > > > > Where exactly is your job hanging? Backend or frontend? Are you
> > running
> > > > it
> > > > > in local mode or remote mode?
> > > > >
> > > > > Thanks,
> > > > > Cheolsoo
> > > > >
> > > > > p.s.
> > > > > There are two known issues with the new filter extractor in 0.12.0
> > > > although
> > > > > these are probably not related to your issue-
> > > > > https://issues.apache.org/jira/browse/PIG-3510
> > > > > https://issues.apache.org/jira/browse/PIG-3657
> > > > >
> > > > >
> > > > > On Thu, Mar 6, 2014 at 5:30 PM, Suhas Satish <
> suhas.satish@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > I seem to be hitting this issue in pig-0.12 although it claims to
> > be
> > > > > fixed
> > > > > > in pig-0.12
> > > > > > https://issues.apache.org/jira/browse/PIG-3395
> > > > > > Large filter expression makes Pig hang
> > > > > >
> > > > > > Cheers,
> > > > > > Suhas.
> > > > > >
> > > > > >
> > > > > > On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <
> > suhas.satish@gmail.com
> > > >
> > > > > > wrote:
> > > > > >
> > > > > > > This is the pig script -
> > > > > > >
> > > > > > > %default previousPeriod $pPeriod
> > > > > > >
> > > > > > > tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS
> > > > (WEEK:int,
> > > > > > > DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
> > > > > > >
> > > > > > > gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
> > > > > > >
> > > > > > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > > > > > >
> > > > > > > pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
> > > > > > >
> > > > > > > gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> > > > > > > store gpWeekRanked INTO 'gpWeekRanked';
> > > > > > > describe gpWeekRanked;
> > > > > > >
> > > > > > >
> > > > > > > Without the filter statement, the code runs without hanging.
> > > > > > >
> > > > > > > Cheers,
> > > > > > > Suhas.
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <
> > > suhas.satish@gmail.com
> > > > > > >wrote:
> > > > > > >
> > > > > > >> Hi
> > > > > > >> I launched the attached pig job on pig-12 with hadoop MRv1
> with
> > > the
> > > > > > >> attached data, but the FILTER function causes the job to get
> > stuck
> > > > in
> > > > > an
> > > > > > >> infinite loop.
> > > > > > >>
> > > > > > >> pig -p pPeriod=201312 -f test.pig
> > > > > > >>
> > > > > > >> The thread in question seems to be stuck forever inside while
> > loop
> > > > of
> > > > > > >> runPipel
> >
> >
> >
> > --
> > Cheers,
> > Suhas.
> >
>

Re: pig12 job stuck in infinite loop

Posted by Cheolsoo Park <pi...@gmail.com>.
I just ran your script in local mode with -Dhadoopversion=20 (Hadoop 1)
using Apache Pig.

1) Branch 0.11 and 0.12 fail with NPE

Caused by: java.lang.NullPointerException
        at org.apache.hadoop.mapred.Counters.getShortName(Counters.java:669)
        at org.apache.hadoop.mapred.Counters.getGroup(Counters.java:405)
        at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler.saveCounters(JobControlCompiler.java:360)
        ... 12 more

This is fixed in trunk.

2) Trunk works fine.

So I cannot reproduce your problem. If you're using a specific distribution
(such as MapR), it should be a distribution specific issue.

Thanks,
Cheolsoo


On Thu, Mar 6, 2014 at 7:38 PM, Suhas Satish <su...@gmail.com> wrote:

> The example that reproduces the issue along with data is attached in the
> very first email on this thread
>
> On Thursday, March 6, 2014, Cheolsoo Park <pi...@gmail.com> wrote:
>
> > So that's backend. It has nothing to do with the filter extractor. The
> > filter extractor is for predicate push down on the frontend.
> >
> > The code that you're showing is the entry point where Pig mapper begins.
> So
> > it doesn't tell us much. The mapper is given a segment of physical plan
> > (pipeline), and the getNext() call pulls records from roots to leaves one
> > by one.
> >
> > You need to find where time is spent in the pipeline. If you're
> suspecting
> > Filter By is slow, then it should be POFilter. Please take thread dump
> > multiple times and see the stack traces. Unless you provide an example
> that
> > reproduces the error, I cannot help you more.
> >
> >
> >
> > On Thu, Mar 6, 2014 at 6:03 PM, Suhas Satish <suhas.satish@gmail.com
> <javascript:;>>
> > wrote:
> >
> > > Hi Cheolsoo,
> > > This is where its hanging -
> > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > >
> > > org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> > > PigGenericMapBase.java:
> > >
> > > protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
> > > InterruptedException {
> > >         while(true){
> > >             Result res = leaf.getNext(DUMMYTUPLE);
> > >             if(res.returnStatus==POStatus.STATUS_OK){
> > >                 collect(outputCollector,(Tuple)res.result);
> > >                 continue;
> > >             }
> > > ....
> > >
> > > Cheers,
> > > Suhas.
> > >
> > >
> > > On Thu, Mar 6, 2014 at 5:56 PM, Cheolsoo Park <pi...@gmail.com>
> > > wrote:
> > >
> > > > Hi Suhas,
> > > >
> > > > No. The issue with PIG-3461 is that Pig hangs at the query
> compilation
> > > with
> > > > a big filter expression before the job is submitted.
> > > > In addition, the filter extractor was totally rewritten in 0.12.
> > > > https://issues.apache.org/jira/browse/PIG-3461
> > > >
> > > > Where exactly is your job hanging? Backend or frontend? Are you
> running
> > > it
> > > > in local mode or remote mode?
> > > >
> > > > Thanks,
> > > > Cheolsoo
> > > >
> > > > p.s.
> > > > There are two known issues with the new filter extractor in 0.12.0
> > > although
> > > > these are probably not related to your issue-
> > > > https://issues.apache.org/jira/browse/PIG-3510
> > > > https://issues.apache.org/jira/browse/PIG-3657
> > > >
> > > >
> > > > On Thu, Mar 6, 2014 at 5:30 PM, Suhas Satish <suhas.satish@gmail.com
> >
> > > > wrote:
> > > >
> > > > > I seem to be hitting this issue in pig-0.12 although it claims to
> be
> > > > fixed
> > > > > in pig-0.12
> > > > > https://issues.apache.org/jira/browse/PIG-3395
> > > > > Large filter expression makes Pig hang
> > > > >
> > > > > Cheers,
> > > > > Suhas.
> > > > >
> > > > >
> > > > > On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <
> suhas.satish@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > This is the pig script -
> > > > > >
> > > > > > %default previousPeriod $pPeriod
> > > > > >
> > > > > > tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS
> > > (WEEK:int,
> > > > > > DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
> > > > > >
> > > > > > gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
> > > > > >
> > > > > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > > > > >
> > > > > > pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
> > > > > >
> > > > > > gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> > > > > > store gpWeekRanked INTO 'gpWeekRanked';
> > > > > > describe gpWeekRanked;
> > > > > >
> > > > > >
> > > > > > Without the filter statement, the code runs without hanging.
> > > > > >
> > > > > > Cheers,
> > > > > > Suhas.
> > > > > >
> > > > > >
> > > > > > On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <
> > suhas.satish@gmail.com
> > > > > >wrote:
> > > > > >
> > > > > >> Hi
> > > > > >> I launched the attached pig job on pig-12 with hadoop MRv1 with
> > the
> > > > > >> attached data, but the FILTER function causes the job to get
> stuck
> > > in
> > > > an
> > > > > >> infinite loop.
> > > > > >>
> > > > > >> pig -p pPeriod=201312 -f test.pig
> > > > > >>
> > > > > >> The thread in question seems to be stuck forever inside while
> loop
> > > of
> > > > > >> runPipel
>
>
>
> --
> Cheers,
> Suhas.
>

Re: pig12 job stuck in infinite loop

Posted by Cheolsoo Park <pi...@gmail.com>.
I just ran your script in local mode with -Dhadoopversion=20 (Hadoop 1)
using Apache Pig.

1) Branch 0.11 and 0.12 fail with NPE

Caused by: java.lang.NullPointerException
        at org.apache.hadoop.mapred.Counters.getShortName(Counters.java:669)
        at org.apache.hadoop.mapred.Counters.getGroup(Counters.java:405)
        at
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler.saveCounters(JobControlCompiler.java:360)
        ... 12 more

This is fixed in trunk.

2) Trunk works fine.

So I cannot reproduce your problem. If you're using a specific distribution
(such as MapR), it should be a distribution specific issue.

Thanks,
Cheolsoo


On Thu, Mar 6, 2014 at 7:38 PM, Suhas Satish <su...@gmail.com> wrote:

> The example that reproduces the issue along with data is attached in the
> very first email on this thread
>
> On Thursday, March 6, 2014, Cheolsoo Park <pi...@gmail.com> wrote:
>
> > So that's backend. It has nothing to do with the filter extractor. The
> > filter extractor is for predicate push down on the frontend.
> >
> > The code that you're showing is the entry point where Pig mapper begins.
> So
> > it doesn't tell us much. The mapper is given a segment of physical plan
> > (pipeline), and the getNext() call pulls records from roots to leaves one
> > by one.
> >
> > You need to find where time is spent in the pipeline. If you're
> suspecting
> > Filter By is slow, then it should be POFilter. Please take thread dump
> > multiple times and see the stack traces. Unless you provide an example
> that
> > reproduces the error, I cannot help you more.
> >
> >
> >
> > On Thu, Mar 6, 2014 at 6:03 PM, Suhas Satish <suhas.satish@gmail.com
> <javascript:;>>
> > wrote:
> >
> > > Hi Cheolsoo,
> > > This is where its hanging -
> > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > >
> > > org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> > > PigGenericMapBase.java:
> > >
> > > protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
> > > InterruptedException {
> > >         while(true){
> > >             Result res = leaf.getNext(DUMMYTUPLE);
> > >             if(res.returnStatus==POStatus.STATUS_OK){
> > >                 collect(outputCollector,(Tuple)res.result);
> > >                 continue;
> > >             }
> > > ....
> > >
> > > Cheers,
> > > Suhas.
> > >
> > >
> > > On Thu, Mar 6, 2014 at 5:56 PM, Cheolsoo Park <pi...@gmail.com>
> > > wrote:
> > >
> > > > Hi Suhas,
> > > >
> > > > No. The issue with PIG-3461 is that Pig hangs at the query
> compilation
> > > with
> > > > a big filter expression before the job is submitted.
> > > > In addition, the filter extractor was totally rewritten in 0.12.
> > > > https://issues.apache.org/jira/browse/PIG-3461
> > > >
> > > > Where exactly is your job hanging? Backend or frontend? Are you
> running
> > > it
> > > > in local mode or remote mode?
> > > >
> > > > Thanks,
> > > > Cheolsoo
> > > >
> > > > p.s.
> > > > There are two known issues with the new filter extractor in 0.12.0
> > > although
> > > > these are probably not related to your issue-
> > > > https://issues.apache.org/jira/browse/PIG-3510
> > > > https://issues.apache.org/jira/browse/PIG-3657
> > > >
> > > >
> > > > On Thu, Mar 6, 2014 at 5:30 PM, Suhas Satish <suhas.satish@gmail.com
> >
> > > > wrote:
> > > >
> > > > > I seem to be hitting this issue in pig-0.12 although it claims to
> be
> > > > fixed
> > > > > in pig-0.12
> > > > > https://issues.apache.org/jira/browse/PIG-3395
> > > > > Large filter expression makes Pig hang
> > > > >
> > > > > Cheers,
> > > > > Suhas.
> > > > >
> > > > >
> > > > > On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <
> suhas.satish@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > This is the pig script -
> > > > > >
> > > > > > %default previousPeriod $pPeriod
> > > > > >
> > > > > > tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS
> > > (WEEK:int,
> > > > > > DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
> > > > > >
> > > > > > gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
> > > > > >
> > > > > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > > > > >
> > > > > > pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
> > > > > >
> > > > > > gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> > > > > > store gpWeekRanked INTO 'gpWeekRanked';
> > > > > > describe gpWeekRanked;
> > > > > >
> > > > > >
> > > > > > Without the filter statement, the code runs without hanging.
> > > > > >
> > > > > > Cheers,
> > > > > > Suhas.
> > > > > >
> > > > > >
> > > > > > On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <
> > suhas.satish@gmail.com
> > > > > >wrote:
> > > > > >
> > > > > >> Hi
> > > > > >> I launched the attached pig job on pig-12 with hadoop MRv1 with
> > the
> > > > > >> attached data, but the FILTER function causes the job to get
> stuck
> > > in
> > > > an
> > > > > >> infinite loop.
> > > > > >>
> > > > > >> pig -p pPeriod=201312 -f test.pig
> > > > > >>
> > > > > >> The thread in question seems to be stuck forever inside while
> loop
> > > of
> > > > > >> runPipel
>
>
>
> --
> Cheers,
> Suhas.
>

Re: pig12 job stuck in infinite loop

Posted by Suhas Satish <su...@gmail.com>.
The example that reproduces the issue along with data is attached in the
very first email on this thread

On Thursday, March 6, 2014, Cheolsoo Park <pi...@gmail.com> wrote:

> So that's backend. It has nothing to do with the filter extractor. The
> filter extractor is for predicate push down on the frontend.
>
> The code that you're showing is the entry point where Pig mapper begins. So
> it doesn't tell us much. The mapper is given a segment of physical plan
> (pipeline), and the getNext() call pulls records from roots to leaves one
> by one.
>
> You need to find where time is spent in the pipeline. If you're suspecting
> Filter By is slow, then it should be POFilter. Please take thread dump
> multiple times and see the stack traces. Unless you provide an example that
> reproduces the error, I cannot help you more.
>
>
>
> On Thu, Mar 6, 2014 at 6:03 PM, Suhas Satish <suhas.satish@gmail.com<javascript:;>>
> wrote:
>
> > Hi Cheolsoo,
> > This is where its hanging -
> > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> >
> > org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> > PigGenericMapBase.java:
> >
> > protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
> > InterruptedException {
> >         while(true){
> >             Result res = leaf.getNext(DUMMYTUPLE);
> >             if(res.returnStatus==POStatus.STATUS_OK){
> >                 collect(outputCollector,(Tuple)res.result);
> >                 continue;
> >             }
> > ....
> >
> > Cheers,
> > Suhas.
> >
> >
> > On Thu, Mar 6, 2014 at 5:56 PM, Cheolsoo Park <pi...@gmail.com>
> > wrote:
> >
> > > Hi Suhas,
> > >
> > > No. The issue with PIG-3461 is that Pig hangs at the query compilation
> > with
> > > a big filter expression before the job is submitted.
> > > In addition, the filter extractor was totally rewritten in 0.12.
> > > https://issues.apache.org/jira/browse/PIG-3461
> > >
> > > Where exactly is your job hanging? Backend or frontend? Are you running
> > it
> > > in local mode or remote mode?
> > >
> > > Thanks,
> > > Cheolsoo
> > >
> > > p.s.
> > > There are two known issues with the new filter extractor in 0.12.0
> > although
> > > these are probably not related to your issue-
> > > https://issues.apache.org/jira/browse/PIG-3510
> > > https://issues.apache.org/jira/browse/PIG-3657
> > >
> > >
> > > On Thu, Mar 6, 2014 at 5:30 PM, Suhas Satish <su...@gmail.com>
> > > wrote:
> > >
> > > > I seem to be hitting this issue in pig-0.12 although it claims to be
> > > fixed
> > > > in pig-0.12
> > > > https://issues.apache.org/jira/browse/PIG-3395
> > > > Large filter expression makes Pig hang
> > > >
> > > > Cheers,
> > > > Suhas.
> > > >
> > > >
> > > > On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <suhas.satish@gmail.com
> >
> > > > wrote:
> > > >
> > > > > This is the pig script -
> > > > >
> > > > > %default previousPeriod $pPeriod
> > > > >
> > > > > tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS
> > (WEEK:int,
> > > > > DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
> > > > >
> > > > > gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
> > > > >
> > > > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > > > >
> > > > > pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
> > > > >
> > > > > gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> > > > > store gpWeekRanked INTO 'gpWeekRanked';
> > > > > describe gpWeekRanked;
> > > > >
> > > > >
> > > > > Without the filter statement, the code runs without hanging.
> > > > >
> > > > > Cheers,
> > > > > Suhas.
> > > > >
> > > > >
> > > > > On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <
> suhas.satish@gmail.com
> > > > >wrote:
> > > > >
> > > > >> Hi
> > > > >> I launched the attached pig job on pig-12 with hadoop MRv1 with
> the
> > > > >> attached data, but the FILTER function causes the job to get stuck
> > in
> > > an
> > > > >> infinite loop.
> > > > >>
> > > > >> pig -p pPeriod=201312 -f test.pig
> > > > >>
> > > > >> The thread in question seems to be stuck forever inside while loop
> > of
> > > > >> runPipel



-- 
Cheers,
Suhas.

Re: pig12 job stuck in infinite loop

Posted by Suhas Satish <su...@gmail.com>.
The example that reproduces the issue along with data is attached in the
very first email on this thread

On Thursday, March 6, 2014, Cheolsoo Park <pi...@gmail.com> wrote:

> So that's backend. It has nothing to do with the filter extractor. The
> filter extractor is for predicate push down on the frontend.
>
> The code that you're showing is the entry point where Pig mapper begins. So
> it doesn't tell us much. The mapper is given a segment of physical plan
> (pipeline), and the getNext() call pulls records from roots to leaves one
> by one.
>
> You need to find where time is spent in the pipeline. If you're suspecting
> Filter By is slow, then it should be POFilter. Please take thread dump
> multiple times and see the stack traces. Unless you provide an example that
> reproduces the error, I cannot help you more.
>
>
>
> On Thu, Mar 6, 2014 at 6:03 PM, Suhas Satish <suhas.satish@gmail.com<javascript:;>>
> wrote:
>
> > Hi Cheolsoo,
> > This is where its hanging -
> > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> >
> > org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> > PigGenericMapBase.java:
> >
> > protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
> > InterruptedException {
> >         while(true){
> >             Result res = leaf.getNext(DUMMYTUPLE);
> >             if(res.returnStatus==POStatus.STATUS_OK){
> >                 collect(outputCollector,(Tuple)res.result);
> >                 continue;
> >             }
> > ....
> >
> > Cheers,
> > Suhas.
> >
> >
> > On Thu, Mar 6, 2014 at 5:56 PM, Cheolsoo Park <pi...@gmail.com>
> > wrote:
> >
> > > Hi Suhas,
> > >
> > > No. The issue with PIG-3461 is that Pig hangs at the query compilation
> > with
> > > a big filter expression before the job is submitted.
> > > In addition, the filter extractor was totally rewritten in 0.12.
> > > https://issues.apache.org/jira/browse/PIG-3461
> > >
> > > Where exactly is your job hanging? Backend or frontend? Are you running
> > it
> > > in local mode or remote mode?
> > >
> > > Thanks,
> > > Cheolsoo
> > >
> > > p.s.
> > > There are two known issues with the new filter extractor in 0.12.0
> > although
> > > these are probably not related to your issue-
> > > https://issues.apache.org/jira/browse/PIG-3510
> > > https://issues.apache.org/jira/browse/PIG-3657
> > >
> > >
> > > On Thu, Mar 6, 2014 at 5:30 PM, Suhas Satish <su...@gmail.com>
> > > wrote:
> > >
> > > > I seem to be hitting this issue in pig-0.12 although it claims to be
> > > fixed
> > > > in pig-0.12
> > > > https://issues.apache.org/jira/browse/PIG-3395
> > > > Large filter expression makes Pig hang
> > > >
> > > > Cheers,
> > > > Suhas.
> > > >
> > > >
> > > > On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <suhas.satish@gmail.com
> >
> > > > wrote:
> > > >
> > > > > This is the pig script -
> > > > >
> > > > > %default previousPeriod $pPeriod
> > > > >
> > > > > tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS
> > (WEEK:int,
> > > > > DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
> > > > >
> > > > > gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
> > > > >
> > > > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > > > >
> > > > > pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
> > > > >
> > > > > gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> > > > > store gpWeekRanked INTO 'gpWeekRanked';
> > > > > describe gpWeekRanked;
> > > > >
> > > > >
> > > > > Without the filter statement, the code runs without hanging.
> > > > >
> > > > > Cheers,
> > > > > Suhas.
> > > > >
> > > > >
> > > > > On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <
> suhas.satish@gmail.com
> > > > >wrote:
> > > > >
> > > > >> Hi
> > > > >> I launched the attached pig job on pig-12 with hadoop MRv1 with
> the
> > > > >> attached data, but the FILTER function causes the job to get stuck
> > in
> > > an
> > > > >> infinite loop.
> > > > >>
> > > > >> pig -p pPeriod=201312 -f test.pig
> > > > >>
> > > > >> The thread in question seems to be stuck forever inside while loop
> > of
> > > > >> runPipel



-- 
Cheers,
Suhas.

Re: pig12 job stuck in infinite loop

Posted by Cheolsoo Park <pi...@gmail.com>.
So that's backend. It has nothing to do with the filter extractor. The
filter extractor is for predicate push down on the frontend.

The code that you're showing is the entry point where Pig mapper begins. So
it doesn't tell us much. The mapper is given a segment of physical plan
(pipeline), and the getNext() call pulls records from roots to leaves one
by one.

You need to find where time is spent in the pipeline. If you're suspecting
Filter By is slow, then it should be POFilter. Please take thread dump
multiple times and see the stack traces. Unless you provide an example that
reproduces the error, I cannot help you more.



On Thu, Mar 6, 2014 at 6:03 PM, Suhas Satish <su...@gmail.com> wrote:

> Hi Cheolsoo,
> This is where its hanging -
> *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
>
> org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> PigGenericMapBase.java:
>
> protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
> InterruptedException {
>         while(true){
>             Result res = leaf.getNext(DUMMYTUPLE);
>             if(res.returnStatus==POStatus.STATUS_OK){
>                 collect(outputCollector,(Tuple)res.result);
>                 continue;
>             }
> ....
>
> Cheers,
> Suhas.
>
>
> On Thu, Mar 6, 2014 at 5:56 PM, Cheolsoo Park <pi...@gmail.com>
> wrote:
>
> > Hi Suhas,
> >
> > No. The issue with PIG-3461 is that Pig hangs at the query compilation
> with
> > a big filter expression before the job is submitted.
> > In addition, the filter extractor was totally rewritten in 0.12.
> > https://issues.apache.org/jira/browse/PIG-3461
> >
> > Where exactly is your job hanging? Backend or frontend? Are you running
> it
> > in local mode or remote mode?
> >
> > Thanks,
> > Cheolsoo
> >
> > p.s.
> > There are two known issues with the new filter extractor in 0.12.0
> although
> > these are probably not related to your issue-
> > https://issues.apache.org/jira/browse/PIG-3510
> > https://issues.apache.org/jira/browse/PIG-3657
> >
> >
> > On Thu, Mar 6, 2014 at 5:30 PM, Suhas Satish <su...@gmail.com>
> > wrote:
> >
> > > I seem to be hitting this issue in pig-0.12 although it claims to be
> > fixed
> > > in pig-0.12
> > > https://issues.apache.org/jira/browse/PIG-3395
> > > Large filter expression makes Pig hang
> > >
> > > Cheers,
> > > Suhas.
> > >
> > >
> > > On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <su...@gmail.com>
> > > wrote:
> > >
> > > > This is the pig script -
> > > >
> > > > %default previousPeriod $pPeriod
> > > >
> > > > tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS
> (WEEK:int,
> > > > DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
> > > >
> > > > gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
> > > >
> > > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > > >
> > > > pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
> > > >
> > > > gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> > > > store gpWeekRanked INTO 'gpWeekRanked';
> > > > describe gpWeekRanked;
> > > >
> > > >
> > > > Without the filter statement, the code runs without hanging.
> > > >
> > > > Cheers,
> > > > Suhas.
> > > >
> > > >
> > > > On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <suhas.satish@gmail.com
> > > >wrote:
> > > >
> > > >> Hi
> > > >> I launched the attached pig job on pig-12 with hadoop MRv1 with the
> > > >> attached data, but the FILTER function causes the job to get stuck
> in
> > an
> > > >> infinite loop.
> > > >>
> > > >> pig -p pPeriod=201312 -f test.pig
> > > >>
> > > >> The thread in question seems to be stuck forever inside while loop
> of
> > > >> runPipeline method.
> > > >>
> > > >> stack trace:
> > > >> -----------
> > > >>
> > > >> "main" prio=10 tid=0x00007fd74800b000 nid=0x2f63 runnable
> > > >> [0x00007fd750d50000]
> > > >>    java.lang.Thread.State: RUNNABLE
> > > >>     at
> > > >> org.apache.pig.backend.hadoop.executionengine.physicalLayer.
> > > >> relationalOperators.POForEach.getNextTuple(POForEach.java:217)
> > > >>     at
> > > >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> > > >> PigGenericMapBase.runPipeline(PigGenericMapBase.java:282)
> > > >>     at
> > > >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> > > >> PigGenericMapBase.map(PigGenericMapBase.java:277)
> > > >>     at
> > > >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> > > >> PigGenericMapBase.map(PigGenericMapBase.java:64)
> > > >>     at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
> > > >>     at
> org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:680)
> > > >>     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:346)
> > > >>     at org.apache.hadoop.mapred.Child$4.run(Child.java:282)
> > > >>     at java.security.AccessController.doPrivileged(Native Method)
> > > >>     at javax.security.auth.Subject.doAs(Subject.java:415)
> > > >>     at
> > > >> org.apache.hadoop.security.UserGroupInformation.doAs(
> > > >> UserGroupInformation.java:1117)
> > > >>     at org.apache.hadoop.mapred.Child.main(Child.java:271)
> > > >>
> > > >>
> > > >>
> > > >>
> > > >> org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> > > >> PigGenericMapBase.java:
> > > >>
> > > >> protected void *runPipeline*(PhysicalOperator leaf) throws
> > IOException,
> > > >> InterruptedException {
> > > >>         while(true){
> > > >>             Result res = leaf.getNext(DUMMYTUPLE);
> > > >>             if(res.returnStatus==POStatus.STATUS_OK){
> > > >>                 collect(outputCollector,(Tuple)res.result);
> > > >>                 continue;
> > > >>             }
> > > >> ....
> > > >>
> > > >>
> > > >>
> > > >> Whats the suggested code fix here?
> > > >>
> > > >>
> > > >> Thanks,
> > > >> Suhas.
> > > >>
> > > >
> > > >
> > >
> >
>

Re: pig12 job stuck in infinite loop

Posted by Cheolsoo Park <pi...@gmail.com>.
So that's backend. It has nothing to do with the filter extractor. The
filter extractor is for predicate push down on the frontend.

The code that you're showing is the entry point where Pig mapper begins. So
it doesn't tell us much. The mapper is given a segment of physical plan
(pipeline), and the getNext() call pulls records from roots to leaves one
by one.

You need to find where time is spent in the pipeline. If you're suspecting
Filter By is slow, then it should be POFilter. Please take thread dump
multiple times and see the stack traces. Unless you provide an example that
reproduces the error, I cannot help you more.



On Thu, Mar 6, 2014 at 6:03 PM, Suhas Satish <su...@gmail.com> wrote:

> Hi Cheolsoo,
> This is where its hanging -
> *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
>
> org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> PigGenericMapBase.java:
>
> protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
> InterruptedException {
>         while(true){
>             Result res = leaf.getNext(DUMMYTUPLE);
>             if(res.returnStatus==POStatus.STATUS_OK){
>                 collect(outputCollector,(Tuple)res.result);
>                 continue;
>             }
> ....
>
> Cheers,
> Suhas.
>
>
> On Thu, Mar 6, 2014 at 5:56 PM, Cheolsoo Park <pi...@gmail.com>
> wrote:
>
> > Hi Suhas,
> >
> > No. The issue with PIG-3461 is that Pig hangs at the query compilation
> with
> > a big filter expression before the job is submitted.
> > In addition, the filter extractor was totally rewritten in 0.12.
> > https://issues.apache.org/jira/browse/PIG-3461
> >
> > Where exactly is your job hanging? Backend or frontend? Are you running
> it
> > in local mode or remote mode?
> >
> > Thanks,
> > Cheolsoo
> >
> > p.s.
> > There are two known issues with the new filter extractor in 0.12.0
> although
> > these are probably not related to your issue-
> > https://issues.apache.org/jira/browse/PIG-3510
> > https://issues.apache.org/jira/browse/PIG-3657
> >
> >
> > On Thu, Mar 6, 2014 at 5:30 PM, Suhas Satish <su...@gmail.com>
> > wrote:
> >
> > > I seem to be hitting this issue in pig-0.12 although it claims to be
> > fixed
> > > in pig-0.12
> > > https://issues.apache.org/jira/browse/PIG-3395
> > > Large filter expression makes Pig hang
> > >
> > > Cheers,
> > > Suhas.
> > >
> > >
> > > On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <su...@gmail.com>
> > > wrote:
> > >
> > > > This is the pig script -
> > > >
> > > > %default previousPeriod $pPeriod
> > > >
> > > > tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS
> (WEEK:int,
> > > > DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
> > > >
> > > > gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
> > > >
> > > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > > >
> > > > pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
> > > >
> > > > gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> > > > store gpWeekRanked INTO 'gpWeekRanked';
> > > > describe gpWeekRanked;
> > > >
> > > >
> > > > Without the filter statement, the code runs without hanging.
> > > >
> > > > Cheers,
> > > > Suhas.
> > > >
> > > >
> > > > On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <suhas.satish@gmail.com
> > > >wrote:
> > > >
> > > >> Hi
> > > >> I launched the attached pig job on pig-12 with hadoop MRv1 with the
> > > >> attached data, but the FILTER function causes the job to get stuck
> in
> > an
> > > >> infinite loop.
> > > >>
> > > >> pig -p pPeriod=201312 -f test.pig
> > > >>
> > > >> The thread in question seems to be stuck forever inside while loop
> of
> > > >> runPipeline method.
> > > >>
> > > >> stack trace:
> > > >> -----------
> > > >>
> > > >> "main" prio=10 tid=0x00007fd74800b000 nid=0x2f63 runnable
> > > >> [0x00007fd750d50000]
> > > >>    java.lang.Thread.State: RUNNABLE
> > > >>     at
> > > >> org.apache.pig.backend.hadoop.executionengine.physicalLayer.
> > > >> relationalOperators.POForEach.getNextTuple(POForEach.java:217)
> > > >>     at
> > > >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> > > >> PigGenericMapBase.runPipeline(PigGenericMapBase.java:282)
> > > >>     at
> > > >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> > > >> PigGenericMapBase.map(PigGenericMapBase.java:277)
> > > >>     at
> > > >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> > > >> PigGenericMapBase.map(PigGenericMapBase.java:64)
> > > >>     at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
> > > >>     at
> org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:680)
> > > >>     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:346)
> > > >>     at org.apache.hadoop.mapred.Child$4.run(Child.java:282)
> > > >>     at java.security.AccessController.doPrivileged(Native Method)
> > > >>     at javax.security.auth.Subject.doAs(Subject.java:415)
> > > >>     at
> > > >> org.apache.hadoop.security.UserGroupInformation.doAs(
> > > >> UserGroupInformation.java:1117)
> > > >>     at org.apache.hadoop.mapred.Child.main(Child.java:271)
> > > >>
> > > >>
> > > >>
> > > >>
> > > >> org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> > > >> PigGenericMapBase.java:
> > > >>
> > > >> protected void *runPipeline*(PhysicalOperator leaf) throws
> > IOException,
> > > >> InterruptedException {
> > > >>         while(true){
> > > >>             Result res = leaf.getNext(DUMMYTUPLE);
> > > >>             if(res.returnStatus==POStatus.STATUS_OK){
> > > >>                 collect(outputCollector,(Tuple)res.result);
> > > >>                 continue;
> > > >>             }
> > > >> ....
> > > >>
> > > >>
> > > >>
> > > >> Whats the suggested code fix here?
> > > >>
> > > >>
> > > >> Thanks,
> > > >> Suhas.
> > > >>
> > > >
> > > >
> > >
> >
>

Re: pig12 job stuck in infinite loop

Posted by Suhas Satish <su...@gmail.com>.
Hi Cheolsoo,
This is where its hanging -
*pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*

org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
PigGenericMapBase.java:

protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
InterruptedException {
        while(true){
            Result res = leaf.getNext(DUMMYTUPLE);
            if(res.returnStatus==POStatus.STATUS_OK){
                collect(outputCollector,(Tuple)res.result);
                continue;
            }
....

Cheers,
Suhas.


On Thu, Mar 6, 2014 at 5:56 PM, Cheolsoo Park <pi...@gmail.com> wrote:

> Hi Suhas,
>
> No. The issue with PIG-3461 is that Pig hangs at the query compilation with
> a big filter expression before the job is submitted.
> In addition, the filter extractor was totally rewritten in 0.12.
> https://issues.apache.org/jira/browse/PIG-3461
>
> Where exactly is your job hanging? Backend or frontend? Are you running it
> in local mode or remote mode?
>
> Thanks,
> Cheolsoo
>
> p.s.
> There are two known issues with the new filter extractor in 0.12.0 although
> these are probably not related to your issue-
> https://issues.apache.org/jira/browse/PIG-3510
> https://issues.apache.org/jira/browse/PIG-3657
>
>
> On Thu, Mar 6, 2014 at 5:30 PM, Suhas Satish <su...@gmail.com>
> wrote:
>
> > I seem to be hitting this issue in pig-0.12 although it claims to be
> fixed
> > in pig-0.12
> > https://issues.apache.org/jira/browse/PIG-3395
> > Large filter expression makes Pig hang
> >
> > Cheers,
> > Suhas.
> >
> >
> > On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <su...@gmail.com>
> > wrote:
> >
> > > This is the pig script -
> > >
> > > %default previousPeriod $pPeriod
> > >
> > > tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS (WEEK:int,
> > > DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
> > >
> > > gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
> > >
> > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > >
> > > pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
> > >
> > > gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> > > store gpWeekRanked INTO 'gpWeekRanked';
> > > describe gpWeekRanked;
> > >
> > >
> > > Without the filter statement, the code runs without hanging.
> > >
> > > Cheers,
> > > Suhas.
> > >
> > >
> > > On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <suhas.satish@gmail.com
> > >wrote:
> > >
> > >> Hi
> > >> I launched the attached pig job on pig-12 with hadoop MRv1 with the
> > >> attached data, but the FILTER function causes the job to get stuck in
> an
> > >> infinite loop.
> > >>
> > >> pig -p pPeriod=201312 -f test.pig
> > >>
> > >> The thread in question seems to be stuck forever inside while loop of
> > >> runPipeline method.
> > >>
> > >> stack trace:
> > >> -----------
> > >>
> > >> "main" prio=10 tid=0x00007fd74800b000 nid=0x2f63 runnable
> > >> [0x00007fd750d50000]
> > >>    java.lang.Thread.State: RUNNABLE
> > >>     at
> > >> org.apache.pig.backend.hadoop.executionengine.physicalLayer.
> > >> relationalOperators.POForEach.getNextTuple(POForEach.java:217)
> > >>     at
> > >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> > >> PigGenericMapBase.runPipeline(PigGenericMapBase.java:282)
> > >>     at
> > >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> > >> PigGenericMapBase.map(PigGenericMapBase.java:277)
> > >>     at
> > >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> > >> PigGenericMapBase.map(PigGenericMapBase.java:64)
> > >>     at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
> > >>     at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:680)
> > >>     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:346)
> > >>     at org.apache.hadoop.mapred.Child$4.run(Child.java:282)
> > >>     at java.security.AccessController.doPrivileged(Native Method)
> > >>     at javax.security.auth.Subject.doAs(Subject.java:415)
> > >>     at
> > >> org.apache.hadoop.security.UserGroupInformation.doAs(
> > >> UserGroupInformation.java:1117)
> > >>     at org.apache.hadoop.mapred.Child.main(Child.java:271)
> > >>
> > >>
> > >>
> > >>
> > >> org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> > >> PigGenericMapBase.java:
> > >>
> > >> protected void *runPipeline*(PhysicalOperator leaf) throws
> IOException,
> > >> InterruptedException {
> > >>         while(true){
> > >>             Result res = leaf.getNext(DUMMYTUPLE);
> > >>             if(res.returnStatus==POStatus.STATUS_OK){
> > >>                 collect(outputCollector,(Tuple)res.result);
> > >>                 continue;
> > >>             }
> > >> ....
> > >>
> > >>
> > >>
> > >> Whats the suggested code fix here?
> > >>
> > >>
> > >> Thanks,
> > >> Suhas.
> > >>
> > >
> > >
> >
>

Re: pig12 job stuck in infinite loop

Posted by Suhas Satish <su...@gmail.com>.
Hi Cheolsoo,
This is where its hanging -
*pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*

org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
PigGenericMapBase.java:

protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
InterruptedException {
        while(true){
            Result res = leaf.getNext(DUMMYTUPLE);
            if(res.returnStatus==POStatus.STATUS_OK){
                collect(outputCollector,(Tuple)res.result);
                continue;
            }
....

Cheers,
Suhas.


On Thu, Mar 6, 2014 at 5:56 PM, Cheolsoo Park <pi...@gmail.com> wrote:

> Hi Suhas,
>
> No. The issue with PIG-3461 is that Pig hangs at the query compilation with
> a big filter expression before the job is submitted.
> In addition, the filter extractor was totally rewritten in 0.12.
> https://issues.apache.org/jira/browse/PIG-3461
>
> Where exactly is your job hanging? Backend or frontend? Are you running it
> in local mode or remote mode?
>
> Thanks,
> Cheolsoo
>
> p.s.
> There are two known issues with the new filter extractor in 0.12.0 although
> these are probably not related to your issue-
> https://issues.apache.org/jira/browse/PIG-3510
> https://issues.apache.org/jira/browse/PIG-3657
>
>
> On Thu, Mar 6, 2014 at 5:30 PM, Suhas Satish <su...@gmail.com>
> wrote:
>
> > I seem to be hitting this issue in pig-0.12 although it claims to be
> fixed
> > in pig-0.12
> > https://issues.apache.org/jira/browse/PIG-3395
> > Large filter expression makes Pig hang
> >
> > Cheers,
> > Suhas.
> >
> >
> > On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <su...@gmail.com>
> > wrote:
> >
> > > This is the pig script -
> > >
> > > %default previousPeriod $pPeriod
> > >
> > > tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS (WEEK:int,
> > > DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
> > >
> > > gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
> > >
> > > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> > >
> > > pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
> > >
> > > gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> > > store gpWeekRanked INTO 'gpWeekRanked';
> > > describe gpWeekRanked;
> > >
> > >
> > > Without the filter statement, the code runs without hanging.
> > >
> > > Cheers,
> > > Suhas.
> > >
> > >
> > > On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <suhas.satish@gmail.com
> > >wrote:
> > >
> > >> Hi
> > >> I launched the attached pig job on pig-12 with hadoop MRv1 with the
> > >> attached data, but the FILTER function causes the job to get stuck in
> an
> > >> infinite loop.
> > >>
> > >> pig -p pPeriod=201312 -f test.pig
> > >>
> > >> The thread in question seems to be stuck forever inside while loop of
> > >> runPipeline method.
> > >>
> > >> stack trace:
> > >> -----------
> > >>
> > >> "main" prio=10 tid=0x00007fd74800b000 nid=0x2f63 runnable
> > >> [0x00007fd750d50000]
> > >>    java.lang.Thread.State: RUNNABLE
> > >>     at
> > >> org.apache.pig.backend.hadoop.executionengine.physicalLayer.
> > >> relationalOperators.POForEach.getNextTuple(POForEach.java:217)
> > >>     at
> > >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> > >> PigGenericMapBase.runPipeline(PigGenericMapBase.java:282)
> > >>     at
> > >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> > >> PigGenericMapBase.map(PigGenericMapBase.java:277)
> > >>     at
> > >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> > >> PigGenericMapBase.map(PigGenericMapBase.java:64)
> > >>     at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
> > >>     at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:680)
> > >>     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:346)
> > >>     at org.apache.hadoop.mapred.Child$4.run(Child.java:282)
> > >>     at java.security.AccessController.doPrivileged(Native Method)
> > >>     at javax.security.auth.Subject.doAs(Subject.java:415)
> > >>     at
> > >> org.apache.hadoop.security.UserGroupInformation.doAs(
> > >> UserGroupInformation.java:1117)
> > >>     at org.apache.hadoop.mapred.Child.main(Child.java:271)
> > >>
> > >>
> > >>
> > >>
> > >> org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> > >> PigGenericMapBase.java:
> > >>
> > >> protected void *runPipeline*(PhysicalOperator leaf) throws
> IOException,
> > >> InterruptedException {
> > >>         while(true){
> > >>             Result res = leaf.getNext(DUMMYTUPLE);
> > >>             if(res.returnStatus==POStatus.STATUS_OK){
> > >>                 collect(outputCollector,(Tuple)res.result);
> > >>                 continue;
> > >>             }
> > >> ....
> > >>
> > >>
> > >>
> > >> Whats the suggested code fix here?
> > >>
> > >>
> > >> Thanks,
> > >> Suhas.
> > >>
> > >
> > >
> >
>

Re: pig12 job stuck in infinite loop

Posted by Cheolsoo Park <pi...@gmail.com>.
Hi Suhas,

No. The issue with PIG-3461 is that Pig hangs at the query compilation with
a big filter expression before the job is submitted.
In addition, the filter extractor was totally rewritten in 0.12.
https://issues.apache.org/jira/browse/PIG-3461

Where exactly is your job hanging? Backend or frontend? Are you running it
in local mode or remote mode?

Thanks,
Cheolsoo

p.s.
There are two known issues with the new filter extractor in 0.12.0 although
these are probably not related to your issue-
https://issues.apache.org/jira/browse/PIG-3510
https://issues.apache.org/jira/browse/PIG-3657


On Thu, Mar 6, 2014 at 5:30 PM, Suhas Satish <su...@gmail.com> wrote:

> I seem to be hitting this issue in pig-0.12 although it claims to be fixed
> in pig-0.12
> https://issues.apache.org/jira/browse/PIG-3395
> Large filter expression makes Pig hang
>
> Cheers,
> Suhas.
>
>
> On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <su...@gmail.com>
> wrote:
>
> > This is the pig script -
> >
> > %default previousPeriod $pPeriod
> >
> > tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS (WEEK:int,
> > DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
> >
> > gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
> >
> > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> >
> > pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
> >
> > gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> > store gpWeekRanked INTO 'gpWeekRanked';
> > describe gpWeekRanked;
> >
> >
> > Without the filter statement, the code runs without hanging.
> >
> > Cheers,
> > Suhas.
> >
> >
> > On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <suhas.satish@gmail.com
> >wrote:
> >
> >> Hi
> >> I launched the attached pig job on pig-12 with hadoop MRv1 with the
> >> attached data, but the FILTER function causes the job to get stuck in an
> >> infinite loop.
> >>
> >> pig -p pPeriod=201312 -f test.pig
> >>
> >> The thread in question seems to be stuck forever inside while loop of
> >> runPipeline method.
> >>
> >> stack trace:
> >> -----------
> >>
> >> "main" prio=10 tid=0x00007fd74800b000 nid=0x2f63 runnable
> >> [0x00007fd750d50000]
> >>    java.lang.Thread.State: RUNNABLE
> >>     at
> >> org.apache.pig.backend.hadoop.executionengine.physicalLayer.
> >> relationalOperators.POForEach.getNextTuple(POForEach.java:217)
> >>     at
> >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> >> PigGenericMapBase.runPipeline(PigGenericMapBase.java:282)
> >>     at
> >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> >> PigGenericMapBase.map(PigGenericMapBase.java:277)
> >>     at
> >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> >> PigGenericMapBase.map(PigGenericMapBase.java:64)
> >>     at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
> >>     at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:680)
> >>     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:346)
> >>     at org.apache.hadoop.mapred.Child$4.run(Child.java:282)
> >>     at java.security.AccessController.doPrivileged(Native Method)
> >>     at javax.security.auth.Subject.doAs(Subject.java:415)
> >>     at
> >> org.apache.hadoop.security.UserGroupInformation.doAs(
> >> UserGroupInformation.java:1117)
> >>     at org.apache.hadoop.mapred.Child.main(Child.java:271)
> >>
> >>
> >>
> >>
> >> org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> >> PigGenericMapBase.java:
> >>
> >> protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
> >> InterruptedException {
> >>         while(true){
> >>             Result res = leaf.getNext(DUMMYTUPLE);
> >>             if(res.returnStatus==POStatus.STATUS_OK){
> >>                 collect(outputCollector,(Tuple)res.result);
> >>                 continue;
> >>             }
> >> ....
> >>
> >>
> >>
> >> Whats the suggested code fix here?
> >>
> >>
> >> Thanks,
> >> Suhas.
> >>
> >
> >
>

Re: pig12 job stuck in infinite loop

Posted by Cheolsoo Park <pi...@gmail.com>.
Hi Suhas,

No. The issue with PIG-3461 is that Pig hangs at the query compilation with
a big filter expression before the job is submitted.
In addition, the filter extractor was totally rewritten in 0.12.
https://issues.apache.org/jira/browse/PIG-3461

Where exactly is your job hanging? Backend or frontend? Are you running it
in local mode or remote mode?

Thanks,
Cheolsoo

p.s.
There are two known issues with the new filter extractor in 0.12.0 although
these are probably not related to your issue-
https://issues.apache.org/jira/browse/PIG-3510
https://issues.apache.org/jira/browse/PIG-3657


On Thu, Mar 6, 2014 at 5:30 PM, Suhas Satish <su...@gmail.com> wrote:

> I seem to be hitting this issue in pig-0.12 although it claims to be fixed
> in pig-0.12
> https://issues.apache.org/jira/browse/PIG-3395
> Large filter expression makes Pig hang
>
> Cheers,
> Suhas.
>
>
> On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <su...@gmail.com>
> wrote:
>
> > This is the pig script -
> >
> > %default previousPeriod $pPeriod
> >
> > tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS (WEEK:int,
> > DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
> >
> > gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
> >
> > *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
> >
> > pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
> >
> > gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> > store gpWeekRanked INTO 'gpWeekRanked';
> > describe gpWeekRanked;
> >
> >
> > Without the filter statement, the code runs without hanging.
> >
> > Cheers,
> > Suhas.
> >
> >
> > On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <suhas.satish@gmail.com
> >wrote:
> >
> >> Hi
> >> I launched the attached pig job on pig-12 with hadoop MRv1 with the
> >> attached data, but the FILTER function causes the job to get stuck in an
> >> infinite loop.
> >>
> >> pig -p pPeriod=201312 -f test.pig
> >>
> >> The thread in question seems to be stuck forever inside while loop of
> >> runPipeline method.
> >>
> >> stack trace:
> >> -----------
> >>
> >> "main" prio=10 tid=0x00007fd74800b000 nid=0x2f63 runnable
> >> [0x00007fd750d50000]
> >>    java.lang.Thread.State: RUNNABLE
> >>     at
> >> org.apache.pig.backend.hadoop.executionengine.physicalLayer.
> >> relationalOperators.POForEach.getNextTuple(POForEach.java:217)
> >>     at
> >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> >> PigGenericMapBase.runPipeline(PigGenericMapBase.java:282)
> >>     at
> >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> >> PigGenericMapBase.map(PigGenericMapBase.java:277)
> >>     at
> >> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> >> PigGenericMapBase.map(PigGenericMapBase.java:64)
> >>     at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
> >>     at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:680)
> >>     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:346)
> >>     at org.apache.hadoop.mapred.Child$4.run(Child.java:282)
> >>     at java.security.AccessController.doPrivileged(Native Method)
> >>     at javax.security.auth.Subject.doAs(Subject.java:415)
> >>     at
> >> org.apache.hadoop.security.UserGroupInformation.doAs(
> >> UserGroupInformation.java:1117)
> >>     at org.apache.hadoop.mapred.Child.main(Child.java:271)
> >>
> >>
> >>
> >>
> >> org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> >> PigGenericMapBase.java:
> >>
> >> protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
> >> InterruptedException {
> >>         while(true){
> >>             Result res = leaf.getNext(DUMMYTUPLE);
> >>             if(res.returnStatus==POStatus.STATUS_OK){
> >>                 collect(outputCollector,(Tuple)res.result);
> >>                 continue;
> >>             }
> >> ....
> >>
> >>
> >>
> >> Whats the suggested code fix here?
> >>
> >>
> >> Thanks,
> >> Suhas.
> >>
> >
> >
>

Re: pig12 job stuck in infinite loop

Posted by Suhas Satish <su...@gmail.com>.
I seem to be hitting this issue in pig-0.12 although it claims to be fixed
in pig-0.12
https://issues.apache.org/jira/browse/PIG-3395
Large filter expression makes Pig hang

Cheers,
Suhas.


On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <su...@gmail.com> wrote:

> This is the pig script -
>
> %default previousPeriod $pPeriod
>
> tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS (WEEK:int,
> DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
>
> gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
>
> *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
>
> pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
>
> gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> store gpWeekRanked INTO 'gpWeekRanked';
> describe gpWeekRanked;
>
>
> Without the filter statement, the code runs without hanging.
>
> Cheers,
> Suhas.
>
>
> On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <su...@gmail.com>wrote:
>
>> Hi
>> I launched the attached pig job on pig-12 with hadoop MRv1 with the
>> attached data, but the FILTER function causes the job to get stuck in an
>> infinite loop.
>>
>> pig -p pPeriod=201312 -f test.pig
>>
>> The thread in question seems to be stuck forever inside while loop of
>> runPipeline method.
>>
>> stack trace:
>> -----------
>>
>> "main" prio=10 tid=0x00007fd74800b000 nid=0x2f63 runnable
>> [0x00007fd750d50000]
>>    java.lang.Thread.State: RUNNABLE
>>     at
>> org.apache.pig.backend.hadoop.executionengine.physicalLayer.
>> relationalOperators.POForEach.getNextTuple(POForEach.java:217)
>>     at
>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
>> PigGenericMapBase.runPipeline(PigGenericMapBase.java:282)
>>     at
>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
>> PigGenericMapBase.map(PigGenericMapBase.java:277)
>>     at
>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
>> PigGenericMapBase.map(PigGenericMapBase.java:64)
>>     at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
>>     at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:680)
>>     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:346)
>>     at org.apache.hadoop.mapred.Child$4.run(Child.java:282)
>>     at java.security.AccessController.doPrivileged(Native Method)
>>     at javax.security.auth.Subject.doAs(Subject.java:415)
>>     at
>> org.apache.hadoop.security.UserGroupInformation.doAs(
>> UserGroupInformation.java:1117)
>>     at org.apache.hadoop.mapred.Child.main(Child.java:271)
>>
>>
>>
>>
>> org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
>> PigGenericMapBase.java:
>>
>> protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
>> InterruptedException {
>>         while(true){
>>             Result res = leaf.getNext(DUMMYTUPLE);
>>             if(res.returnStatus==POStatus.STATUS_OK){
>>                 collect(outputCollector,(Tuple)res.result);
>>                 continue;
>>             }
>> ....
>>
>>
>>
>> Whats the suggested code fix here?
>>
>>
>> Thanks,
>> Suhas.
>>
>
>

Re: pig12 job stuck in infinite loop

Posted by Suhas Satish <su...@gmail.com>.
I seem to be hitting this issue in pig-0.12 although it claims to be fixed
in pig-0.12
https://issues.apache.org/jira/browse/PIG-3395
Large filter expression makes Pig hang

Cheers,
Suhas.


On Thu, Mar 6, 2014 at 4:26 PM, Suhas Satish <su...@gmail.com> wrote:

> This is the pig script -
>
> %default previousPeriod $pPeriod
>
> tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS (WEEK:int,
> DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);
>
> gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;
>
> *pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*
>
> pWeekRanked = RANK pWeek BY WEEK ASC DENSE;
>
> gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
> store gpWeekRanked INTO 'gpWeekRanked';
> describe gpWeekRanked;
>
>
> Without the filter statement, the code runs without hanging.
>
> Cheers,
> Suhas.
>
>
> On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <su...@gmail.com>wrote:
>
>> Hi
>> I launched the attached pig job on pig-12 with hadoop MRv1 with the
>> attached data, but the FILTER function causes the job to get stuck in an
>> infinite loop.
>>
>> pig -p pPeriod=201312 -f test.pig
>>
>> The thread in question seems to be stuck forever inside while loop of
>> runPipeline method.
>>
>> stack trace:
>> -----------
>>
>> "main" prio=10 tid=0x00007fd74800b000 nid=0x2f63 runnable
>> [0x00007fd750d50000]
>>    java.lang.Thread.State: RUNNABLE
>>     at
>> org.apache.pig.backend.hadoop.executionengine.physicalLayer.
>> relationalOperators.POForEach.getNextTuple(POForEach.java:217)
>>     at
>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
>> PigGenericMapBase.runPipeline(PigGenericMapBase.java:282)
>>     at
>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
>> PigGenericMapBase.map(PigGenericMapBase.java:277)
>>     at
>> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
>> PigGenericMapBase.map(PigGenericMapBase.java:64)
>>     at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
>>     at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:680)
>>     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:346)
>>     at org.apache.hadoop.mapred.Child$4.run(Child.java:282)
>>     at java.security.AccessController.doPrivileged(Native Method)
>>     at javax.security.auth.Subject.doAs(Subject.java:415)
>>     at
>> org.apache.hadoop.security.UserGroupInformation.doAs(
>> UserGroupInformation.java:1117)
>>     at org.apache.hadoop.mapred.Child.main(Child.java:271)
>>
>>
>>
>>
>> org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
>> PigGenericMapBase.java:
>>
>> protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
>> InterruptedException {
>>         while(true){
>>             Result res = leaf.getNext(DUMMYTUPLE);
>>             if(res.returnStatus==POStatus.STATUS_OK){
>>                 collect(outputCollector,(Tuple)res.result);
>>                 continue;
>>             }
>> ....
>>
>>
>>
>> Whats the suggested code fix here?
>>
>>
>> Thanks,
>> Suhas.
>>
>
>

Re: pig12 job stuck in infinite loop

Posted by Suhas Satish <su...@gmail.com>.
This is the pig script -

%default previousPeriod $pPeriod

tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS (WEEK:int,
DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);

gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;

*pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*

pWeekRanked = RANK pWeek BY WEEK ASC DENSE;

gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
store gpWeekRanked INTO 'gpWeekRanked';
describe gpWeekRanked;


Without the filter statement, the code runs without hanging.

Cheers,
Suhas.


On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <su...@gmail.com> wrote:

> Hi
> I launched the attached pig job on pig-12 with hadoop MRv1 with the
> attached data, but the FILTER function causes the job to get stuck in an
> infinite loop.
>
> pig -p pPeriod=201312 -f test.pig
>
> The thread in question seems to be stuck forever inside while loop of
> runPipeline method.
>
> stack trace:
> -----------
>
> "main" prio=10 tid=0x00007fd74800b000 nid=0x2f63 runnable
> [0x00007fd750d50000]
>    java.lang.Thread.State: RUNNABLE
>     at
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.
> relationalOperators.POForEach.getNextTuple(POForEach.java:217)
>     at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> PigGenericMapBase.runPipeline(PigGenericMapBase.java:282)
>     at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> PigGenericMapBase.map(PigGenericMapBase.java:277)
>     at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> PigGenericMapBase.map(PigGenericMapBase.java:64)
>     at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
>     at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:680)
>     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:346)
>     at org.apache.hadoop.mapred.Child$4.run(Child.java:282)
>     at java.security.AccessController.doPrivileged(Native Method)
>     at javax.security.auth.Subject.doAs(Subject.java:415)
>     at
> org.apache.hadoop.security.UserGroupInformation.doAs(
> UserGroupInformation.java:1117)
>     at org.apache.hadoop.mapred.Child.main(Child.java:271)
>
>
>
>
> org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> PigGenericMapBase.java:
>
> protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
> InterruptedException {
>         while(true){
>             Result res = leaf.getNext(DUMMYTUPLE);
>             if(res.returnStatus==POStatus.STATUS_OK){
>                 collect(outputCollector,(Tuple)res.result);
>                 continue;
>             }
> ....
>
>
>
> Whats the suggested code fix here?
>
>
> Thanks,
> Suhas.
>

Re: pig12 job stuck in infinite loop

Posted by Suhas Satish <su...@gmail.com>.
This is the pig script -

%default previousPeriod $pPeriod

tWeek = LOAD '/tmp/test_data.txt' USING PigStorage ('|') AS (WEEK:int,
DESCRIPTION:chararray, END_DATE:chararray, PERIOD:int);

gTWeek = FOREACH tWeek GENERATE WEEK AS WEEK, PERIOD AS PERIOD;

*pWeek = FILTER gTWeek BY PERIOD == $previousPeriod;*

pWeekRanked = RANK pWeek BY WEEK ASC DENSE;

gpWeekRanked = FOREACH pWeekRanked GENERATE $0;
store gpWeekRanked INTO 'gpWeekRanked';
describe gpWeekRanked;


Without the filter statement, the code runs without hanging.

Cheers,
Suhas.


On Thu, Mar 6, 2014 at 3:05 PM, Suhas Satish <su...@gmail.com> wrote:

> Hi
> I launched the attached pig job on pig-12 with hadoop MRv1 with the
> attached data, but the FILTER function causes the job to get stuck in an
> infinite loop.
>
> pig -p pPeriod=201312 -f test.pig
>
> The thread in question seems to be stuck forever inside while loop of
> runPipeline method.
>
> stack trace:
> -----------
>
> "main" prio=10 tid=0x00007fd74800b000 nid=0x2f63 runnable
> [0x00007fd750d50000]
>    java.lang.Thread.State: RUNNABLE
>     at
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.
> relationalOperators.POForEach.getNextTuple(POForEach.java:217)
>     at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> PigGenericMapBase.runPipeline(PigGenericMapBase.java:282)
>     at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> PigGenericMapBase.map(PigGenericMapBase.java:277)
>     at
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.
> PigGenericMapBase.map(PigGenericMapBase.java:64)
>     at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:144)
>     at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:680)
>     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:346)
>     at org.apache.hadoop.mapred.Child$4.run(Child.java:282)
>     at java.security.AccessController.doPrivileged(Native Method)
>     at javax.security.auth.Subject.doAs(Subject.java:415)
>     at
> org.apache.hadoop.security.UserGroupInformation.doAs(
> UserGroupInformation.java:1117)
>     at org.apache.hadoop.mapred.Child.main(Child.java:271)
>
>
>
>
> org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/
> PigGenericMapBase.java:
>
> protected void *runPipeline*(PhysicalOperator leaf) throws IOException,
> InterruptedException {
>         while(true){
>             Result res = leaf.getNext(DUMMYTUPLE);
>             if(res.returnStatus==POStatus.STATUS_OK){
>                 collect(outputCollector,(Tuple)res.result);
>                 continue;
>             }
> ....
>
>
>
> Whats the suggested code fix here?
>
>
> Thanks,
> Suhas.
>