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/05/14 00:59:52 UTC

svn commit: r774583 - in /hadoop/pig/branches/pre-multiquery-phase2: CHANGES.txt src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt test/org/apache/pig/test/TestTypeCheckingValidator.java

Author: pradeepkth
Date: Wed May 13 22:59:51 2009
New Revision: 774583

URL: http://svn.apache.org/viewvc?rev=774583&view=rev
Log:
PIG-804: problem with lineage with double map redirection (pradeepkth)

Modified:
    hadoop/pig/branches/pre-multiquery-phase2/CHANGES.txt
    hadoop/pig/branches/pre-multiquery-phase2/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt
    hadoop/pig/branches/pre-multiquery-phase2/test/org/apache/pig/test/TestTypeCheckingValidator.java

Modified: hadoop/pig/branches/pre-multiquery-phase2/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/pre-multiquery-phase2/CHANGES.txt?rev=774583&r1=774582&r2=774583&view=diff
==============================================================================
--- hadoop/pig/branches/pre-multiquery-phase2/CHANGES.txt (original)
+++ hadoop/pig/branches/pre-multiquery-phase2/CHANGES.txt Wed May 13 22:59:51 2009
@@ -32,6 +32,8 @@
 
 BUG FIXES
 
+PIG-804: problem with lineage with double map redirection (pradeepkth)
+
 PIG-733: Order by sampling dumps entire sample to hdfs which causes dfs
 "FileSystem closed" error on large input (pradeepkth)
 

Modified: hadoop/pig/branches/pre-multiquery-phase2/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/pre-multiquery-phase2/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt?rev=774583&r1=774582&r2=774583&view=diff
==============================================================================
--- hadoop/pig/branches/pre-multiquery-phase2/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt (original)
+++ hadoop/pig/branches/pre-multiquery-phase2/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt Wed May 13 22:59:51 2009
@@ -2502,6 +2502,13 @@
         if(castRequired) {
             cast = new LOCast(lp, new OperatorKey(scope, getNextId()), fs.type);
             fs.alias = exprOp.getFieldSchema().alias;
+            // unset the field schema computed above since it should
+            // be recomputed later from the TypeCheckingVisitor
+            // This is because operators might be added/removed
+            // from the plan which might affect the field schema. So
+            // the TypeCheckingVisitor would be right place to
+            // compute the field schema
+            exprOp.unsetFieldSchema();
             cast.setFieldSchema(fs);
             lp.add(cast);
 		    log.debug("Added operator " + cast.getClass().getName() + " " + cast + " to logical plan " + lp);

Modified: hadoop/pig/branches/pre-multiquery-phase2/test/org/apache/pig/test/TestTypeCheckingValidator.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/pre-multiquery-phase2/test/org/apache/pig/test/TestTypeCheckingValidator.java?rev=774583&r1=774582&r2=774583&view=diff
==============================================================================
--- hadoop/pig/branches/pre-multiquery-phase2/test/org/apache/pig/test/TestTypeCheckingValidator.java (original)
+++ hadoop/pig/branches/pre-multiquery-phase2/test/org/apache/pig/test/TestTypeCheckingValidator.java Wed May 13 22:59:51 2009
@@ -31,8 +31,10 @@
 import org.apache.pig.impl.logicalLayer.* ;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema ;
+import org.apache.pig.impl.plan.DepthFirstWalker;
 import org.apache.pig.impl.plan.PlanValidationException;
 import org.apache.pig.impl.plan.CompilationMessageCollector;
+import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.util.MultiMap;
 import org.apache.pig.data.*;
 import org.apache.pig.impl.io.FileSpec;
@@ -5534,6 +5536,52 @@
     }
 
     @Test
+    public void testMapLookupLineage3() throws Throwable {
+        planTester.buildPlan("a = load 'a' as (s, m, l);") ;
+        planTester.buildPlan("b = foreach a generate s#'src_spaceid' AS vspaceid, flatten(l#'viewinfo') as viewinfo ;") ;
+        LogicalPlan plan = planTester.buildPlan("c = foreach b generate (chararray)vspaceid#'foo', (chararray)viewinfo#'pos' as position;") ;
+
+        // 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("Expect no  error") ;
+        }
+
+        CastFinder cf = new CastFinder(plan);
+        cf.visit();
+        List<LOCast> casts = cf.casts;
+        for (LOCast cast : casts) {
+            assertTrue(cast.getLoadFuncSpec().getClassName().startsWith("org.apache.pig.builtin.PigStorage"));    
+        }
+    }
+    
+    class CastFinder extends LOVisitor {
+        List<LOCast> casts = new ArrayList<LOCast>();
+        /**
+         * 
+         */
+        public CastFinder(LogicalPlan lp) {
+            // TODO Auto-generated constructor stub
+            super(lp, new DepthFirstWalker<LogicalOperator, LogicalPlan>(lp));
+        }
+        
+        /* (non-Javadoc)
+         * @see org.apache.pig.impl.logicalLayer.LOVisitor#visit(org.apache.pig.impl.logicalLayer.LOCast)
+         */
+        @Override
+        protected void visit(LOCast cast) throws VisitorException {
+            casts.add(cast);
+        }
+    }
+    
+    @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;") ;