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/26 06:16:01 UTC

svn commit: r748020 - in /hadoop/pig/trunk: ./ src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/ src/org/apache/pig/builtin/ test/org/apache/pig/test/

Author: sms
Date: Thu Feb 26 05:16:00 2009
New Revision: 748020

URL: http://svn.apache.org/viewvc?rev=748020&view=rev
Log:
PIG-664: Semantics of * is not consistent

Added:
    hadoop/pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java
Modified:
    hadoop/pig/trunk/CHANGES.txt
    hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java
    hadoop/pig/trunk/src/org/apache/pig/builtin/ARITY.java
    hadoop/pig/trunk/src/org/apache/pig/builtin/TupleSize.java
    hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java
    hadoop/pig/trunk/test/org/apache/pig/test/TestFRJoin.java

Modified: hadoop/pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=748020&r1=748019&r2=748020&view=diff
==============================================================================
--- hadoop/pig/trunk/CHANGES.txt (original)
+++ hadoop/pig/trunk/CHANGES.txt Thu Feb 26 05:16:00 2009
@@ -432,3 +432,5 @@
     via sms)
 
     PIG-591: Error handling phase four (sms via pradeepkth)
+
+    PIG-664: Semantics of * is not consistent (sms)

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java?rev=748020&r1=748019&r2=748020&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java Thu Feb 26 05:16:00 2009
@@ -175,8 +175,7 @@
                 if(temp.returnStatus!=POStatus.STATUS_OK)
                     return temp;
                 
-                /* Refer Pig-597 */
-                /* if(op instanceof POProject &&
+                if(op instanceof POProject &&
                         op.getResultType() == DataType.TUPLE){
                     POProject projOp = (POProject)op;
                     if(projOp.isStar()){
@@ -186,7 +185,7 @@
                             rslt.append(trslt.get(i));
                         continue;
                     }
-                }*/
+                }
                 ((Tuple)res.result).append(temp.result);
 			}
 			res.returnStatus = temp.returnStatus;

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/ARITY.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/ARITY.java?rev=748020&r1=748019&r2=748020&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/ARITY.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/ARITY.java Thu Feb 26 05:16:00 2009
@@ -31,12 +31,9 @@
 
     @Override
     public Integer exec(Tuple input) throws IOException {
-        if (input.size() == 0)
-            return null;
         try{
-            Tuple t = (Tuple)input.get(0);
-            if (t == null) return null;
-            return new Integer(t.size());
+            if (input == null) return null;
+            return new Integer(input.size());
         } catch (Exception e) {
             int errCode = 2106;
             String msg = "Error while computing arity in " + this.getClass().getSimpleName();

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/TupleSize.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/TupleSize.java?rev=748020&r1=748019&r2=748020&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/TupleSize.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/TupleSize.java Thu Feb 26 05:16:00 2009
@@ -33,12 +33,9 @@
 
     @Override
     public Long exec(Tuple input) throws IOException {
-        if (input.size() == 0)
-            return null;
         try{
-            Tuple t = (Tuple)input.get(0);
-            if (t == null) return null;
-            return new Long(t.size());
+            if (input == null) return null;
+            return new Long(input.size());
         }catch(Exception e){
             int errCode = 2106;
             String msg = "Error while computing size in " + this.getClass().getSimpleName();

Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java?rev=748020&r1=748019&r2=748020&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java Thu Feb 26 05:16:00 2009
@@ -1047,18 +1047,16 @@
         
         
         // Tuple size
-        Tuple t5 = TupleFactory.getInstance().newTuple();
-        t5.append(t1);
         expected = new Long(3);
         size = new TupleSize();
         msg = "[Testing TupleSize on input type: Tuple]";
-        assertTrue(msg, expected.equals(size.exec(t5)));
+        assertTrue(msg, expected.equals(size.exec(t1)));
         
         // Test for ARITY function.
         // It is depricated but we still need to make sure it works
         ARITY arrity = new ARITY();
         msg = "[Testing ARRITY on input type: Tuple]";
-        assertTrue(msg, expected.equals(new Long(arrity.exec(t5))));
+        assertTrue(msg, expected.equals(new Long(arrity.exec(t1))));
     }
 
     // Builtin APPLY Functions

Added: hadoop/pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java?rev=748020&view=auto
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java (added)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java Thu Feb 26 05:16:00 2009
@@ -0,0 +1,95 @@
+/*
+ * 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 static org.apache.pig.ExecType.MAPREDUCE;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.PrintStream;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.pig.ExecType;
+import org.apache.pig.PigServer;
+import org.apache.pig.data.*;
+import org.apache.pig.impl.io.FileLocalizer;
+import org.apache.pig.test.utils.Identity;
+
+import junit.framework.TestCase;
+
+public class TestEvalPipeline2 extends TestCase {
+    
+    MiniCluster cluster = MiniCluster.buildCluster();
+    private PigServer pigServer;
+
+    TupleFactory mTf = TupleFactory.getInstance();
+    BagFactory mBf = BagFactory.getInstance();
+    
+    @Before
+    @Override
+    public void setUp() throws Exception{
+        FileLocalizer.setR(new Random());
+        pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+//        pigServer = new PigServer(ExecType.LOCAL);
+    }
+    
+    @Test
+    public void testUDFwithStarInput() throws Exception {
+        int LOOP_COUNT = 10;
+        File tmpFile = File.createTempFile("test", "txt");
+        PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
+        Random r = new Random();
+        for(int i = 0; i < LOOP_COUNT; i++) {
+            for(int j=0;j<LOOP_COUNT;j+=2){
+                ps.println(i+"\t"+j);
+                ps.println(i+"\t"+j);
+            }
+        }
+        ps.close();
+
+        pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "';");
+        pigServer.registerQuery("B = group A by $0;");
+        String query = "C = foreach B {"
+        + "generate " + Identity.class.getName() +"(*);"
+        + "};";
+
+        pigServer.registerQuery(query);
+        Iterator<Tuple> iter = pigServer.openIterator("C");
+        if(!iter.hasNext()) fail("No output found");
+        int numIdentity = 0;
+        while(iter.hasNext()){
+            Tuple tuple = iter.next();
+            Tuple t = (Tuple)tuple.get(0);
+            assertEquals(DataByteArray.class, t.get(0).getClass());
+            int group = Integer.parseInt(new String(((DataByteArray)t.get(0)).get()));
+            assertEquals(numIdentity, group);
+            assertTrue(t.get(1) instanceof DataBag);
+            DataBag bag = (DataBag)t.get(1);
+            assertEquals(10, bag.size());
+            assertEquals(2, t.size());
+            ++numIdentity;
+        }
+        assertEquals(LOOP_COUNT, numIdentity);
+
+    }
+
+}

Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestFRJoin.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestFRJoin.java?rev=748020&r1=748019&r2=748020&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestFRJoin.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestFRJoin.java Thu Feb 26 05:16:00 2009
@@ -98,13 +98,12 @@
         }
 
         @Override
-        public DataBag exec(Tuple in) throws IOException {
+        public DataBag exec(Tuple input) throws IOException {
             if(!isTblSetUp){
                 setUpHashTable();
                 isTblSetUp = true;
             }
             try {
-                Tuple input = (Tuple)in.get(0);
                 String key = (String) input.get(keyField);
                 if(!replTbl.containsKey(key)) return BagFactory.getInstance().newDefaultBag();
                 return replTbl.get(key);