You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by sm...@apache.org on 2009/02/07 07:43:43 UTC

svn commit: r741847 - in /hadoop/pig/trunk: ./ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/ src/org/apache/pig/backend/local/executionengine/ test/org/apache/pig/test/

Author: sms
Date: Sat Feb  7 06:43:42 2009
New Revision: 741847

URL: http://svn.apache.org/viewvc?rev=741847&view=rev
Log:
PIG-553: EvalFunc.finish() not getting called (shravanmn via sms)

Added:
    hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/UDFFinishVisitor.java
    hadoop/pig/trunk/test/org/apache/pig/test/TestFinish.java
Modified:
    hadoop/pig/trunk/CHANGES.txt
    hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java
    hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java
    hadoop/pig/trunk/src/org/apache/pig/backend/local/executionengine/LocalPigLauncher.java

Modified: hadoop/pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=741847&r1=741846&r2=741847&view=diff
==============================================================================
--- hadoop/pig/trunk/CHANGES.txt (original)
+++ hadoop/pig/trunk/CHANGES.txt Sat Feb  7 06:43:42 2009
@@ -409,3 +409,5 @@
 
     PIG-637: Limit broken after order by in the local mode (shubhamc via
     olgan)
+
+    PIG-553: EvalFunc.finish() not getting called (shravanmn via sms)

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java?rev=741847&r1=741846&r2=741847&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java Sat Feb  7 06:43:42 2009
@@ -35,7 +35,9 @@
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.io.PigNullableWritable;
 import org.apache.pig.data.TupleFactory;
+import org.apache.pig.impl.plan.DependencyOrderWalker;
 import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
@@ -99,6 +101,15 @@
                  throw ioe;
             }
         }
+        
+        //Calling EvalFunc.finish()
+        UDFFinishVisitor finisher = new UDFFinishVisitor(mp, new DependencyOrderWalker<PhysicalOperator, PhysicalPlan>(mp));
+        try {
+            finisher.visit();
+        } catch (VisitorException e) {
+            throw new IOException("Error trying to finish UDFs",e);
+        }
+        
         mp = null;
         
     }

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java?rev=741847&r1=741846&r2=741847&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java Sat Feb  7 06:43:42 2009
@@ -47,6 +47,8 @@
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.io.PigNullableWritable;
 import org.apache.pig.impl.io.NullableTuple;
+import org.apache.pig.impl.plan.DependencyOrderWalker;
+import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.util.ObjectSerializer;
 import org.apache.pig.impl.util.SpillableMemoryManager;
 import org.apache.pig.impl.util.WrappedIOException;
@@ -347,6 +349,14 @@
                      throw ioe;
                 }
             }
+            
+            //Calling EvalFunc.finish()
+            UDFFinishVisitor finisher = new UDFFinishVisitor(rp, new DependencyOrderWalker<PhysicalOperator, PhysicalPlan>(rp));
+            try {
+                finisher.visit();
+            } catch (VisitorException e) {
+                throw new IOException("Error trying to finish UDFs",e);
+            }
         }
     }
     

Added: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/UDFFinishVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/UDFFinishVisitor.java?rev=741847&view=auto
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/UDFFinishVisitor.java (added)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/UDFFinishVisitor.java Sat Feb  7 06:43:42 2009
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.backend.hadoop.executionengine.mapReduceLayer;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.impl.plan.PlanWalker;
+import org.apache.pig.impl.plan.VisitorException;
+
+public class UDFFinishVisitor extends PhyPlanVisitor {
+
+    public UDFFinishVisitor(PhysicalPlan plan, PlanWalker<PhysicalOperator, PhysicalPlan> walker) {
+        super(plan, walker);
+    }
+
+    @Override
+    public void visitUserFunc(POUserFunc userFunc) throws VisitorException {
+        super.visitUserFunc(userFunc);
+        userFunc.finish();
+    }
+
+}
\ No newline at end of file

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/local/executionengine/LocalPigLauncher.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/local/executionengine/LocalPigLauncher.java?rev=741847&r1=741846&r2=741847&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/local/executionengine/LocalPigLauncher.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/local/executionengine/LocalPigLauncher.java Sat Feb  7 06:43:42 2009
@@ -27,13 +27,18 @@
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobCreationException;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.Launcher;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.UDFFinishVisitor;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.plan.DependencyOrderWalker;
 import org.apache.pig.impl.plan.PlanException;
+import org.apache.pig.impl.plan.PlanWalker;
 import org.apache.pig.impl.plan.VisitorException;
 
 public class LocalPigLauncher extends Launcher {
@@ -63,7 +68,10 @@
             if (res.returnStatus != POStatus.STATUS_EOP)
                 failedJobs++;
         }
-
+        
+        UDFFinishVisitor finisher = new UDFFinishVisitor(php, new DependencyOrderWalker<PhysicalOperator, PhysicalPlan>(php));
+        finisher.visit();
+        
         if (failedJobs == 0) {
             log.info("100% complete!");
             log.info("Success!!");
@@ -75,5 +83,7 @@
         return false;
 
     }
+    
+    
 
 }

Added: hadoop/pig/trunk/test/org/apache/pig/test/TestFinish.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestFinish.java?rev=741847&view=auto
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestFinish.java (added)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestFinish.java Sat Feb  7 06:43:42 2009
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Iterator;
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.ExecType;
+import org.apache.pig.PigServer;
+import org.apache.pig.builtin.PigStorage;
+import org.apache.pig.data.BagFactory;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.pig.impl.io.FileLocalizer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFinish extends TestCase {
+
+    private PigServer pigServer;
+
+    TupleFactory mTf = TupleFactory.getInstance();
+    BagFactory mBf = BagFactory.getInstance();
+    File f1;
+    
+    public static int gCount = 0;
+    
+    static public class MyEvalFunction extends EvalFunc<Tuple>{
+        int count = 0;
+        
+        @Override
+        public Tuple exec(Tuple input) throws IOException {
+            ++count;
+            return input;
+        }
+
+        @Override
+        public void finish() {
+            gCount = count;
+        }
+    }
+    
+    @Before
+    @Override
+    public void setUp() throws Exception{
+        pigServer = new PigServer(ExecType.MAPREDUCE);
+        f1 = File.createTempFile("test", "txt");
+        PrintStream ps = new PrintStream(new FileOutputStream(f1));
+        for(int i = 0; i < 3; i++) {
+            ps.println('a'+i + ":1");
+        }
+        ps.close();
+    }
+    
+    @Test
+    public void testFinishInMapMR() throws Exception{
+        gCount = 0;
+        pigServer.registerQuery("a = load '" + Util.generateURI(f1.toString()) + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("b = foreach a generate " + MyEvalFunction.class.getName() + "(*);");
+        Iterator<Tuple> iter = pigServer.openIterator("b");
+        int count = 0;
+        while(iter.hasNext()){
+            ++count;
+            iter.next();
+        }
+        
+        System.out.println(count + ", " + gCount);
+        assertEquals(true, gCount==3);
+    }
+    
+    @Test
+    public void testFinishInReduceMR() throws Exception{
+        gCount = 0;
+        pigServer.registerQuery("a = load '" + Util.generateURI(f1.toString()) + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("a1 = group a by $1;");
+        pigServer.registerQuery("b = foreach a1 generate " + MyEvalFunction.class.getName() + "(*);");
+        Iterator<Tuple> iter = pigServer.openIterator("b");
+        int count = 0;
+        while(iter.hasNext()){
+            ++count;
+            iter.next();
+        }
+        
+        System.out.println(count + ", " + gCount);
+        assertEquals(true, gCount==1);
+    }
+    
+    @Test
+    public void testFinishInMapLoc() throws Exception{
+        pigServer = new PigServer(ExecType.LOCAL);
+        gCount = 0;
+        pigServer.registerQuery("a = load '" + Util.generateURI(f1.toString()) + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("b = foreach a generate " + MyEvalFunction.class.getName() + "(*);");
+        pigServer.openIterator("b");
+        assertEquals(true, gCount==3);
+    }
+    
+    @Test
+    public void testFinishInReduceLoc() throws Exception{
+        pigServer = new PigServer(ExecType.LOCAL);
+        gCount = 0;
+        pigServer.registerQuery("a = load '" + Util.generateURI(f1.toString()) + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("a1 = group a by $1;");
+        pigServer.registerQuery("b = foreach a1 generate " + MyEvalFunction.class.getName() + "(*);");
+        pigServer.openIterator("b");
+        assertEquals(true, gCount==1);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+    }
+
+}