You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by th...@apache.org on 2011/01/19 00:54:08 UTC

svn commit: r1060624 - in /pig/trunk: src/org/apache/pig/newplan/logical/relational/ src/org/apache/pig/parser/ test/org/apache/pig/parser/

Author: thejas
Date: Tue Jan 18 23:54:07 2011
New Revision: 1060624

URL: http://svn.apache.org/viewvc?rev=1060624&view=rev
Log:
PIG-1618: Switch to new parser generator technology - NewParser-12.patch - (xuefuz via thejas)

Modified:
    pig/trunk/src/org/apache/pig/newplan/logical/relational/LOUnion.java
    pig/trunk/src/org/apache/pig/parser/AstValidator.g
    pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java
    pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g
    pig/trunk/src/org/apache/pig/parser/QueryParser.g
    pig/trunk/test/org/apache/pig/parser/TestAstValidator.java
    pig/trunk/test/org/apache/pig/parser/TestQueryParser.java

Modified: pig/trunk/src/org/apache/pig/newplan/logical/relational/LOUnion.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/newplan/logical/relational/LOUnion.java?rev=1060624&r1=1060623&r2=1060624&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/newplan/logical/relational/LOUnion.java (original)
+++ pig/trunk/src/org/apache/pig/newplan/logical/relational/LOUnion.java Tue Jan 18 23:54:07 2011
@@ -31,13 +31,24 @@ import org.apache.pig.newplan.PlanVisito
 import org.apache.pig.newplan.logical.expression.LogicalExpression;
 
 public class LOUnion extends LogicalRelationalOperator {
-
+    private boolean onSchema;
+    
     // uid mapping from output uid to input uid
     private List<Pair<Long, Long>> uidMapping = new ArrayList<Pair<Long, Long>>();
     
     public LOUnion(OperatorPlan plan) {
         super("LOUnion", plan);
-    }    
+    }
+    
+    public LOUnion(OperatorPlan plan, boolean onSchema) {
+        this( plan );
+        this.onSchema = onSchema;
+    }
+    
+    public boolean isOnSchema() {
+        return onSchema;
+    }
+    
     @Override
     public LogicalSchema getSchema() throws FrontendException {
         if (schema != null) {

Modified: pig/trunk/src/org/apache/pig/parser/AstValidator.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/AstValidator.g?rev=1060624&r1=1060623&r2=1060624&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/AstValidator.g (original)
+++ pig/trunk/src/org/apache/pig/parser/AstValidator.g Tue Jan 18 23:54:07 2011
@@ -158,26 +158,23 @@ load_clause : ^( LOAD filename func_clau
 filename : QUOTEDSTRING
 ;
 
-as_clause: ^( AS tuple_def )
+as_clause: ^( AS field_def_list )
 ;
 
-tuple_def : tuple_def_full | tuple_def_simple
+field_def[Set<String> fieldNames] throws Exception
+ : ^( FIELD_DEF IDENTIFIER { validateSchemaAliasName( fieldNames, $IDENTIFIER.text ); } )
+-> ^( FIELD_DEF IDENTIFIER BYTEARRAY )
+ | ^( FIELD_DEF IDENTIFIER { validateSchemaAliasName( fieldNames, $IDENTIFIER.text ); } type )
 ;
 
-tuple_def_full
+field_def_list
 scope{
-Set<String> fieldNames;
-} : { $tuple_def_full::fieldNames = new HashSet<String>(); }
-    ^( TUPLE_DEF field[$tuple_def_full::fieldNames]+ )
-;
-
-tuple_def_simple : ^( TUPLE_DEF field[new HashSet<String>()] )
-;
-
-field[Set<String> fieldNames] throws Exception
- : ^( FIELD IDENTIFIER { validateSchemaAliasName( fieldNames, $IDENTIFIER.text ); } )
--> ^( FIELD IDENTIFIER BYTEARRAY )
- | ^( FIELD IDENTIFIER { validateSchemaAliasName( fieldNames, $IDENTIFIER.text ); } type )
+    Set<String> fieldNames;
+}
+@init {
+    $field_def_list::fieldNames = new HashSet<String>();
+}
+ : ( field_def[$field_def_list::fieldNames] )+
 ;
 
 type : simple_type | tuple_type | bag_type | map_type
@@ -186,10 +183,10 @@ type : simple_type | tuple_type | bag_ty
 simple_type : INT | LONG | FLOAT | DOUBLE | CHARARRAY | BYTEARRAY
 ;
 
-tuple_type : ^( TUPLE_TYPE tuple_def_full )
+tuple_type : ^( TUPLE_TYPE field_def_list )
 ;
 
-bag_type : ^( BAG_TYPE tuple_def? )
+bag_type : ^( BAG_TYPE tuple_type? )
 ;
 
 map_type : MAP_TYPE
@@ -222,8 +219,7 @@ rel : alias {  validateAliasRef( aliases
     | op_clause
 ;
 
-flatten_generated_item : flatten_clause as_clause?
-                       | ( expr | STAR ) field[new HashSet<String>()]?
+flatten_generated_item : ( flatten_clause | expr | START ) field_def_list?
 ;
 
 flatten_clause : ^( FLATTEN expr )

Modified: pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java?rev=1060624&r1=1060623&r2=1060624&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java (original)
+++ pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java Tue Jan 18 23:54:07 2011
@@ -139,8 +139,8 @@ public class LogicalPlanBuilder {
         return buildFilterOp( new LOFilter( plan ), alias, parallel, inputAlias, filterPlan );
     }
     
-    String buildUnionOp(String alias, Integer parallel, List<String> inputAliases) {
-        LOUnion op = new LOUnion( plan );
+    String buildUnionOp(String alias, Integer parallel, List<String> inputAliases, boolean onSchema) {
+        LOUnion op = new LOUnion( plan, onSchema );
         return buildOp( op, alias, parallel, inputAliases, null );
     }
 
@@ -389,7 +389,7 @@ public class LogicalPlanBuilder {
             if( limit != null )
                 command.setLogFilesLimit( limit );
         } catch(IOException e) {
-        	throw new PlanGenerationFailureException( input, e );
+            throw new PlanGenerationFailureException( input, e );
         }
         
         return command;
@@ -403,7 +403,7 @@ public class LogicalPlanBuilder {
     }
     
     String buildStreamOp(String alias, Integer parallel, String inputAlias, StreamingCommand command,
-    		LogicalSchema schema, IntStream input)
+            LogicalSchema schema, IntStream input)
     throws RecognitionException {
         try {
             LOStream op = new LOStream( plan, pigContext.createExecutableManager(), command, schema );

Modified: pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g?rev=1060624&r1=1060623&r2=1060624&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g (original)
+++ pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g Tue Jan 18 23:54:07 2011
@@ -267,34 +267,24 @@ filename returns[String filename]
 ;
 
 as_clause returns[LogicalSchema logicalSchema]
- : ^( AS tuple_def ) { $logicalSchema = $tuple_def.logicalSchema; }
+ : ^( AS field_def_list ) { $logicalSchema = $field_def_list.schema; }
 ;
 
-tuple_def returns[LogicalSchema logicalSchema]
- : tuple_def_full { $logicalSchema = $tuple_def_full.logicalSchema; }
- | tuple_def_simple { $logicalSchema = $tuple_def_simple.logicalSchema; }
-;
-
-tuple_def_full returns[LogicalSchema logicalSchema]
-@init { $logicalSchema = new LogicalSchema(); }
- : ^( TUPLE_DEF ( field { $logicalSchema.addField( $field.fieldSchema ); } )+ ) 
-;
-
-tuple_def_simple returns[LogicalSchema logicalSchema]
- : ^( TUPLE_DEF field )
+field_def returns[LogicalFieldSchema fieldSchema]
+ : ^( FIELD_DEF IDENTIFIER type )
    {
-       $logicalSchema = new LogicalSchema();
-       $logicalSchema.addField( $field.fieldSchema );
+       $fieldSchema = new LogicalFieldSchema( $IDENTIFIER.text, $type.logicalSchema, $type.datatype );
    }
 ;
 
-field returns[LogicalFieldSchema fieldSchema]
- : ^( FIELD IDENTIFIER type )
-   {
-       $fieldSchema = new LogicalFieldSchema( $IDENTIFIER.text, $type.logicalSchema, $type.datatype );
-   }
+field_def_list returns[LogicalSchema schema]
+@init {
+    $schema = new LogicalSchema();
+}
+ : ( field_def { $schema.addField( $field_def.fieldSchema ); } )+
 ;
 
+
 type returns[byte datatype, LogicalSchema logicalSchema]
  : simple_type
    {
@@ -326,16 +316,16 @@ simple_type returns[byte datatype]
 ;
 
 tuple_type returns[LogicalSchema logicalSchema]
- : ^( TUPLE_TYPE tuple_def_full )
+ : ^( TUPLE_TYPE field_def_list )
    { 
-       $logicalSchema = $tuple_def_full.logicalSchema;
+       $logicalSchema = $field_def_list.schema;
    }
 ;
 
 bag_type returns[LogicalSchema logicalSchema]
- : ^( BAG_TYPE tuple_def? )
+ : ^( BAG_TYPE tuple_type? )
    { 
-       $logicalSchema = $tuple_def.logicalSchema;
+       $logicalSchema = $tuple_type.logicalSchema;
    }
 ;
 
@@ -350,6 +340,8 @@ func_clause returns[FuncSpec funcSpec]
  | ^( FUNC_REF func_alias )
    {
        $funcSpec = builder.lookupFunction( $func_alias.alias );
+       if( $funcSpec == null )
+           $funcSpec = builder.buildFuncSpec( $func_alias.alias, new ArrayList<String>() );
    }
 ;
 
@@ -453,15 +445,14 @@ flatten_generated_item returns[LogicalEx
 @init {
     $plan = new LogicalExpressionPlan();
 }
- : flatten_clause[$plan] { $flattenFlag = true; }
-   ( as_clause  { $schema = $as_clause.logicalSchema; } )?
- | ( expr[$plan] 
-   | STAR
+ : ( flatten_clause[$plan] { $flattenFlag = true; }
+   | expr[$plan]
+   | START
      {
          builder.buildProjectExpr( $plan, currentOp, $statement::inputIndex, null, -1 );
      }
    )
-   ( field { $schema =  new LogicalSchema(); $schema.addField( $field.fieldSchema ); } )?
+   ( field_def_list { $schema = $field_def_list.schema; } )?
 ;
 
 flatten_clause[LogicalExpressionPlan plan]
@@ -825,11 +816,13 @@ join_group_by_expr returns[LogicalExpres
 ;
 
 union_clause returns[String alias]
- : ^( UNION ONSCHEMA? rel_list )
+@init {
+    boolean onSchema = false;
+}
+ : ^( UNION ( ONSCHEMA { onSchema = true; } )? rel_list )
    {
-      // TODO: onschema support
       $alias = builder.buildUnionOp( $statement::alias,
-          $statement::parallel, $rel_list.aliasList );
+          $statement::parallel, $rel_list.aliasList, onSchema );
    }
 ;
 

Modified: pig/trunk/src/org/apache/pig/parser/QueryParser.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/QueryParser.g?rev=1060624&r1=1060623&r2=1060624&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/QueryParser.g (original)
+++ pig/trunk/src/org/apache/pig/parser/QueryParser.g Tue Jan 18 23:54:07 2011
@@ -44,8 +44,7 @@ tokens {
     MAP_VAL;
     BAG_VAL;
     KEY_VAL_PAIR;
-    TUPLE_DEF;
-    FIELD;
+    FIELD_DEF;
     NESTED_CMD_ASSI;
     NESTED_CMD;
     NESTED_PROJ;
@@ -193,22 +192,15 @@ load_clause : LOAD^ filename ( USING! fu
 filename : QUOTEDSTRING
 ;
 
-as_clause: AS^ tuple_def
+as_clause: AS^ ( field_def | field_def_list )
 ;
 
-tuple_def : tuple_def_full | tuple_def_simple
+field_def : IDENTIFIER ( COLON type )?
+     -> ^( FIELD_DEF IDENTIFIER type? )
 ;
 
-tuple_def_full : LEFT_PAREN field ( COMMA field )* RIGHT_PAREN
-            -> ^( TUPLE_DEF field+ )
-;
-
-tuple_def_simple : field
-         -> ^( TUPLE_DEF field )
-;
-
-field : IDENTIFIER ( COLON type )?
-     -> ^( FIELD IDENTIFIER type? )
+field_def_list : LEFT_PAREN field_def ( COMMA field_def )* RIGHT_PAREN
+     -> field_def+
 ;
 
 type : simple_type | tuple_type | bag_type | map_type
@@ -217,12 +209,12 @@ type : simple_type | tuple_type | bag_ty
 simple_type : INT | LONG | FLOAT | DOUBLE | CHARARRAY | BYTEARRAY
 ;
 
-tuple_type : TUPLE? tuple_def_full
-          -> ^( TUPLE_TYPE tuple_def_full )
+tuple_type : TUPLE? field_def_list
+          -> ^( TUPLE_TYPE field_def_list )
 ;
 
-bag_type : BAG? LEFT_CURLY tuple_def? RIGHT_CURLY
-        -> ^( BAG_TYPE tuple_def? )
+bag_type : BAG? LEFT_CURLY ( ( IDENTIFIER COLON )? tuple_type )? RIGHT_CURLY
+        -> ^( BAG_TYPE tuple_type? )
 ;
 
 map_type : MAP LEFT_BRACKET RIGHT_BRACKET
@@ -261,8 +253,9 @@ group_item : rel ( join_group_by_clause 
 rel : alias | LEFT_PAREN! op_clause RIGHT_PAREN!
 ;
 
-flatten_generated_item : flatten_clause as_clause?
-                       | ( expr | STAR ) ( AS! field )?
+flatten_generated_item : flatten_clause ( AS! ( field_def | field_def_list ) )?
+                       | expr ( AS! field_def )?
+                       | STAR ( AS! ( field_def | field_def_list ) )?
 ;
 
 flatten_clause : FLATTEN^ LEFT_PAREN! expr RIGHT_PAREN!

Modified: pig/trunk/test/org/apache/pig/parser/TestAstValidator.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/parser/TestAstValidator.java?rev=1060624&r1=1060623&r2=1060624&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/parser/TestAstValidator.java (original)
+++ pig/trunk/test/org/apache/pig/parser/TestAstValidator.java Tue Jan 18 23:54:07 2011
@@ -66,7 +66,7 @@ public class TestAstValidator {
             if( tree.getText().equals( "TUPLE_DEF" ) ) {
                 for ( int i = 0; i < tree.getChildCount(); i++ ) {
                     CommonTree child = (CommonTree)tree.getChild( i ); // FIELD node
-                    Assert.assertTrue( "FIELD".equals( child.getText() ) );
+                    Assert.assertTrue( "FIELD_DEF".equals( child.getText() ) );
                     CommonTree datatype = (CommonTree)child.getChild( 1 );
                     Assert.assertTrue( datatype != null );
                     String typeName = datatype.getText();

Modified: pig/trunk/test/org/apache/pig/parser/TestQueryParser.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/parser/TestQueryParser.java?rev=1060624&r1=1060623&r2=1060624&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/parser/TestQueryParser.java (original)
+++ pig/trunk/test/org/apache/pig/parser/TestQueryParser.java Tue Jan 18 23:54:07 2011
@@ -98,6 +98,23 @@ public class TestQueryParser {
     }
 
     @Test
+    public void testBagType() throws IOException, RecognitionException {
+        String query = "a = load '1.txt' as ( u : bag{}, v : bag{tuple(x, y)} );" +
+            "b = load '2.x' as ( t : {}, u : {(r,s)}, v : bag{ T : tuple( x, y ) }, w : bag{(z1, z2)} );" +
+            "c = load '3.x' as p : int;";
+        int errorCount = parse( query );
+        Assert.assertTrue( errorCount == 0 );
+    }
+
+    @Test
+    public void testFlatten() throws IOException, RecognitionException {
+        String query = "a = load '1.txt' as ( u, v, w : int );" +
+            "b = foreach a generate * as ( x, y, z ), flatten( u ) as ( r, s ), flatten( v ) as d, w + 5 as e:int;";
+        int errorCount = parse( query );
+        Assert.assertTrue( errorCount == 0 );
+    }
+
+    @Test
     public void testAST() throws IOException, RecognitionException  {
         CharStream input = new QueryParserFileStream( "test/org/apache/pig/parser/TestAST.pig" );
         QueryLexer lexer = new QueryLexer(input);