You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by pr...@apache.org on 2009/03/03 03:53:42 UTC

svn commit: r749514 - in /hadoop/pig/trunk: CHANGES.txt src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java test/org/apache/pig/test/TestTypeCheckingValidator.java

Author: pradeepkth
Date: Tue Mar  3 02:53:42 2009
New Revision: 749514

URL: http://svn.apache.org/viewvc?rev=749514&view=rev
Log:
PIG-655: Comparison of schemas of bincond operands is flawed (sms via pradeepkth)

Modified:
    hadoop/pig/trunk/CHANGES.txt
    hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java
    hadoop/pig/trunk/test/org/apache/pig/test/TestTypeCheckingValidator.java

Modified: hadoop/pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=749514&r1=749513&r2=749514&view=diff
==============================================================================
--- hadoop/pig/trunk/CHANGES.txt (original)
+++ hadoop/pig/trunk/CHANGES.txt Tue Mar  3 02:53:42 2009
@@ -436,3 +436,7 @@
     PIG-664: Semantics of * is not consistent (sms)
 
     PIG-684: outputSchema method in TOKENIZE is broken (thejas via sms)
+
+    PIG-655: Comparison of schemas of bincond operands is flawed (sms via
+    pradeepkth)
+

Modified: hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java?rev=749514&r1=749513&r2=749514&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/validators/TypeCheckingVisitor.java Tue Mar  3 02:53:42 2009
@@ -1773,9 +1773,11 @@
             // Matching schemas if we're working with tuples
             if (DataType.isSchemaType(lhsType)) {            
                 try {
-                    if (!Schema.equals(binCond.getLhsOp().getSchema(), binCond.getRhsOp().getSchema(), false, true)) {
+                    if (!Schema.FieldSchema.equals(binCond.getLhsOp().getFieldSchema(), binCond.getRhsOp().getFieldSchema(), false, true)) {
                         int errCode = 1048;
-                        String msg = "Two inputs of BinCond must have compatible schemas" ;
+                        String msg = "Two inputs of BinCond must have compatible schemas." 
+                            + " left hand side: " + binCond.getLhsOp().getFieldSchema() 
+                            + " right hand side: " + binCond.getRhsOp().getFieldSchema();
                         msgCollector.collect(msg, MessageType.Error) ;
                         throw new TypeCheckerException(msg, errCode, PigException.INPUT) ;
                     }

Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestTypeCheckingValidator.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestTypeCheckingValidator.java?rev=749514&r1=749513&r2=749514&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestTypeCheckingValidator.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestTypeCheckingValidator.java Tue Mar  3 02:53:42 2009
@@ -25,6 +25,7 @@
 
 import junit.framework.TestCase;
 
+import org.apache.pig.EvalFunc;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.impl.logicalLayer.validators.*;
 import org.apache.pig.impl.logicalLayer.* ;
@@ -5556,7 +5557,79 @@
         assertTrue(cast.getLoadFuncSpec().getClassName().startsWith("org.apache.pig.builtin.PigStorage"));
 
     }
+    
+    @Test
+    public void testBincond() throws Throwable {
+        planTester.buildPlan("a = load 'a' as (name: chararray, age: int, gpa: float);") ;
+        planTester.buildPlan("b = group a by name;") ;
+        LogicalPlan plan = planTester.buildPlan("c = foreach b generate (IsEmpty(a) ? " + TestBinCondFieldSchema.class.getName() + "(*): a) ;") ;
+
+        // validate
+        CompilationMessageCollector collector = new CompilationMessageCollector() ;
+        TypeCheckingValidator typeValidator = new TypeCheckingValidator() ;
+        
+        typeValidator.validate(plan, collector) ;
+
+        printMessageCollector(collector) ;
+        printTypeGraph(plan) ;
+        planTester.printPlan(plan, TypeCheckingTestUtil.getCurrentMethodName());
+
+        if (collector.hasError()) {
+            throw new AssertionError("Did not expect an error") ;
+        }
+
 
+        LOForEach foreach = (LOForEach)plan.getLeaves().get(0);
+        
+        Schema.FieldSchema charFs = new FieldSchema(null, DataType.CHARARRAY);
+        Schema.FieldSchema intFs = new FieldSchema(null, DataType.INTEGER);
+        Schema.FieldSchema floatFs = new FieldSchema(null, DataType.FLOAT);
+        Schema bagSchema = new Schema();
+        bagSchema.add(charFs);
+        bagSchema.add(intFs);
+        bagSchema.add(floatFs);
+        Schema.FieldSchema bagFs = null;
+        try {
+            bagFs = new Schema.FieldSchema(null, bagSchema, DataType.BAG);
+        } catch (FrontendException fee) {
+            fail("Did not expect an error");
+        }
+        
+        Schema expectedSchema = new Schema(bagFs);
+        
+        assertTrue(Schema.equals(foreach.getSchema(), expectedSchema, false, true));
+
+    }
+
+    /*
+     * A test UDF that does not data processing but implements the getOutputSchema for
+     * checking the type checker
+     */
+    public static class TestBinCondFieldSchema extends EvalFunc<DataBag> {
+        //no-op exec method
+        public DataBag exec(Tuple input) {
+            return null;
+        }
+        
+        @Override
+        public Schema outputSchema(Schema input) {
+            Schema.FieldSchema charFs = new FieldSchema(null, DataType.CHARARRAY);
+            Schema.FieldSchema intFs = new FieldSchema(null, DataType.INTEGER);
+            Schema.FieldSchema floatFs = new FieldSchema(null, DataType.FLOAT);
+            Schema bagSchema = new Schema();
+            bagSchema.add(charFs);
+            bagSchema.add(intFs);
+            bagSchema.add(floatFs);
+            Schema.FieldSchema bagFs;
+            try {
+                bagFs = new Schema.FieldSchema(null, bagSchema, DataType.BAG);
+            } catch (FrontendException fee) {
+                return null;
+            }
+            return new Schema(bagFs);
+        }
+    }
+    
     ////////////////////////// Helper //////////////////////////////////
     private void checkForEachCasting(LOForEach foreach, int idx, boolean isCast, byte toType) {
         LogicalPlan plan = foreach.getForEachPlans().get(idx) ;