You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by pi...@apache.org on 2008/07/04 17:23:01 UTC

svn commit: r674075 - in /incubator/pig/branches/types: src/org/apache/pig/impl/logicalLayer/schema/ test/org/apache/pig/test/ test/org/apache/pig/test/data/DotFiles/ test/org/apache/pig/test/utils/dotGraph/

Author: pisong
Date: Fri Jul  4 08:23:00 2008
New Revision: 674075

URL: http://svn.apache.org/viewvc?rev=674075&view=rev
Log:
work around null alias issue in Dot

Added:
    incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript5.dot
    incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript6.dot
Modified:
    incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestTypeChecking.java
    incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript4.dot
    incubator/pig/branches/types/test/org/apache/pig/test/utils/dotGraph/LogicalPlanLoader.java

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java?rev=674075&r1=674074&r2=674075&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/schema/Schema.java Fri Jul  4 08:23:00 2008
@@ -247,8 +247,23 @@
                 return false ;
             }
 
-            if ( (!relaxAlias) && (!fschema.alias.equals(fother.alias)) ) {
-                return false ;
+
+            if (!relaxAlias) {
+                if ( (fschema.alias == null) &&
+                     (fother.alias == null) ) {
+                    // good
+                }
+                else if ( (fschema.alias != null) &&
+                          (fother.alias == null) ) {
+                    return false ;
+                }
+                else if ( (fschema.alias == null) &&
+                          (fother.alias != null) ) {
+                    return false ;
+                }
+                else if (!fschema.alias.equals(fother.alias)) {
+                    return false ;
+                }
             }
 
             if ( (!relaxInner) && (DataType.isSchemaType(fschema.type))) {
@@ -744,8 +759,22 @@
             FieldSchema myFs = i.next() ;
             FieldSchema otherFs = j.next() ;
 
-            if ( (!relaxAlias) && (!myFs.alias.equals(otherFs.alias)) ) {
-                return false ;
+            if (!relaxAlias) {
+                if ( (myFs.alias == null) &&
+                     (otherFs.alias == null) ) {
+                    // good
+                }
+                else if ( (myFs.alias != null) &&
+                     (otherFs.alias == null) ) {
+                    return false ;
+                }
+                else if ( (myFs.alias == null) && 
+                     (otherFs.alias != null) ) {
+                    return false ;
+                }
+                else if (!myFs.alias.equals(otherFs.alias)) {
+                    return false ;
+                }
             }
 
             if (myFs.type != otherFs.type) {

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestTypeChecking.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestTypeChecking.java?rev=674075&r1=674074&r2=674075&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestTypeChecking.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestTypeChecking.java Fri Jul  4 08:23:00 2008
@@ -58,9 +58,9 @@
         planTester.typeCheckUsingDotFile(FILE_BASE_LOCATION + "testScript2.dot");
     }
 
-    // Problem with schema parser in QueryParser
+    // Problem with "group" keyword in QueryParser
     /*
-        @Test
+    @Test
     public void testByScript4() throws Throwable {
         TypeCheckingTestUtil.printCurrentMethodName() ;
         planTester.typeCheckUsingDotFile(FILE_BASE_LOCATION + "testScript4.dot");
@@ -68,15 +68,25 @@
     */
 
     /*
-
     @Test
     public void testByScript3() throws Throwable {
         TypeCheckingTestUtil.printCurrentMethodName() ;
         planTester.typeCheckUsingDotFile(FILE_BASE_LOCATION + "testScript3.dot");
     }
-
     */
 
+    @Test
+    public void testByScript5() throws Throwable {
+        TypeCheckingTestUtil.printCurrentMethodName() ;
+        planTester.typeCheckUsingDotFile(FILE_BASE_LOCATION + "testScript5.dot");
+    }
+    
+    @Test
+    public void testByScript6() throws Throwable {
+        TypeCheckingTestUtil.printCurrentMethodName() ;
+        planTester.typeCheckUsingDotFile(FILE_BASE_LOCATION + "testScript6.dot");
+    }
+
     // TODO: Convert all of these to dot files
 
     /*
@@ -174,21 +184,7 @@
         validatePlan(plan) ;
     }
 
-    @Test
-    public void testValidation11() throws Throwable {
-        TypeCheckingTestUtil.printCurrentMethodName() ;
-        buildPlan("a = load 'a' as (name: chararray, details: tuple(age, gpa), field3: tuple(a,b));");
-		LogicalPlan plan = buildPlan("e = foreach a generate name, details.(age, gpa), field3.(a,b) ;");
-        validatePlan(plan) ;
-    }
 
-    @Test
-    public void testValidation11_2() throws Throwable {
-        TypeCheckingTestUtil.printCurrentMethodName() ;
-        buildPlan("a = load 'a' as (field1: tuple(a, b), field2: tuple(a, b), field3: tuple(a,b));");
-		LogicalPlan plan = buildPlan("e = foreach a generate field1.(a,b) , field2.(a, b), field3.(a,b) ;");
-        validatePlan(plan) ;
-    }
 
     @Test
     public void testValidation12() throws Throwable {

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript4.dot
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript4.dot?rev=674075&r1=674074&r2=674075&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript4.dot (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript4.dot Fri Jul  4 08:23:00 2008
@@ -5,10 +5,10 @@
 
     load1 [key="10", type="LOLoad", schema="field1: int, field2: long"] ;
     load2 [key="15", type="LOLoad", schema="field1: bytearray, field2: double"] ;
-    group [key="20", type="LOCogroup" , schema="group: tuple(field1:int, field2:double) ,
+    group [key="20", type="LOCogroup" , schema="group2: tuple([NoAlias]:int, [NoAlias]:double) ,
                                                 a: bag{tuple1:tuple(field1: int,field2: long)},
                                                 b: bag{tuple1:tuple(field1: bytearray,field2: double)} "] ;
 
     load1 -> group ;
     load2 -> group ;
-}
\ No newline at end of file
+}

Added: incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript5.dot
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript5.dot?rev=674075&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript5.dot (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript5.dot Fri Jul  4 08:23:00 2008
@@ -0,0 +1,9 @@
+digraph graph1 {
+    graph [pigScript=" a = load 'a' as (field1: long, field2: tuple(inner1 : bytearray, inner2 : float) ) ;
+                       b = FOREACH a GENERATE field1*field1 ;"] ;
+
+    load [key="10", type="LOLoad", schema="field1: long, field2: tuple(inner1 : bytearray, inner2 : float)"] ;
+    foreach [key="20", type="LOForEach" , schema="[NoAlias] : long"] ;
+
+    load -> foreach ;
+}

Added: incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript6.dot
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript6.dot?rev=674075&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript6.dot (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/DotFiles/testScript6.dot Fri Jul  4 08:23:00 2008
@@ -0,0 +1,9 @@
+digraph graph1 {
+    graph [pigScript=" a = load 'a' as (name: chararray, details: tuple(age, gpa), field3: tuple(a,b));
+                       e = foreach a generate name, details.(age, gpa), field3.(a,b) ; "] ;
+
+    load1 [key="10", type="LOLoad", schema="name: chararray, details: tuple(age, gpa), field3: tuple(a,b)"] ;
+    foreach1 [key="15", type="LOForEach", schema="name: chararray,details: tuple(age: bytearray,gpa: bytearray),field3: tuple(a: bytearray,b: bytearray)"] ;
+
+    load1 -> foreach1 ;
+}

Modified: incubator/pig/branches/types/test/org/apache/pig/test/utils/dotGraph/LogicalPlanLoader.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/utils/dotGraph/LogicalPlanLoader.java?rev=674075&r1=674074&r2=674075&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/utils/dotGraph/LogicalPlanLoader.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/utils/dotGraph/LogicalPlanLoader.java Fri Jul  4 08:23:00 2008
@@ -24,10 +24,13 @@
 import org.apache.pig.impl.logicalLayer.parser.ParseException ;
 import org.apache.pig.impl.io.FileSpec;
 import org.apache.pig.builtin.PigStorage;
+import org.apache.pig.data.DataType;
 
 import java.io.IOException;
 import java.io.ByteArrayInputStream;
 import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.regex.Matcher;
 
 
 public class LogicalPlanLoader
@@ -158,14 +161,30 @@
 
     private void fillSchema(LogicalOperator op, Map<String,String> attributes) {
         String schemaString = attributes.get("schema") ;
+
         if (schemaString != null) {
 
+            // Replace [NoAlias] with dummy names before set back to null
+            // due to the fact that the parser doesn't allow null alias
+            int dummyAliasCounter = 0 ;
+            String DUMMY_ALIAS_PREFIX = "MY_DUMMY_ALIAS_" ;
+            while (schemaString.indexOf("[NoAlias]") != -1) {
+                schemaString = schemaString.replaceFirst("\\[NoAlias\\]",
+                                    DUMMY_ALIAS_PREFIX + dummyAliasCounter++) ;
+            }
+
             ByteArrayInputStream stream
                     = new ByteArrayInputStream(schemaString.getBytes()) ;
             QueryParser queryParser = new QueryParser(stream) ;
             Schema schema = null ;
             try {
                 schema = queryParser.TupleSchema() ;
+                
+                // set all the [NoAlias] to null
+                for(int i=0; i < dummyAliasCounter; i++) {
+                    replaceAliasByNull(schema, DUMMY_ALIAS_PREFIX + i) ;
+                }
+
                 op.forceSchema(schema);
                 op.setSchemaComputed(true);
             }
@@ -178,4 +197,32 @@
             op.forceSchema(null);
         }
     }
+
+    private boolean replaceAliasByNull(Schema schema, String alias) {
+        if (schema != null) {
+            for(int i=0; i < schema.size(); i++) {
+                try {
+                    if ( (schema.getField(i).alias != null) &&
+                         (schema.getField(i).alias.equals(alias)) ) {
+                        schema.getField(i).alias = null ;
+                        return true ;
+                    }
+                    // We only do 1 alias per call so having an else
+                    // here is reasonable
+                    else {
+                        if ( (schema.getField(i).type == DataType.BAG) || 
+                             (schema.getField(i).type == DataType.TUPLE) ) {
+                            if (replaceAliasByNull(schema.getField(i).schema, alias)) {
+                                return true ;
+                            }
+                        }
+                    }
+
+                } catch (ParseException e) {
+                    throw new AssertionError("Cannot access schema internals") ;
+                }
+            }
+        }
+        return false ;
+    }
 }