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 2011/11/03 01:14:02 UTC

svn commit: r1196881 - in /pig/branches/branch-0.10: ./ shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/ shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/ src/org/apache/pig/backend/hadoop/...

Author: daijy
Date: Thu Nov  3 00:14:01 2011
New Revision: 1196881

URL: http://svn.apache.org/viewvc?rev=1196881&view=rev
Log:
PIG-2313: NPE in ILLUSTRATE trying to get StatusReporter in STORE

Added:
    pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/IllustrateDummyReporter.java
Modified:
    pig/branches/branch-0.10/CHANGES.txt
    pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java
    pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java
    pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java
    pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java
    pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigGenericMapBase.java
    pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigGenericMapReduce.java
    pig/branches/branch-0.10/test/org/apache/pig/test/TestExampleGenerator.java

Modified: pig/branches/branch-0.10/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/CHANGES.txt?rev=1196881&r1=1196880&r2=1196881&view=diff
==============================================================================
--- pig/branches/branch-0.10/CHANGES.txt (original)
+++ pig/branches/branch-0.10/CHANGES.txt Thu Nov  3 00:14:01 2011
@@ -146,6 +146,8 @@ PIG-2228: support partial aggregation in
 
 BUG FIXES
 
+PIG-2313: NPE in ILLUSTRATE trying to get StatusReporter in STORE (daijy)
+
 PIG-2335: bin/pig does not work with bash 3.0 (azaroth)
 
 PIG-2275: NullPointerException from ILLUSTRATE (daijy)

Modified: pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java?rev=1196881&r1=1196880&r2=1196881&view=diff
==============================================================================
--- pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java (original)
+++ pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java Thu Nov  3 00:14:01 2011
@@ -60,38 +60,6 @@ abstract public class PigMapBase extends
     public boolean inIllustrator(Context context) {
         return (context instanceof PigMapBase.IllustratorContext);
     }
-
-    /**
-     * Dummy implementation of StatusReporter for illustrate mode
-     *
-     */
-    @SuppressWarnings("deprecation")
-    public static class IllustrateDummyReporter extends StatusReporter{
-
-
-        Counters countGen = new Counters();
-        
-        @Override
-        public Counter getCounter(Enum<?> arg0) {
-            return countGen.findCounter(arg0);
-        }
-
-        @Override
-        public Counter getCounter(String group, String name) {
-            return countGen.findCounter(group, name);
-        }
-
-        @Override
-        public void progress() {
-            //no-op
-        }
-
-        @Override
-        public void setStatus(String arg0) {
-            //no-op
-        }
-        
-    }
     
     public class IllustratorContext extends Context {
         private DataBag input;

Modified: pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java?rev=1196881&r1=1196880&r2=1196881&view=diff
==============================================================================
--- pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java (original)
+++ pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java Thu Nov  3 00:14:01 2011
@@ -70,7 +70,7 @@ public class PigMapReduce extends PigGen
                   POPackage pkg
                   ) throws IOException, InterruptedException {
                 super(job.getJobConf(), new TaskAttemptID(), new FakeRawKeyValueIterator(input.iterator().hasNext()),
-                    null, null, null, null, null, null, PigNullableWritable.class, NullableTuple.class);
+                    null, null, null, null, new IllustrateDummyReporter(), null, PigNullableWritable.class, NullableTuple.class);
                 bos = new ByteArrayOutputStream();
                 dos = new DataOutputStream(bos);
                 org.apache.hadoop.mapreduce.Job nwJob = new org.apache.hadoop.mapreduce.Job(job.getJobConf());

Modified: pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java?rev=1196881&r1=1196880&r2=1196881&view=diff
==============================================================================
--- pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java (original)
+++ pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapBase.java Thu Nov  3 00:14:01 2011
@@ -85,7 +85,7 @@ abstract public class PigMapBase extends
         public IllustratorContext(Configuration conf, DataBag input,
               List<Pair<PigNullableWritable, Writable>> output,
               InputSplit split) throws IOException, InterruptedException {
-            super(conf, new TaskAttemptID(), null, null, null, null, split);
+            super(conf, new TaskAttemptID(), null, null, null, new IllustrateDummyReporter(), split);
             if (output == null)
                 throw new IOException("Null output can not be used");
             this.input = input; this.output = output;

Modified: pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java?rev=1196881&r1=1196880&r2=1196881&view=diff
==============================================================================
--- pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java (original)
+++ pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigMapReduce.java Thu Nov  3 00:14:01 2011
@@ -112,7 +112,7 @@ public class PigMapReduce extends PigGen
                   POPackage pkg
                   ) throws IOException, InterruptedException {
                 super(job.getJobConf(), new TaskAttemptID(), new FakeRawKeyValueIterator(input.iterator().hasNext()),
-                    null, null, null, null, null, null, PigNullableWritable.class, NullableTuple.class);
+                    null, null, null, null, new IllustrateDummyReporter(), null, PigNullableWritable.class, NullableTuple.class);
                 bos = new ByteArrayOutputStream();
                 dos = new DataOutputStream(bos);
                 org.apache.hadoop.mapreduce.Job nwJob = new org.apache.hadoop.mapreduce.Job(job.getJobConf());

Added: pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/IllustrateDummyReporter.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/IllustrateDummyReporter.java?rev=1196881&view=auto
==============================================================================
--- pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/IllustrateDummyReporter.java (added)
+++ pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/IllustrateDummyReporter.java Thu Nov  3 00:14:01 2011
@@ -0,0 +1,59 @@
+/*
+ * 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.hadoop.mapred.Counters;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.StatusReporter;
+
+/**
+ * Dummy implementation of StatusReporter for illustrate mode
+ *
+ */
+@SuppressWarnings("deprecation")
+public class IllustrateDummyReporter extends StatusReporter{
+
+
+    Counters countGen = new Counters();
+    
+    @Override
+    public Counter getCounter(Enum<?> arg0) {
+        return countGen.findCounter(arg0);
+    }
+
+    @Override
+    public Counter getCounter(String group, String name) {
+        return countGen.findCounter(group, name);
+    }
+
+    @Override
+    public void progress() {
+        //no-op
+    }
+
+    @Override
+    public void setStatus(String arg0) {
+        //no-op
+    }
+
+    public float getProgress() {
+        return 0;
+    }
+    
+}
\ No newline at end of file

Modified: pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigGenericMapBase.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigGenericMapBase.java?rev=1196881&r1=1196880&r2=1196881&view=diff
==============================================================================
--- pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigGenericMapBase.java (original)
+++ pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigGenericMapBase.java Thu Nov  3 00:14:01 2011
@@ -124,14 +124,16 @@ public abstract class PigGenericMapBase 
             runPipeline(leaf);
         }
 
-        for (POStore store: stores) {
-            if (!initialized) {
-                MapReducePOStoreImpl impl 
-                    = new MapReducePOStoreImpl(context);
-                store.setStoreImpl(impl);
-                store.setUp();
+        if (!inIllustrator) {
+            for (POStore store: stores) {
+                if (!initialized) {
+                    MapReducePOStoreImpl impl 
+                        = new MapReducePOStoreImpl(context);
+                    store.setStoreImpl(impl);
+                    store.setUp();
+                }
+                store.tearDown();
             }
-            store.tearDown();
         }
         
         //Calling EvalFunc.finish()
@@ -227,12 +229,14 @@ public abstract class PigGenericMapBase 
             pigReporter.setRep(context);
             PhysicalOperator.setReporter(pigReporter);
            
-            for (POStore store: stores) {
-                MapReducePOStoreImpl impl 
-                    = new MapReducePOStoreImpl(context);
-                store.setStoreImpl(impl);
-                if (!pigContext.inIllustrator)
-                    store.setUp();
+            if (!inIllustrator) {
+                for (POStore store: stores) {
+                    MapReducePOStoreImpl impl 
+                        = new MapReducePOStoreImpl(context);
+                    store.setStoreImpl(impl);
+                    if (!pigContext.inIllustrator)
+                        store.setUp();
+                }
             }
             
             boolean aggregateWarning = "true".equalsIgnoreCase(pigContext.getProperties().getProperty("aggregate.warning"));

Modified: pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigGenericMapReduce.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigGenericMapReduce.java?rev=1196881&r1=1196880&r2=1196881&view=diff
==============================================================================
--- pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigGenericMapReduce.java (original)
+++ pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigGenericMapReduce.java Thu Nov  3 00:14:01 2011
@@ -518,14 +518,16 @@ public class PigGenericMapReduce {
                 runPipeline(leaf);
             }
 
-            for (POStore store: stores) {
-                if (!initialized) {
-                    MapReducePOStoreImpl impl 
-                        = new MapReducePOStoreImpl(context);
-                    store.setStoreImpl(impl);
-                    store.setUp();
+            if (!inIllustrator) {
+                for (POStore store: stores) {
+                    if (!initialized) {
+                        MapReducePOStoreImpl impl 
+                            = new MapReducePOStoreImpl(context);
+                        store.setStoreImpl(impl);
+                        store.setUp();
+                    }
+                    store.tearDown();
                 }
-                store.tearDown();
             }
                         
             //Calling EvalFunc.finish()

Modified: pig/branches/branch-0.10/test/org/apache/pig/test/TestExampleGenerator.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/test/org/apache/pig/test/TestExampleGenerator.java?rev=1196881&r1=1196880&r2=1196881&view=diff
==============================================================================
--- pig/branches/branch-0.10/test/org/apache/pig/test/TestExampleGenerator.java (original)
+++ pig/branches/branch-0.10/test/org/apache/pig/test/TestExampleGenerator.java Thu Nov  3 00:14:01 2011
@@ -390,4 +390,22 @@ public class TestExampleGenerator {
         assertTrue(derivedData != null);
     }
 
+    @Test
+    public void testFilterGroupCountStore() throws Exception {
+        File out = File.createTempFile("testFilterGroupCountStoreOutput", "");
+        out.deleteOnExit();
+        out.delete();
+    
+        PigServer pigServer = new PigServer(pigContext);
+        pigServer.setBatchOn();
+        pigServer.registerQuery("A = load " + A.toString() + " as (x, y);");
+        pigServer.registerQuery("B = filter A by x < 5;");
+        pigServer.registerQuery("C = group B by x;");
+        pigServer.registerQuery("D = foreach C generate group as x, COUNT(B) as the_count;");
+        pigServer.registerQuery("store D into '" +  out.getAbsolutePath() + "';");
+        Map<Operator, DataBag> derivedData = pigServer.getExamples(null);
+    
+        assertTrue(derivedData != null);
+    }
+
 }