[ 
https://issues.apache.org/jira/browse/PIG-1145?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12789188#action_12789188
 ] 

Jing Huang commented on PIG-1145:
---------------------------------

found another failure on merge join
This merge join script failed:
register $zebraJar;
--fs -rmr $outputDir


--a1 = LOAD '$inputDir/unsorted1' USING 
org.apache.hadoop.zebra.pig.TableLoader('count,seed,int1,str2,byte2');
--a2 = LOAD '$inputDir/unsorted2' USING 
org.apache.hadoop.zebra.pig.TableLoader('count,seed,int1,str2,byte2');

--sort1 = order a1 by byte2;
--sort2 = order a2 by byte2;

--store sort1 into '$outputDir/100Msortedbyte21' using 
org.apache.hadoop.zebra.pig.TableStorer('[count,seed,int1,str2];[byte2]');
--store sort2 into '$outputDir/100Msortedbyte22' using 
org.apache.hadoop.zebra.pig.TableStorer('[count,seed,int1,str2];[byte2]');

rec1 = load '$outputDir/100Msortedbyte21' using 
org.apache.hadoop.zebra.pig.TableLoader('','sorted');
rec2 = load '$outputDir/100Msortedbyte22' using 
org.apache.hadoop.zebra.pig.TableLoader('','sorted');

joina = join rec1 by byte2, rec2 by byte2 using "merge" ;

E = foreach joina  generate $0 as count,  $1 as seed,  $2 as int1,  $3 as str2, 
$4 as byte2;

store E into '$outputDir/bad1' using 
org.apache.hadoop.zebra.pig.TableStorer('');
=========
instead, this similiar script works with the previous patch:
register $zebraJar;
--fs -rmr $outputDir


a1 = LOAD '$inputDir/unsorted1' USING 
org.apache.hadoop.zebra.pig.TableLoader('count,seed,int1,str2,byte2');
a2 = LOAD '$inputDir/unsorted2' USING 
org.apache.hadoop.zebra.pig.TableLoader('count,seed,int1,str2,byte2');

sort1 = order a1 by byte2;
sort2 = order a2 by byte2;

store sort1 into '$outputDir/100Msortedbyte21' using 
org.apache.hadoop.zebra.pig.TableStorer('[count,seed,int1,str2,byte2]');
store sort2 into '$outputDir/100Msortedbyte22' using 
org.apache.hadoop.zebra.pig.TableStorer('[count,seed,int1,str2,byte2]');

rec1 = load '$outputDir/100Msortedbyte21' using 
org.apache.hadoop.zebra.pig.TableLoader('','sorted');
rec2 = load '$outputDir/100Msortedbyte22' using 
org.apache.hadoop.zebra.pig.TableLoader('','sorted');

joina = join rec1 by byte2, rec2 by byte2 using "merge" ;

E = foreach joina  generate $0 as count,  $1 as seed,  $2 as int1,  $3 as str2, 
$4 as byte2;

store E into '$outputDir/join3' using 
org.apache.hadoop.zebra.pig.TableStorer('');
~         
================
Here is stack trace:
Backend error message
---------------------
org.apache.pig.backend.executionengine.ExecException: ERROR 2176: Error 
processing right input during merge join
        at 
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin.throwProcessingException(POMergeJoin.java:453)
        at 
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin.getNextRightInp(POMergeJoin.java:443)
        at 
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin.getNext(POMergeJoin.java:337)
        at 
org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:260)
        at 
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.getNext(POForEach.java:237)
        at 
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.runPipeline(PigMapBase.java:253)
        at 
org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.close(PigMapBase.java:107)
        at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:57)
        at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:358)
        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:307)
        at org.apache.hadoop.mapred.Child.main(Child.java:159)
Caused by: java.io.EOFException: No key-value to read
        at 
org.apache.hadoop.zebra.tfile.TFile$Reader$Scanner.checkKey(TFile.java:1590)
        at 
org.apache.hadoop.zebra.tfile.TFile$Reader$Scanner.entry(TFile.java:1611)
        at 
org.apache.hadoop.zebra.io.ColumnGroup$Reader$TFileScanner.getKey(ColumnGroup.java:854)
        at 
org.apache.hadoop.zebra.io.ColumnGroup$Reader$CGScanner.getCGKey(ColumnGroup.java:1035)
        at 
org.apache.hadoop.zebra.io.BasicTable$Reader$BTScanner.getKey(BasicTable.java:1083)
        at 
org.apache.hadoop.zebra.mapred.TableRecordReader.next(TableRecordReader.java:105)
        at org.apache.hadoop.zebra.pig.TableLoader.getNext(TableLoader.java:414)
        at 
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin.getNextRightInp(POMergeJoin.java:415)
        ... 9 more
=============
This is how I run it (i disabled pruning to simply the possible problem)
java -cp 
/grid/0/dev/hadoopqa/jing1234/conf:/grid/0/dev/hadoopqa/jars/pig.jar:/grid/0/dev/hadoopqa/jars/tfile.jar:/grid/0/dev/hadoopqa/jars/zebra.jar
 org.apache.pig.Main -m config -M -t PruneColumns bad_join.pig 



> [zebra] merge join on large table ( 100,000.000 rows zebra table) failed
> ------------------------------------------------------------------------
>
>                 Key: PIG-1145
>                 URL: https://issues.apache.org/jira/browse/PIG-1145
>             Project: Pig
>          Issue Type: Bug
>    Affects Versions: 0.6.0, 0.7.0
>            Reporter: Jing Huang
>            Assignee: Yan Zhou
>             Fix For: 0.6.0, 0.7.0
>
>         Attachments: PIG-1145.patch, PIG-1145.patch
>
>
> Pig script :
> register $zebraJar;
> --fs -rmr $outputDir
> a1 = LOAD '$inputDir/unsorted1' USING 
> org.apache.hadoop.zebra.pig.TableLoader('count,seed,int1,str2');
> a2 = LOAD '$inputDir/unsorted2' USING 
> org.apache.hadoop.zebra.pig.TableLoader('count,seed,int1,str2');
> sort1 = order a1 by str2;
> sort2 = order a2 by str2;
> --store sort1 into '$outputDir/sorted11' using 
> org.apache.hadoop.zebra.pig.TableStorer('[count,seed,int1,str2]');
> --store sort2 into '$outputDir/sorted21' using 
> org.apache.hadoop.zebra.pig.TableStorer('[count,seed,int1,str2]');
> rec1 = load '$outputDir/sorted11' using 
> org.apache.hadoop.zebra.pig.TableLoader();
> rec2 = load '$outputDir/sorted21' using 
> org.apache.hadoop.zebra.pig.TableLoader();
> joina = join rec1 by str2, rec2 by str2 using "merge" ;
> --E = foreach joina  generate $0 as count,  $1 as seed,  $2 as int1,  $3 as 
> str2;
> store joina into '$outputDir/join1' using 
> org.apache.hadoop.zebra.pig.TableStorer('');
> ~                                                                             
>                                                                               
>     
> ~                                                                             
>                                                                               
>     
> ~                                                  
> ======
> stacktrace:
> org.apache.pig.backend.executionengine.ExecException: ERROR 2176: Error 
> processing right input during merge join at 
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin.throwProcessingException(POMergeJoin.java:453)
>  at 
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin.getNextRightInp(POMergeJoin.java:443)
>  at 
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin.getNext(POMergeJoin.java:337)
>  at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.runPipeline(PigMapBase.java:253)
>  at 
> org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapBase.close(PigMapBase.java:107)
>  at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:57) at 
> org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:358) at 
> org.apache.hadoop.mapred.MapTask.run(MapTask.java:307) at 
> org.apache.hadoop.mapred.Child.main(Child.java:159) Caused by: 
> java.io.EOFException: No key-value to read at 
> org.apache.hadoop.zebra.tfile.TFile$Reader$Scanner.checkKey(TFile.java:1590) 
> at org.apache.hadoop.zebra.tfile.TFile$Reader$Scanner.entry(TFile.java:1611) 
> at 
> org.apache.hadoop.zebra.io.ColumnGroup$Reader$TFileScanner.getKey(ColumnGroup.java:854)
>  at 
> org.apache.hadoop.zebra.io.ColumnGroup$Reader$CGScanner.getCGKey(ColumnGroup.java:1035)
>  at 
> org.apache.hadoop.zebra.io.BasicTable$Reader$BTScanner.getKey(BasicTable.java:1082)
>  at 
> org.apache.hadoop.zebra.mapred.TableRecordReader.next(TableRecordReader.java:105)
>  at org.apache.hadoop.zebra.pig.TableLoader.getNext(TableLoader.java:414) at 
> org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin.getNextRightInp(POMergeJoin.java:415)
>  ... 7 more 

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to