You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by da...@apache.org on 2012/04/21 02:49:15 UTC

svn commit: r1328550 - in /pig/trunk: CHANGES.txt src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MultiQueryOptimizer.java test/org/apache/pig/test/TestMultiQueryCompiler.java

Author: daijy
Date: Sat Apr 21 00:49:14 2012
New Revision: 1328550

URL: http://svn.apache.org/viewvc?rev=1328550&view=rev
Log:
PIG-2627: Custom partitioner not set when POSplit is involved in Plan

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MultiQueryOptimizer.java
    pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1328550&r1=1328549&r2=1328550&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Sat Apr 21 00:49:14 2012
@@ -106,6 +106,8 @@ OPTIMIZATIONS
 
 BUG FIXES
 
+PIG-2627: Custom partitioner not set when POSplit is involved in Plan (aniket486 via daijy)
+
 PIG-2596: Jython UDF does not handle boolean output (aniket486 via daijy)
 
 PIG-2652: Skew join and order by don't trigger reducer estimation (billgraham via dvryaboy)

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MultiQueryOptimizer.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MultiQueryOptimizer.java?rev=1328550&r1=1328549&r2=1328550&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MultiQueryOptimizer.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MultiQueryOptimizer.java Sat Apr 21 00:49:14 2012
@@ -1051,6 +1051,7 @@ class MultiQueryOptimizer extends MROpPl
         splitter.reducePlan = mapReduce.reducePlan;
         splitter.setReduceDone(true);
         splitter.combinePlan = mapReduce.combinePlan;
+        splitter.customPartitioner = mapReduce.customPartitioner;
                 
         // replace store operator in the splitter with split operator
         if (leaf instanceof POStore) {                            

Modified: pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java?rev=1328550&r1=1328549&r2=1328550&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestMultiQueryCompiler.java Sat Apr 21 00:49:14 2012
@@ -46,6 +46,7 @@ import org.apache.pig.impl.plan.Operator
 import org.apache.pig.impl.plan.OperatorPlan;
 import org.apache.pig.impl.util.LogUtils;
 import org.apache.pig.newplan.logical.relational.LogicalPlan;
+import org.apache.pig.test.utils.SimpleCustomPartitioner;
 import org.apache.pig.tools.grunt.GruntParser;
 import org.apache.pig.tools.pigscript.parser.ParseException;
 import org.junit.After;
@@ -1367,6 +1368,37 @@ public class TestMultiQueryCompiler {
         }
     }
     
+    @Test
+    public void testMultiQueryWithCustomPartitioner() {
+
+        System.out.println("===== multi-query with intermediate stores =====");
+
+        try {            
+            myPig.setBatchOn();
+            
+            myPig.registerQuery("a = load 'passwd' " +
+                                "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
+            myPig.registerQuery("l = FILTER a BY uname == 'foo';");
+            myPig.registerQuery("b = GROUP a BY uname PARTITION BY " + SimpleCustomPartitioner.class.getName() + " PARALLEL 3;");
+            myPig.registerQuery("c = FOREACH b GENERATE FLATTEN(a) PARALLEL 3;");
+            myPig.registerQuery("STORE c INTO 'output1';");
+            myPig.registerQuery("STORE l INTO 'output2';");
+
+            LogicalPlan lp = checkLogicalPlan(1, 2, 6);
+
+            PhysicalPlan pp = checkPhysicalPlan(lp, 1, 2, 12);
+
+            MROperPlan mrp = checkMRPlan(pp, 1, 1, 1); 
+            
+            MapReduceOper mrop = mrp.getRoots().get(0);
+            Assert.assertTrue(mrop.getCustomPartitioner().equals(SimpleCustomPartitioner.class.getName()));
+
+        } catch (Exception e) {
+            e.printStackTrace();
+            Assert.fail();
+        } 
+    }    
+    
     // --------------------------------------------------------------------------
     // Helper methods