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/01/27 08:11:26 UTC

svn commit: r1064011 - in /pig/trunk: ./ src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/ src/org/apache/pig/newplan/logical/relational/ test/org/apache/pig/test/

Author: daijy
Date: Thu Jan 27 07:11:25 2011
New Revision: 1064011

URL: http://svn.apache.org/viewvc?rev=1064011&view=rev
Log:
PIG-1813: Pig 0.8 throws ERROR 1075 while trying to refer a map in the result of eval udf.Works with 0.7

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java
    pig/trunk/src/org/apache/pig/newplan/logical/relational/LOGenerate.java
    pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1064011&r1=1064010&r2=1064011&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Thu Jan 27 07:11:25 2011
@@ -282,6 +282,9 @@ PIG-1309: Map-side Cogroup (ashutoshc)
 
 BUG FIXES
 
+PIG-1813: Pig 0.8 throws ERROR 1075 while trying to refer a map in the result
+of eval udf.Works with 0.7 (daijy)
+
 PIG-1776: changing statement corresponding to alias after explain , then 
   doing dump gives incorrect result (thejas)
 

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java?rev=1064011&r1=1064010&r2=1064011&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java Thu Jan 27 07:11:25 2011
@@ -203,13 +203,6 @@ public class POUserFunc extends Expressi
                     result.result = func.exec((Tuple) result.result);
                     }
                 }
-                if(resultType == DataType.BYTEARRAY) {
-                    // This is needed if some EvalFunc has default datatype as bytearray and returns arbitrary objects
-                    // We see such behavior in case of script EvalFunc, which is used to run udfs in scripting langs
-                    if(result.result != null && DataType.findType(result.result) != DataType.BYTEARRAY) {
-                        result.result = new DataByteArray(result.result.toString().getBytes());
-                    }
-                }
                 return result;
             }
 

Modified: pig/trunk/src/org/apache/pig/newplan/logical/relational/LOGenerate.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/newplan/logical/relational/LOGenerate.java?rev=1064011&r1=1064010&r2=1064011&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/newplan/logical/relational/LOGenerate.java (original)
+++ pig/trunk/src/org/apache/pig/newplan/logical/relational/LOGenerate.java Thu Jan 27 07:11:25 2011
@@ -82,6 +82,7 @@ public class LOGenerate extends LogicalR
             
             LogicalFieldSchema fieldSchema = null;
             
+            // schema of the expression after flatten
             LogicalSchema expSchema = null;
             
             if (exp.getFieldSchema()!=null) {
@@ -101,7 +102,7 @@ public class LOGenerate extends LogicalR
                         expSchema = null;
                     }
                     else {
-                        // if flatten is set, set schema of tuple field to this schema
+                     // if we come here, we get a BAG/Tuple with flatten, extract inner schema of the tuple as expSchema
                         List<LogicalSchema.LogicalFieldSchema> innerFieldSchemas = new ArrayList<LogicalSchema.LogicalFieldSchema>();
                         if (flattenFlags[i]) {
                             if (fieldSchema.type == DataType.BAG) {

Modified: pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java?rev=1064011&r1=1064010&r2=1064011&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java Thu Jan 27 07:11:25 2011
@@ -42,6 +42,7 @@ import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.DataType;
+import org.apache.pig.data.DefaultBagFactory;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.data.TupleFactory;
 import org.apache.pig.impl.io.FileLocalizer;
@@ -1216,4 +1217,61 @@ public class TestEvalPipeline2 extends T
         }
         fail();
     }
+    
+    public static class BagGenerateNoSchema extends EvalFunc<DataBag> {
+        @Override
+        public DataBag exec(Tuple input) throws IOException {
+            DataBag bg = DefaultBagFactory.getInstance().newDefaultBag();
+            bg.add(input);
+            return bg;
+        }
+    }
+    
+    // See PIG-1813
+    @Test
+    public void testUDFNoSchemaPropagate1() throws Exception{
+        String[] input1 = {
+                "[key#1,key2#2]",
+                "[key#2,key2#3]",
+        };
+        
+        Util.createInputFile(cluster, "table_testUDFNoSchemaPropagate1", input1);
+
+        pigServer.registerQuery("a = load 'table_testUDFNoSchemaPropagate1' as (a0:map[]);");
+        pigServer.registerQuery("b = foreach a generate " + BagGenerateNoSchema.class.getName() + "(*) as b0;");
+        pigServer.registerQuery("c = foreach b generate flatten(IdentityColumn(b0));");
+        pigServer.registerQuery("d = foreach c generate $0#'key';");
+        
+        Iterator<Tuple> iter = pigServer.openIterator("d");
+        
+        Tuple t = iter.next();
+        assertTrue(t.toString().contains("(1)"));
+        t = iter.next();
+        assertTrue(t.toString().contains("(2)"));
+        assertFalse(iter.hasNext());
+    }
+    
+    // See PIG-1813
+    @Test
+    public void testUDFNoSchemaPropagate2() throws Exception{
+        String[] input1 = {
+                "[key#1,key2#2]",
+                "[key#2,key2#3]",
+        };
+        
+        Util.createInputFile(cluster, "table_testUDFNoSchemaPropagate2", input1);
+
+        pigServer.registerQuery("a = load 'table_testUDFNoSchemaPropagate2' as (a0:map[]);");
+        pigServer.registerQuery("b = foreach a generate flatten(" + BagGenerateNoSchema.class.getName() + "(*)) as b0;");
+        pigServer.registerQuery("c = foreach b generate IdentityColumn(b0);");
+        pigServer.registerQuery("d = foreach c generate $0#'key';");
+        
+        Iterator<Tuple> iter = pigServer.openIterator("d");
+        
+        Tuple t = iter.next();
+        assertTrue(t.toString().contains("(1)"));
+        t = iter.next();
+        assertTrue(t.toString().contains("(2)"));
+        assertFalse(iter.hasNext());
+    }
 }