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 2010/08/05 22:41:36 UTC

svn commit: r982772 - in /hadoop/pig/trunk: ./ src/org/apache/pig/ test/org/apache/pig/test/ test/org/apache/pig/test/utils/

Author: daijy
Date: Thu Aug  5 20:41:35 2010
New Revision: 982772

URL: http://svn.apache.org/viewvc?rev=982772&view=rev
Log:
PIG-1288: EvalFunc returnType is wrong for generic subclasses

Added:
    hadoop/pig/trunk/test/org/apache/pig/test/TestUDF.java
    hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF1.java
    hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF2.java
    hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF3.java
Removed:
    hadoop/pig/trunk/test/org/apache/pig/test/TestUDFReturnMap.java
Modified:
    hadoop/pig/trunk/CHANGES.txt
    hadoop/pig/trunk/src/org/apache/pig/EvalFunc.java

Modified: hadoop/pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=982772&r1=982771&r2=982772&view=diff
==============================================================================
--- hadoop/pig/trunk/CHANGES.txt (original)
+++ hadoop/pig/trunk/CHANGES.txt Thu Aug  5 20:41:35 2010
@@ -118,6 +118,8 @@ PIG-1309: Map-side Cogroup (ashutoshc)
 
 BUG FIXES
 
+PIG-1288: EvalFunc returnType is wrong for generic subclasses (daijy)
+
 PIG-1534: Code discovering UDFs in the script has a bug in a order by case
 (pradeepkth)
 

Modified: hadoop/pig/trunk/src/org/apache/pig/EvalFunc.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/EvalFunc.java?rev=982772&r1=982771&r2=982772&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/EvalFunc.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/EvalFunc.java Thu Aug  5 20:41:35 2010
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
 import java.util.List;
+import java.util.Stack;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -92,24 +93,46 @@ public abstract class EvalFunc<T>  {
         Class<?> superClass = getClass();
         Type superType = getClass();
         
-        while (!superClass.isAssignableFrom(EvalFunc.class)){
+        Stack<Type> geneticsStack = new Stack<Type>();
+        
+        // Go up the hierachy of the class up to the EvalFunc
+        while (!superClass.isAssignableFrom(EvalFunc.class))
+        {
             superType = superClass.getGenericSuperclass();
             superClass = superClass.getSuperclass();
+            geneticsStack.push(superType);
+        }
+        
+        // From EvalFunc (superclass), go downward (subclass), 
+        // find the first class materialize the genetics
+        Type materializedType = null;
+        while (!geneticsStack.isEmpty()) {
+            Type aType = geneticsStack.pop();
+            if (aType instanceof ParameterizedType) {
+                // We materialized something, eg, materialized the type to Double,
+                // or materialized the type to Map<String, Object>, or materialized the type
+                // to T(another genetics). In the 1st case, getActualTypeArguments()
+                // returns a class, we can tell easily; In the 2nd and 3th case, 
+                // getActualTypeArguments() returns a ParameterizedType, 
+                // we cannot tell 2nd case from 3th case.
+                // So we need further check if the type inside materializedType 
+                // are materialized (case 2)
+                materializedType = ((ParameterizedType)aType).getActualTypeArguments()[0];
+            }
+            Type currentType = materializedType;
+            while (currentType instanceof ParameterizedType)
+                currentType = ((ParameterizedType)currentType).getActualTypeArguments()[0];
+            if (currentType instanceof Class) {
+                returnType = materializedType;
+                break;
+            }
         }
+
         String errMsg = getClass() + "extends the raw type EvalFunc. It should extend the parameterized type EvalFunc<T> instead.";
         
-        if (!(superType instanceof ParameterizedType))
+        if (returnType==null)
             throw new RuntimeException(errMsg);
         
-        Type[] parameters  = ((ParameterizedType)superType).getActualTypeArguments();
-        
-        if (parameters.length != 1)
-                throw new RuntimeException(errMsg);
-        
-        returnType = parameters[0];
-        
-        
-        
         //Type check the initial, intermediate, and final functions
         if (this instanceof Algebraic){
             Algebraic a = (Algebraic)this;

Added: hadoop/pig/trunk/test/org/apache/pig/test/TestUDF.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestUDF.java?rev=982772&view=auto
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestUDF.java (added)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestUDF.java Thu Aug  5 20:41:35 2010
@@ -0,0 +1,141 @@
+/*
+ * 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.FileWriter;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+
+import junit.framework.TestCase;
+
+import org.apache.pig.ExecType;
+import org.apache.pig.PigServer;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.io.FileLocalizer;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.test.utils.MyUDFReturnMap;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class TestUDF extends TestCase {
+
+	static String[] ScriptStatement = {
+			"A = LOAD 'test/org/apache/pig/test/data/passwd' USING PigStorage();",
+			"B = FOREACH A GENERATE org.apache.pig.test.utils.MyUDFReturnMap(1);" };
+
+	static File TempScriptFile = null;
+
+	static MiniCluster cluster = MiniCluster.buildCluster();
+
+	@Override
+	@Before
+	public void setUp() throws Exception {
+		TempScriptFile = File.createTempFile("temp_jira_851", ".pig");
+		FileWriter writer = new FileWriter(TempScriptFile);
+		for (String line : ScriptStatement) {
+			writer.write(line + "\n");
+		}
+		writer.close();
+	}
+
+	@AfterClass
+	public static void oneTimeTearDown() throws Exception {
+	    cluster.shutDown();
+	}
+	
+	@Test
+	public void testUDFReturnMap_LocalMode() {
+		try {
+			PigServer pig = new PigServer(ExecType.LOCAL);
+			pig.registerScript(TempScriptFile.getAbsolutePath());
+
+			Iterator<Tuple> iterator = pig.openIterator("B");
+			int index = 0;
+			while (iterator.hasNext()) {
+				Tuple tuple = iterator.next();
+				index++;
+				Map<Object, Object> result = (Map<Object, Object>) tuple.get(0);
+				assertEquals(result, MyUDFReturnMap.map);
+			}
+		} catch (IOException e) {
+			e.printStackTrace();
+			fail();
+		}
+	}
+
+	@Test
+	public void testUDFReturnMap_MapReduceMode() {
+		try {
+			Util.createInputFile(cluster, "a.txt", new String[] { "dummy",
+					"dummy" });
+			FileLocalizer.deleteTempFiles();
+			PigServer pig = new PigServer(ExecType.MAPREDUCE, cluster
+					.getProperties());
+			pig.registerQuery("A = LOAD 'a.txt';");
+			pig
+					.registerQuery("B = FOREACH A GENERATE org.apache.pig.test.utils.MyUDFReturnMap();");
+
+			Iterator<Tuple> iterator = pig.openIterator("B");
+			int index = 0;
+			while (iterator.hasNext()) {
+				Tuple tuple = iterator.next();
+				index++;
+				Map<Object, Object> result = (Map<Object, Object>) tuple.get(0);
+				assertEquals(result, MyUDFReturnMap.map);
+			}
+		} catch (IOException e) {
+			e.printStackTrace();
+			fail();
+		}
+	}
+	
+	@Test
+	public void testUDFMultiLevelOutputSchema() {
+        try {
+            PigServer pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+            pig.registerQuery("A = LOAD 'a.txt';");
+            pig.registerQuery("B = FOREACH A GENERATE org.apache.pig.test.utils.MultiLevelDerivedUDF1();");
+            pig.registerQuery("C = FOREACH A GENERATE org.apache.pig.test.utils.MultiLevelDerivedUDF2();");
+            pig.registerQuery("D = FOREACH A GENERATE org.apache.pig.test.utils.MultiLevelDerivedUDF3();");
+            Schema s = pig.dumpSchema("B");
+            assertTrue(s.getField(0).type == DataType.DOUBLE);
+            s = pig.dumpSchema("C");
+            assertTrue(s.getField(0).type == DataType.DOUBLE);
+            s = pig.dumpSchema("D");
+            assertTrue(s.getField(0).type == DataType.DOUBLE);
+        } catch (IOException e) {
+            e.printStackTrace();
+            fail();
+        }
+    }
+
+	@Override
+	@After
+	public void tearDown() throws Exception {
+		TempScriptFile.delete();
+	}
+}

Added: hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF1.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF1.java?rev=982772&view=auto
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF1.java (added)
+++ hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF1.java Thu Aug  5 20:41:35 2010
@@ -0,0 +1,40 @@
+/*
+ * 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.utils;
+
+import java.io.IOException;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+abstract class MyAbstractEvalFunc1<T> extends EvalFunc<T> {
+}
+
+public class MultiLevelDerivedUDF1  extends MyAbstractEvalFunc1<Double>{
+    public Double exec(Tuple input) throws IOException {
+        return new Double(1);
+    }
+
+    @Override
+    public Schema outputSchema(Schema input) {
+        return new Schema(new Schema.FieldSchema(getSchemaName(this.getClass().getName().toLowerCase(), input), DataType.DOUBLE));
+    }
+}

Added: hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF2.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF2.java?rev=982772&view=auto
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF2.java (added)
+++ hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF2.java Thu Aug  5 20:41:35 2010
@@ -0,0 +1,40 @@
+/*
+ * 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.utils;
+
+import java.io.IOException;
+
+import org.apache.pig.EvalFunc;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+abstract class MyAbstractEvalFunc2 extends EvalFunc<Double> {
+}
+
+public class MultiLevelDerivedUDF2  extends MyAbstractEvalFunc2{
+    public Double exec(Tuple input) throws IOException {
+        return new Double(1);
+    }
+
+    @Override
+    public Schema outputSchema(Schema input) {
+        return new Schema(new Schema.FieldSchema(getSchemaName(this.getClass().getName().toLowerCase(), input), DataType.DOUBLE));
+    }
+}

Added: hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF3.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF3.java?rev=982772&view=auto
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF3.java (added)
+++ hadoop/pig/trunk/test/org/apache/pig/test/utils/MultiLevelDerivedUDF3.java Thu Aug  5 20:41:35 2010
@@ -0,0 +1,22 @@
+/*
+ * 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.utils;
+
+public class MultiLevelDerivedUDF3 extends MultiLevelDerivedUDF2 {
+}