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 2010/12/14 17:41:51 UTC

svn commit: r1049153 - in /pig/trunk: ./ src/org/apache/pig/impl/io/ src/org/apache/pig/newplan/logical/expression/ src/org/apache/pig/parser/ test/org/apache/pig/parser/

Author: thejas
Date: Tue Dec 14 16:41:51 2010
New Revision: 1049153

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

Added:
    pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java
    pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g
    pig/trunk/test/org/apache/pig/parser/TestLogicalPlanGenerator.java
Modified:
    pig/trunk/build.xml
    pig/trunk/src/org/apache/pig/impl/io/FileSpec.java
    pig/trunk/src/org/apache/pig/newplan/logical/expression/UserFuncExpression.java
    pig/trunk/src/org/apache/pig/parser/AstValidator.g
    pig/trunk/src/org/apache/pig/parser/QueryLexer.g
    pig/trunk/src/org/apache/pig/parser/QueryParser.g
    pig/trunk/test/org/apache/pig/parser/ParserTestingUtils.java
    pig/trunk/test/org/apache/pig/parser/TestQueryParser.java

Modified: pig/trunk/build.xml
URL: http://svn.apache.org/viewvc/pig/trunk/build.xml?rev=1049153&r1=1049152&r2=1049153&view=diff
==============================================================================
--- pig/trunk/build.xml (original)
+++ pig/trunk/build.xml Tue Dec 14 16:41:51 2010
@@ -335,7 +335,7 @@
 	    description="generates tree parser class from an ANTLR grammar">
 	    <java classname="org.antlr.Tool"
 	      classpathref="classpath" fork="true">
-	      <arg line="-o ${src.gen.dir}/${grammar.package.dir} ${src.dir}/${grammar.package.dir}/AstValidator.g"/>
+	      <arg line="-o ${src.gen.dir}/${grammar.package.dir} ${src.dir}/${grammar.package.dir}/AstValidator.g ${src.dir}/${grammar.package.dir}/LogicalPlanGenerator.g"/>
 	    </java>
 	  </target>
 

Modified: pig/trunk/src/org/apache/pig/impl/io/FileSpec.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/impl/io/FileSpec.java?rev=1049153&r1=1049152&r2=1049153&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/impl/io/FileSpec.java (original)
+++ pig/trunk/src/org/apache/pig/impl/io/FileSpec.java Tue Dec 14 16:41:51 2010
@@ -20,6 +20,7 @@ package org.apache.pig.impl.io;
 import java.io.Serializable;
 
 import org.apache.pig.FuncSpec;
+import org.apache.pig.builtin.PigStorage;
 
 
 /**
@@ -35,7 +36,7 @@ public class FileSpec implements Seriali
     
     public FileSpec(String fileName, FuncSpec funcSpec){
         this.fileName = fileName;
-        this.funcSpec = funcSpec;
+        this.funcSpec = funcSpec != null ? funcSpec : new FuncSpec( PigStorage.class.getName() + "()" );
     }
     
     public String getFileName(){

Modified: pig/trunk/src/org/apache/pig/newplan/logical/expression/UserFuncExpression.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/newplan/logical/expression/UserFuncExpression.java?rev=1049153&r1=1049152&r2=1049153&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/newplan/logical/expression/UserFuncExpression.java (original)
+++ pig/trunk/src/org/apache/pig/newplan/logical/expression/UserFuncExpression.java Tue Dec 14 16:41:51 2010
@@ -49,6 +49,15 @@ public class UserFuncExpression extends 
         mFuncSpec = funcSpec;
         plan.add(this);
     }
+    
+    
+    public UserFuncExpression(OperatorPlan plan, FuncSpec funcSpec, List<LogicalExpression> args) {
+        this( plan, funcSpec );
+        
+        for( LogicalExpression arg : args ) {
+        	plan.connect( this, arg );
+        }
+    }
 
     public FuncSpec getFuncSpec() {
         return mFuncSpec;

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=1049153&r1=1049152&r2=1049153&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/AstValidator.g (original)
+++ pig/trunk/src/org/apache/pig/parser/AstValidator.g Tue Dec 14 16:41:51 2010
@@ -162,7 +162,7 @@ tuple_def_simple : ^( TUPLE_DEF field[ne
 field[Set<String> fieldNames] throws Exception
  : ^( FIELD IDENTIFIER { validateSchemaAliasName( fieldNames, $IDENTIFIER.text ); } )
 -> ^( FIELD IDENTIFIER BYTEARRAY )
- | ^( FIELD IDENTIFIER { validateSchemaAliasName( fieldNames, $IDENTIFIER.text ); } type ) 
+ | ^( FIELD IDENTIFIER { validateSchemaAliasName( fieldNames, $IDENTIFIER.text ); } type )
 ;
 
 type : simple_type | tuple_type | bag_type | map_type
@@ -184,7 +184,7 @@ func_clause : ^( FUNC func_name func_arg
             | ^( FUNC func_alias )
 ;
 
-func_name : eid+
+func_name : eid ( ( PERIOD | DOLLAR ) eid )*
 ;
 
 func_alias : IDENTIFIER
@@ -224,16 +224,12 @@ filter_clause : ^( FILTER rel cond )
 cond : ^( OR cond cond )
      | ^( AND cond cond )
      | ^( NOT cond )
-     | ^( NULL expr NOT )
-     | ^( FILTEROP expr expr )
-     | func_eval
      | ^( NULL expr NOT? )
+     | ^( rel_op expr expr )
+     | func_eval
 ;
 
-func_eval: ^( FUNC_EVAL func_name real_arg_list? )
-;
-
-real_arg_list : real_arg+
+func_eval: ^( FUNC_EVAL func_name real_arg* )
 ;
 
 real_arg : expr | STAR
@@ -247,17 +243,9 @@ expr : ^( PLUS expr expr )
      | ^( CAST_EXPR type expr )
      | const_expr
      | var_expr
-     | neg_expr
-;
-
-cast_expr : ^( CAST_EXPR type unary_expr )
-          | unary_expr
-;
-
-unary_expr : expr_eval | expr | neg_expr
-;
-
-expr_eval : const_expr | var_expr
+     | ^( NEG expr )
+     | ^( CAST_EXPR type expr )
+     | ^( EXPR_IN_PAREN expr )
 ;
 
 var_expr : projectable_expr ( dot_proj | pound_proj )*
@@ -275,9 +263,6 @@ pound_proj : ^( POUND ( QUOTEDSTRING | N
 bin_expr : ^( BIN_EXPR cond expr expr )
 ;
 
-neg_expr : ^( MINUS cast_expr )
-;
-
 limit_clause : ^( LIMIT rel ( INTEGER | LONGINTEGER ) )
 ;
 
@@ -358,16 +343,16 @@ nested_proj : ^( NESTED_PROJ col_ref col
 col_ref_list : col_ref+
 ;
 
-nested_filter : ^( FILTER ( IDENTIFIER | nested_proj | expr_eval ) cond )
+nested_filter : ^( FILTER ( IDENTIFIER | nested_proj | expr ) cond )
 ;
 
-nested_sort : ^( ORDER ( IDENTIFIER | nested_proj | expr_eval )  order_by_clause func_clause? )
+nested_sort : ^( ORDER ( IDENTIFIER | nested_proj | expr )  order_by_clause func_clause? )
 ;
 
-nested_distinct : ^( DISTINCT ( IDENTIFIER | nested_proj | expr_eval ) )
+nested_distinct : ^( DISTINCT ( IDENTIFIER | nested_proj | expr ) )
 ;
 
-nested_limit : ^( LIMIT ( IDENTIFIER | nested_proj | expr_eval ) INTEGER )
+nested_limit : ^( LIMIT ( IDENTIFIER | nested_proj | expr ) INTEGER )
 ;
 
 stream_clause : ^( STREAM rel ( EXECCOMMAND | IDENTIFIER ) as_clause? )
@@ -391,29 +376,32 @@ alias_col_ref : GROUP | IDENTIFIER
 dollar_col_ref : ^( DOLLAR INTEGER )
 ;
 
-const_expr : scalar | map | bag | tuple
+const_expr : literal
+;
+
+literal : scalar | map | bag | tuple
 ;
 
 scalar : INTEGER | LONGINEGER | FLOATNUMBER | DOUBLENUMBER | QUOTEDSTRING | NULL
 ;
 
-map : ^( MAP_VAL keyvalue+ )
+map : ^( MAP_VAL keyvalue* )
 ;
 
-keyvalue : ^( KEY_VAL_PAIR string_val const_expr )
+keyvalue : ^( KEY_VAL_PAIR map_key const_expr )
 ;
 
-string_val : QUOTEDSTRING | NULL
+map_key : QUOTEDSTRING | NULL
 ;
 
-bag : ^( BAG_VAL tuple+ )
+bag : ^( BAG_VAL tuple* )
 ;
 
-tuple : ^( TUPLE_VAL const_expr+ )
+tuple : ^( TUPLE_VAL literal* )
 ;
 
 // extended identifier, handling the keyword and identifier conflicts. Ugly but there is no other choice.
-eid : FILTEROP
+eid : rel_str_op
     | DEFINE
     | LOAD
     | FILTER
@@ -474,3 +462,40 @@ eid : FILTEROP
     | FULL
     | IDENTIFIER
 ;
+
+// relational operator
+rel_op : rel_op_eq
+       | rel_op_ne
+       | rel_op_gt
+       | rel_op_gte
+       | rel_op_lt
+       | rel_op_lte
+       | STR_OP_MATCHES
+;
+
+rel_op_eq : STR_OP_EQ | NUM_OP_EQ
+;
+
+rel_op_ne : STR_OP_NE | NUM_OP_NE
+;
+
+rel_op_gt : STR_OP_GT | NUM_OP_GT
+;
+
+rel_op_gte : STR_OP_GTE | NUM_OP_GTE
+;
+
+rel_op_lt : STR_OP_LT | NUM_OP_LT
+;
+
+rel_op_lte : STR_OP_LTE | NUM_OP_LTE
+;
+
+rel_str_op : STR_OP_EQ
+           | STR_OP_NE
+           | STR_OP_GT
+           | STR_OP_LT
+           | STR_OP_GTE
+           | STR_OP_LTE
+           | STR_OP_MATCHES
+;

Added: 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=1049153&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java (added)
+++ pig/trunk/src/org/apache/pig/parser/LogicalPlanBuilder.java Tue Dec 14 16:41:51 2010
@@ -0,0 +1,148 @@
+/*
+ * 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.parser;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.pig.FuncSpec;
+import org.apache.pig.data.BagFactory;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.pig.impl.io.FileSpec;
+import org.apache.pig.impl.util.StringUtils;
+import org.apache.pig.newplan.logical.expression.LogicalExpressionPlan;
+import org.apache.pig.newplan.logical.relational.LOFilter;
+import org.apache.pig.newplan.logical.relational.LOLimit;
+import org.apache.pig.newplan.logical.relational.LOLoad;
+import org.apache.pig.newplan.logical.relational.LOStore;
+import org.apache.pig.newplan.logical.relational.LogicalPlan;
+import org.apache.pig.newplan.logical.relational.LogicalRelationalOperator;
+import org.apache.pig.newplan.logical.relational.LogicalSchema;
+import org.apache.pig.newplan.Operator;
+
+public class LogicalPlanBuilder {
+    private LogicalPlan plan = new LogicalPlan();
+
+    private Map<String, Operator> operators = new HashMap<String, Operator>();
+    private Map<String, FuncSpec> functions = new HashMap<String, FuncSpec>();
+    
+    FuncSpec lookupFunction(String alias) {
+        return functions.get( alias );
+    }
+    
+    void defineFunction(String alias, FuncSpec fs) {
+        functions.put( alias, fs );
+    }
+    
+    LogicalPlan getPlan() {
+        return plan;
+    }
+
+    String buildFilterOp(String alias, Integer parallel, String inputAlias, LogicalExpressionPlan expr) {
+        LOFilter op = new LOFilter( plan, expr );
+        setAlias( op, alias );
+        setParallel( op, parallel );
+        plan.add( op );
+        Operator pred = operators.get( inputAlias );
+        if( pred == null ) {
+            // error out
+        }
+        plan.connect( pred, op );
+        operators.put( op.getAlias(), op );
+        return op.getAlias();
+    }
+
+    String buildLimitOp(String alias, Integer parallel, String inputAlias, long limit) {
+        LOLimit op = new LOLimit( plan, limit );
+        setAlias( op, alias );
+        setParallel( op, parallel );
+        plan.add( op );
+        Operator pred = operators.get( inputAlias );
+        if( pred == null ) {
+            // error out
+        }
+        plan.connect( pred, op );
+        operators.put( op.getAlias(), op );
+        return op.getAlias();
+    }
+    
+    String buildLoadOp(String alias, Integer parallel, String filename, FuncSpec funcSpec, LogicalSchema schema) {
+        FileSpec loader = new FileSpec( filename, funcSpec );
+        LOLoad op = new LOLoad( loader, schema, plan, null );
+        setAlias( op, alias );
+        setParallel( op, parallel );
+        plan.add( op );
+        operators.put( op.getAlias(), op );
+        return op.getAlias();
+    }
+    
+    String buildStoreOp(String alias, Integer parallel, String inputAlias, String filename, FuncSpec funcSpec) {
+        FileSpec fileSpec = new FileSpec( filename, funcSpec );
+        LOStore op = new LOStore( plan, fileSpec );
+        setAlias( op, alias );
+        setParallel( op, parallel );
+        Operator pred = operators.get( inputAlias );
+        if( pred == null ) {
+            // error out
+        }
+        plan.connect( pred, op );
+        plan.add( op );
+        operators.put( op.getAlias(), op );
+        return op.getAlias();
+    }
+    
+    static void setAlias(LogicalRelationalOperator op, String alias) {
+        if( alias != null ) {
+            op.setAlias( alias );
+        } else {
+            // TODO: generate an alias.
+        }
+    }
+    
+    static void setParallel(LogicalRelationalOperator op, Integer parallel) {
+        if( parallel != null )
+            op.setRequestedParallelism( parallel );
+    }
+    
+    static FuncSpec buildFuncSpec(String funcName, List<String> args) {
+        String[] argArray = new String[args.size()];
+        return new FuncSpec( funcName, args.toArray( argArray ) );
+    }
+    
+//    static FuncSpec buildFuncSpec(String funcName, List<LogicalExpression> args) {
+//    }
+    
+    static String unquote(String s) {
+        return StringUtils.unescapeInputString( s.substring(1, s.length() - 1 ) );
+    }
+    
+    static Tuple buildTuple(List<Object> objList) {
+        TupleFactory tf = TupleFactory.getInstance();
+        return tf.newTuple( objList );
+    }
+    
+    static DataBag createDataBag() {
+        BagFactory bagFactory = BagFactory.getInstance();
+        return bagFactory.newDefaultBag();
+    }
+    
+}

Added: 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=1049153&view=auto
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g (added)
+++ pig/trunk/src/org/apache/pig/parser/LogicalPlanGenerator.g Tue Dec 14 16:41:51 2010
@@ -0,0 +1,781 @@
+/*
+ * 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.
+ */
+ 
+/**
+ * Grammar file for Pig tree parser (for schema alias validation).
+ *
+ * NOTE: THIS FILE IS BASED ON QueryParser.g, SO IF YOU CHANGE THAT FILE, YOU WILL 
+ *       PROBABLY NEED TO MAKE CORRESPONDING CHANGES TO THIS FILE AS WELL.
+ */
+
+tree grammar LogicalPlanGenerator;
+
+options {
+    tokenVocab=QueryParser;
+    ASTLabelType=CommonTree;
+    output=AST;
+    backtrack=true;
+}
+
+@header {
+package org.apache.pig.parser;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.newplan.logical.expression.AddExpression;
+import org.apache.pig.newplan.logical.expression.AndExpression;
+import org.apache.pig.newplan.logical.expression.BinCondExpression;
+import org.apache.pig.newplan.logical.expression.CastExpression;
+import org.apache.pig.newplan.logical.expression.ConstantExpression;
+import org.apache.pig.newplan.logical.expression.DivideExpression;
+import org.apache.pig.newplan.logical.expression.EqualExpression;
+import org.apache.pig.newplan.logical.expression.GreaterThanEqualExpression;
+import org.apache.pig.newplan.logical.expression.GreaterThanExpression;
+import org.apache.pig.newplan.logical.expression.IsNullExpression;
+import org.apache.pig.newplan.logical.expression.LessThanEqualExpression;
+import org.apache.pig.newplan.logical.expression.LessThanExpression;
+import org.apache.pig.newplan.logical.expression.LogicalExpression;
+import org.apache.pig.newplan.logical.expression.LogicalExpressionPlan;
+import org.apache.pig.newplan.logical.expression.ModExpression;
+import org.apache.pig.newplan.logical.expression.MultiplyExpression;
+import org.apache.pig.newplan.logical.expression.NegativeExpression;
+import org.apache.pig.newplan.logical.expression.NotEqualExpression;
+import org.apache.pig.newplan.logical.expression.NotExpression;
+import org.apache.pig.newplan.logical.expression.OrExpression;
+import org.apache.pig.newplan.logical.expression.RegexExpression;
+import org.apache.pig.newplan.logical.expression.SubtractExpression;
+import org.apache.pig.newplan.logical.relational.LogicalPlan;
+import org.apache.pig.newplan.logical.relational.LogicalRelationalOperator;
+import org.apache.pig.newplan.logical.relational.LogicalSchema;
+import org.apache.pig.newplan.logical.relational.LogicalSchema.LogicalFieldSchema;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+}
+
+@members {
+private static Log log = LogFactory.getLog( LogicalPlanGenerator.class );
+
+private LogicalPlanBuilder builder = new LogicalPlanBuilder();
+
+public LogicalPlan getLogicalPlan() {
+    return builder.getPlan();
+}
+} // End of @members
+
+query : ^( QUERY statement* )
+;
+
+statement : general_statement | foreach_statement
+;
+
+general_statement 
+scope {
+    String alias;
+    Integer parallel;
+}
+@init {
+}
+: ^( STATEMENT ( alias { $general_statement::alias = $alias.ret; } )? 
+  op_clause ( INTEGER { $general_statement::parallel = Integer.parseInt( $INTEGER.text ); } )? )
+;
+
+// We need to handle foreach specifically because of the ending ';', which is not required 
+// if there is a nested block. This is ugly, but it gets the job done.
+foreach_statement : ^( STATEMENT alias? foreach_clause[$alias.text] )
+;
+
+alias returns[String ret]: IDENTIFIER { $ret = $IDENTIFIER.text; }
+;
+
+op_clause returns[String alias] : 
+            define_clause 
+          | load_clause { $alias = $load_clause.alias; }
+          | group_clause
+          | store_clause
+          | filter_clause
+          | distinct_clause
+          | limit_clause { $alias = $limit_clause.alias; }
+          | sample_clause
+          | order_clause
+          | partition_clause
+          | cross_clause
+          | joint_clause
+          | union_clause
+          | stream_clause
+          | mr_clause
+          | split_clause
+;
+
+define_clause 
+ : ^( DEFINE alias cmd ) 
+ | ^( DEFINE alias func_clause )
+   {
+       builder.defineFunction( $alias.ret, $func_clause.funcSpec );
+   }
+;
+
+cmd : ^( EXECCOMMAND ( ship_clause | cache_caluse | input_clause | output_clause | error_clause )* )
+;
+
+ship_clause : ^( SHIP path_list? )
+;
+
+path_list : QUOTEDSTRING+
+;
+
+cache_caluse : ^( CACHE path_list )
+;
+
+input_clause : ^( INPUT stream_cmd_list )
+;
+
+stream_cmd_list : stream_cmd+
+;
+
+stream_cmd : ^( STDIN func_clause? )
+           | ^( STDOUT func_clause? )
+           | ^( QUOTEDSTRING func_clause? )
+;
+
+output_clause : ^( OUTPUT stream_cmd_list )
+;
+
+error_clause : ^( ERROR error_cmd? )
+;
+
+error_cmd : ^( QUOTEDSTRING INTEGER? )
+;
+
+load_clause returns[String alias]
+ : ^( LOAD filename func_clause? as_clause? )
+  {
+      $alias = builder.buildLoadOp( $general_statement::alias,
+          $general_statement::parallel, $filename.filename, $func_clause.funcSpec, $as_clause.logicalSchema  );
+  }
+;
+
+filename returns[String filename]
+ : QUOTEDSTRING { $filename = builder.unquote( $QUOTEDSTRING.text ); }
+;
+
+as_clause returns[LogicalSchema logicalSchema]
+ : ^( AS tuple_def ) { $logicalSchema = $tuple_def.logicalSchema; }
+;
+
+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 )
+   {
+       $logicalSchema = new LogicalSchema();
+       $logicalSchema.addField( $field.fieldSchema );
+   }
+;
+
+field returns[LogicalFieldSchema fieldSchema]
+ : ^( FIELD IDENTIFIER type )
+   {
+       $fieldSchema = new LogicalFieldSchema( $IDENTIFIER.text, $type.logicalSchema, $type.datatype );
+   }
+;
+
+type returns[byte datatype, LogicalSchema logicalSchema]
+ : simple_type
+   {
+        $datatype = $simple_type.datatype;
+   }
+ | tuple_type
+   {
+       $datatype = DataType.TUPLE;
+       $logicalSchema = $tuple_type.logicalSchema;
+   }
+ | bag_type
+   {
+       $datatype = DataType.BAG;
+       $logicalSchema = $bag_type.logicalSchema;
+   }
+ | map_type
+   {
+       $datatype = DataType.MAP;
+   }
+;
+
+simple_type returns[byte datatype]
+ : INT { $datatype = DataType.INTEGER; }
+ | LONG { $datatype = DataType.LONG; }
+ | FLOAT { $datatype = DataType.FLOAT; }
+ | DOUBLE { $datatype = DataType.DOUBLE; }
+ | CHARARRAY { $datatype = DataType.CHARARRAY; }
+ | BYTEARRAY { $datatype = DataType.BYTEARRAY; }
+;
+
+tuple_type returns[LogicalSchema logicalSchema]
+ : ^( TUPLE_TYPE tuple_def_full )
+   { 
+       $logicalSchema = $tuple_def_full.logicalSchema;
+   }
+;
+
+bag_type returns[LogicalSchema logicalSchema]
+ : ^( BAG_TYPE tuple_def? )
+   { 
+       $logicalSchema = $tuple_def.logicalSchema;
+   }
+;
+
+map_type : MAP_TYPE
+;
+
+func_clause returns[FuncSpec funcSpec]
+ : ^( FUNC func_name func_args? )
+   { 
+       $funcSpec = builder.buildFuncSpec( $func_name.funcName, $func_args.args );
+   }
+ | ^( FUNC func_alias )
+   {
+       $funcSpec = builder.lookupFunction( $func_alias.alias );
+   }
+;
+
+func_name returns[String funcName]
+@init { StringBuilder buf = new StringBuilder(); } 
+ : p1 = eid { buf.append( $p1.id ); }
+      ( ( PERIOD { buf.append( $PERIOD.text ); } | DOLLAR { buf.append( $PERIOD.text ); } )
+      p2 = eid { buf.append( $p2.id ); } )*
+   {
+       $funcName = buf.toString();
+   }
+;
+
+func_alias returns[String alias]
+ : IDENTIFIER
+   {
+       $alias = $IDENTIFIER.text;
+   }
+;
+
+func_args returns[List<String> args]
+@init { $args = new ArrayList<String>(); }
+: ( QUOTEDSTRING { $args.add( builder.unquote( $QUOTEDSTRING.text ) ); } )+
+;
+
+group_clause : ^( GROUP group_item_list QUOTEDSTRING? )
+             | ^( COGROUP group_item_list QUOTEDSTRING? )
+;
+
+group_item_list : group_item+
+;
+
+group_item : rel ( ( flatten_generated_item_list ) | ALL | ANY ) ( INNER | OUTER )?
+;
+
+rel returns[String ret] : alias { $ret = $alias.ret; }
+                        | op_clause { $ret = $op_clause.alias; }
+;
+
+flatten_generated_item_list : flatten_generated_item+
+;
+
+flatten_generated_item : ( flatten_clause | expr[null] | STAR ) as_clause?
+;
+
+flatten_clause : ^( FLATTEN expr[null] )
+;
+
+store_clause returns[String alias]
+ : ^( STORE alias filename func_clause? )
+   {
+       $alias= builder.buildStoreOp( $general_statement::alias,
+          $general_statement::parallel, $alias.ret, $filename.filename, $func_clause.funcSpec );
+   }
+;
+
+filter_clause returns[String alias]
+@init { LogicalExpressionPlan exprPlan = new LogicalExpressionPlan(); }
+ : ^( FILTER rel cond[exprPlan] )
+   {
+       $alias = builder.buildFilterOp( $general_statement::alias,
+          $general_statement::parallel, $rel.ret, exprPlan );
+   }
+;
+
+cond[LogicalExpressionPlan exprPlan] returns[LogicalExpression expr]
+ : ^( OR left = cond[exprPlan] right = cond[exprPlan] )
+   {
+       $expr = new OrExpression( $exprPlan, $left.expr, $right.expr );
+   }
+ | ^( AND left = cond[exprPlan] right = cond[exprPlan] )
+   {
+       $expr = new AndExpression( $exprPlan, $left.expr, $right.expr );
+   }
+ | ^( NOT c = cond[exprPlan] )
+   {
+       $expr = new NotExpression( $exprPlan, $c.expr );
+   }
+ | ^( NULL expr[$exprPlan] NOT? )
+   {
+       $expr = new IsNullExpression( $exprPlan, $expr.expr );
+       if( $NOT != null )
+           $expr = new NotExpression( $exprPlan, $expr );
+   }
+ | ^( rel_op_eq e1 = expr[$exprPlan] e2 = expr[$exprPlan] )
+   {
+       $expr = new EqualExpression( $exprPlan, $e1.expr, $e2.expr );
+   } 
+ | ^( rel_op_ne e1 = expr[$exprPlan] e2 = expr[$exprPlan] )
+   {
+       $expr = new NotEqualExpression( $exprPlan, $e1.expr, $e2.expr );
+   } 
+ | ^( rel_op_lt e1 = expr[$exprPlan] e2 = expr[$exprPlan] )
+   {
+       $expr = new LessThanExpression( $exprPlan, $e1.expr, $e2.expr );
+   } 
+ | ^( rel_op_lte e1 = expr[$exprPlan] e2 = expr[$exprPlan] )
+   {
+       $expr = new LessThanEqualExpression( $exprPlan, $e1.expr, $e2.expr );
+   }
+ | ^( rel_op_gt e1 = expr[$exprPlan] e2 = expr[$exprPlan] )
+   {
+       $expr = new GreaterThanExpression( $exprPlan, $e1.expr, $e2.expr );
+   } 
+ | ^( rel_op_gte e1 = expr[$exprPlan] e2 = expr[$exprPlan] )
+   {
+       $expr = new GreaterThanEqualExpression( $exprPlan, $e1.expr, $e2.expr );
+   }
+ | ^( STR_OP_MATCHES e1 = expr[$exprPlan] e2 = expr[$exprPlan] )
+   {
+       $expr = new RegexExpression( $exprPlan, $e1.expr, $e2.expr );
+   }
+ | func_eval[$exprPlan]
+;
+
+func_eval[LogicalExpressionPlan plan] returns[LogicalExpression expr]
+@init { List<LogicalExpression> args = new ArrayList<LogicalExpression>(); }
+ : ^( FUNC_EVAL func_name ( real_arg[$plan] { args.add( $real_arg.expr ); } )* )
+   {
+       //TODO: FuncSpec funcSpec = builder.buildFuncSpec( $func_name.funcName, $func_args.args );
+   }
+;
+
+real_arg [LogicalExpressionPlan plan] returns[LogicalExpression expr]
+ : e = expr[$plan] { $expr = $e.expr; }
+ | STAR
+;
+
+expr[LogicalExpressionPlan plan] returns[LogicalExpression expr]
+ : ^( PLUS left = expr[$plan] right = expr[$plan] )
+   {
+       $expr = new AddExpression( $plan, $left.expr, $right.expr );
+   }
+ | ^( MINUS left = expr[$plan] right = expr[$plan] )
+   {
+       $expr = new SubtractExpression( $plan, $left.expr, $right.expr );
+   }
+ | ^( STAR left = expr[$plan] right = expr[$plan] )
+   {
+       $expr = new MultiplyExpression( $plan, $left.expr, $right.expr );
+   }
+ | ^( DIV left = expr[$plan] right = expr[$plan] )
+   {
+       $expr = new DivideExpression( $plan, $left.expr, $right.expr );
+   }
+ | ^( PERCENT left = expr[$plan] right = expr[$plan] )
+   {
+       $expr = new ModExpression( $plan, $left.expr, $right.expr );
+   }
+ | const_expr[$plan]
+   {
+       $expr = new ConstantExpression( $plan, $const_expr.expr, null );
+   }
+ | var_expr[$plan]
+   {
+       $expr = $var_expr.expr; 
+   }
+ | ^( NEG e = expr[$plan] )
+   {
+       $expr = new NegativeExpression( $plan, $e.expr );
+   }
+ | ^( CAST_EXPR type e = expr[$plan] ) // cast expr
+   {
+       $expr = new CastExpression( $plan, $e.expr, 
+           new LogicalFieldSchema( null , $type.logicalSchema, $type.datatype ) );
+   }
+ | ^( EXPR_IN_PAREN e = expr[$plan] ) // unary expr
+   {
+       $expr = $e.expr;
+   }
+;
+
+var_expr[LogicalExpressionPlan plan] returns[LogicalExpression expr]
+ : projectable_expr[$plan] ( dot_proj | pound_proj )*
+;
+
+projectable_expr[LogicalExpressionPlan plan] returns[LogicalExpression expr]
+ : func_eval[$plan] { $expr = $func_eval.expr; }
+ | col_ref[$plan] { }
+ | bin_expr[$plan] { $expr = $bin_expr.expr; }
+;
+
+dot_proj : ^( PERIOD col_ref[null]+ )
+;
+
+pound_proj : ^( POUND ( QUOTEDSTRING | NULL ) )
+;
+
+bin_expr[LogicalExpressionPlan plan] returns[LogicalExpression expr]
+ : ^( BIN_EXPR cond[$plan] e1 = expr[$plan] e2 = expr[$plan] )
+   {
+       $expr = new BinCondExpression( $plan, $cond.expr, $e1.expr, $e2.expr );
+   }
+;
+
+limit_clause returns[String alias] : ^( LIMIT rel INTEGER  )
+{
+$alias = builder.buildLimitOp( $general_statement::alias,
+    $general_statement::parallel, $rel.ret, Long.valueOf( $INTEGER.text ) );
+}
+             | ^( LIMIT rel LONGINTEGER )
+;
+
+sample_clause : ^( SAMPLE rel DOUBLENUMBER )
+;
+
+order_clause : ^( ORDER rel order_by_clause func_clause? )
+;
+
+order_by_clause : STAR ( ASC | DESC )?
+                | order_col_list
+;
+
+order_col_list : order_col+
+;
+
+order_col : col_ref[null] ( ASC | DESC )?
+;
+
+distinct_clause : ^( DISTINCT rel partition_clause? )
+;
+
+partition_clause : ^( PARTITION func_name )
+;
+
+cross_clause : ^( CROSS rel_list partition_clause? )
+;
+
+rel_list : rel+
+;
+
+joint_clause : ^( JOIN join_sub_clause QUOTEDSTRING? partition_clause? )
+;
+
+join_sub_clause : join_item ( LEFT | RIGHT | FULL ) OUTER? join_item
+                | join_item_list
+;
+
+join_item_list : join_item ( join_item )+
+;
+
+join_item : rel flatten_generated_item_list
+;
+
+union_clause : ^( UNION ONSCHEMA? rel_list )
+;
+
+foreach_clause[String alias] returns[LogicalRelationalOperator op] : ^( FOREACH rel nested_plan )
+;
+
+nested_plan : ^( NESTED_PLAN foreach_blk )
+            | ^( NESTED_PLAN generate_clause )
+;
+
+foreach_blk : nested_command_list generate_clause
+;
+
+generate_clause : ^( GENERATE flatten_generated_item+ )
+;
+
+nested_command_list : nested_command*
+;
+
+nested_command : ^( NESTED_CMD IDENTIFIER expr[null]  )
+               | ^( NESTED_CMD IDENTIFIER nested_op )
+;
+
+nested_op : nested_proj
+          | nested_filter
+          | nested_sort
+          | nested_distinct
+          | nested_limit
+;
+
+nested_proj : ^( NESTED_PROJ col_ref[null] col_ref_list )
+;
+
+col_ref_list : col_ref[null]+
+;
+
+nested_filter : ^( FILTER ( IDENTIFIER | nested_proj | expr[null] ) cond[null] )
+;
+
+nested_sort : ^( ORDER ( IDENTIFIER | nested_proj | expr[null] )  order_by_clause func_clause? )
+;
+
+nested_distinct : ^( DISTINCT ( IDENTIFIER | nested_proj | expr[null] ) )
+;
+
+nested_limit : ^( LIMIT ( IDENTIFIER | nested_proj | expr[null] ) INTEGER )
+;
+
+stream_clause : ^( STREAM rel ( EXECCOMMAND | IDENTIFIER ) as_clause? )
+;
+
+mr_clause : ^( MAPREDUCE QUOTEDSTRING path_list? store_clause load_clause EXECCOMMAND? )
+;
+
+split_clause : ^( SPLIT rel split_branch+ )
+;
+
+split_branch : ^( SPLIT_BRANCH IDENTIFIER cond[null] )
+;
+
+col_ref[LogicalExpressionPlan plan] returns[LogicalExpression expr]
+ : alias_col_ref[$plan] { $expr = $alias_col_ref.expr; }
+ | dollar_col_ref[$plan] { $expr = $dollar_col_ref.expr; }
+;
+
+alias_col_ref[LogicalExpressionPlan plan] returns[LogicalExpression expr]
+ : GROUP 
+ | IDENTIFIER
+;
+
+dollar_col_ref[LogicalExpressionPlan plan] returns[LogicalExpression expr]
+ : ^( DOLLAR INTEGER )
+;
+
+const_expr[LogicalExpressionPlan plan] returns[LogicalExpression expr]
+ : literal
+   {
+       $expr = new ConstantExpression( $plan, $literal.value,
+           new LogicalFieldSchema( null , null, $literal.type ) );
+   }
+;
+
+literal returns[Object value, byte type]
+ : scalar
+   {
+       $value = $scalar.value;
+       $type = $scalar.type;
+   }
+ | map
+   {
+       $value = $map.value;
+       $type = DataType.MAP;
+   }
+ | bag
+   {
+       $value = $bag.value;
+       $type = DataType.BAG;
+   }
+ | tuple
+   {
+       $value = $tuple.value;
+       $type = DataType.TUPLE;
+   }
+;
+
+scalar returns[Object value, byte type]
+ : INTEGER
+   { 
+       $type = DataType.INTEGER;
+       $value = Integer.valueOf( $INTEGER.text );
+   }
+ | LONGINEGER 
+   { 
+       $type = DataType.LONG;
+       String num = $LONGINEGER.text.substring( 0, $LONGINEGER.text.length() - 1 );
+       $value = Long.valueOf( $LONGINEGER.text );
+   }
+ | FLOATNUMBER 
+   { 
+       $type = DataType.FLOAT;
+       $value = Float.valueOf( $FLOATNUMBER.text );
+   }
+ | DOUBLENUMBER 
+   { 
+       $type = DataType.DOUBLE;
+       $value = Double.valueOf( $DOUBLENUMBER.text );
+   }
+ | QUOTEDSTRING 
+   { 
+       $type = DataType.CHARARRAY;
+       $value = builder.unquote( $QUOTEDSTRING.text );
+   }
+ | NULL
+   { 
+       $type = DataType.NULL;
+   }
+;
+
+map returns[Object value]
+@init { Map<String, Object> kvs = new HashMap<String, Object>(); }
+ : ^( MAP_VAL ( keyvalue { kvs.put( $keyvalue.key, $keyvalue.value ); } )* )
+   {
+       $value = kvs;
+   }
+;
+
+keyvalue returns[String key, Object value]
+ : ^( KEY_VAL_PAIR map_key literal )
+   {
+       $key = $map_key.value;
+       $value = $literal.value;
+   }
+;
+
+map_key returns[String value]
+ : QUOTEDSTRING { $value = builder.unquote( $QUOTEDSTRING.text ); }
+ | NULL { $value = null; }
+;
+
+bag returns[Object value]
+@init { DataBag dataBag = builder.createDataBag(); }
+ : ^( BAG_VAL ( tuple { dataBag.add( $tuple.value ); } )* )
+   {
+       $value = dataBag;
+   }
+;
+
+tuple returns[Tuple value]
+@init { List<Object> objList = new ArrayList<Object>(); }
+ : ^( TUPLE_VAL ( literal { objList.add( $literal.value ); } )* )
+   {
+       $value = builder.buildTuple( objList );
+   }
+;
+
+// extended identifier, handling the keyword and identifier conflicts. Ugly but there is no other choice.
+eid returns[String id] : rel_str_op { $id = $rel_str_op.id; }
+    | DEFINE { $id = $DEFINE.text; }
+    | LOAD { $id = $LOAD.text; }
+    | FILTER { $id = $FILTER.text; }
+    | FOREACH { $id = $FOREACH.text; }
+    | MATCHES { $id = $MATCHES.text; }
+    | ORDER { $id = $ORDER.text; }
+    | DISTINCT { $id = $DISTINCT.text; }
+    | COGROUP { $id = $COGROUP.text; }
+    | JOIN { $id = $JOIN.text; }
+    | CROSS { $id = $CROSS.text; }
+    | UNION { $id = $UNION.text; }
+    | SPLIT { $id = $SPLIT.text; }
+    | INTO { $id = $INTO.text; }
+    | IF { $id = $IF.text; }
+    | ALL { $id = $ALL.text; }
+    | AS { $id = $AS.text; }
+    | BY { $id = $BY.text; }
+    | USING { $id = $USING.text; }
+    | INNER { $id = $INNER.text; }
+    | OUTER { $id = $OUTER.text; }
+    | PARALLEL { $id = $PARALLEL.text; }
+    | PARTITION { $id = $PARTITION.text; }
+    | GROUP { $id = $GROUP.text; }
+    | AND { $id = $AND.text; }
+    | OR { $id = $OR.text; }
+    | NOT { $id = $NOT.text; }
+    | GENERATE { $id = $GENERATE.text; }
+    | FLATTEN { $id = $FLATTEN.text; }
+    | EVAL { $id = $EVAL.text; }
+    | ASC { $id = $ASC.text; }
+    | DESC { $id = $DESC.text; }
+    | INT { $id = $INT.text; }
+    | LONG { $id = $LONG.text; }
+    | FLOAT { $id = $FLOAT.text; }
+    | DOUBLE { $id = $DOUBLE.text; }
+    | CHARARRAY { $id = $CHARARRAY.text; }
+    | BYTEARRAY { $id = $BYTEARRAY.text; }
+    | BAG { $id = $BAG.text; }
+    | TUPLE { $id = $TUPLE.text; }
+    | MAP { $id = $MAP.text; }
+    | IS { $id = $IS.text; }
+    | NULL { $id = $NULL.text; }
+    | STREAM { $id = $STREAM.text; }
+    | THROUGH { $id = $THROUGH.text; }
+    | STORE { $id = $STORE.text; }
+    | MAPREDUCE { $id = $MAPREDUCE.text; }
+    | SHIP { $id = $SHIP.text; }
+    | CACHE { $id = $CACHE.text; }
+    | INPUT { $id = $INPUT.text; }
+    | OUTPUT { $id = $OUTPUT.text; }
+    | ERROR { $id = $ERROR.text; }
+    | STDIN { $id = $STDIN.text; }
+    | STDOUT { $id = $STDOUT.text; }
+    | LIMIT { $id = $LIMIT.text; }
+    | SAMPLE { $id = $SAMPLE.text; }
+    | LEFT { $id = $LEFT.text; }
+    | RIGHT { $id = $RIGHT.text; }
+    | FULL { $id = $FULL.text; }
+    | IDENTIFIER { $id = $IDENTIFIER.text; }
+;
+
+// relational operator
+rel_op : rel_op_eq
+       | rel_op_ne
+       | rel_op_gt
+       | rel_op_gte
+       | rel_op_lt
+       | rel_op_lte
+       | STR_OP_MATCHES
+;
+
+rel_op_eq : STR_OP_EQ | NUM_OP_EQ
+;
+
+rel_op_ne : STR_OP_NE | NUM_OP_NE
+;
+
+rel_op_gt : STR_OP_GT | NUM_OP_GT
+;
+
+rel_op_gte : STR_OP_GTE | NUM_OP_GTE
+;
+
+rel_op_lt : STR_OP_LT | NUM_OP_LT
+;
+
+rel_op_lte : STR_OP_LTE | NUM_OP_LTE
+;
+
+rel_str_op returns[String id]
+ : STR_OP_EQ { $id = $STR_OP_EQ.text; }
+ | STR_OP_NE { $id = $STR_OP_NE.text; }
+ | STR_OP_GT { $id = $STR_OP_GT.text; }
+ | STR_OP_LT { $id = $STR_OP_LT.text; }
+ | STR_OP_GTE { $id = $STR_OP_GTE.text; }
+ | STR_OP_LTE { $id = $STR_OP_LTE.text; }
+ | STR_OP_MATCHES { $id = $STR_OP_MATCHES.text; }
+;
+

Modified: pig/trunk/src/org/apache/pig/parser/QueryLexer.g
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/QueryLexer.g?rev=1049153&r1=1049152&r2=1049153&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/QueryLexer.g (original)
+++ pig/trunk/src/org/apache/pig/parser/QueryLexer.g Tue Dec 14 16:41:51 2010
@@ -200,15 +200,45 @@ RIGHT : 'RIGHT'
 FULL : 'FULL'
 ;
 
-fragment STRFILTEROP : 'EQ' | 'GT' | 'LT' | 'GTE' | 'LTE' | 'NEQ' | 'MATCHES'
+STR_OP_EQ : 'EQ'
 ;
-    
-fragment NUMFILTEROP : '==' | '<' | '<=' | '>' | '>=' | '!='
+
+STR_OP_GT : 'GT'
+;
+
+STR_OP_LT : 'LT'
+;
+
+STR_OP_GTE : 'GTE'
+;
+
+STR_OP_LTE : 'LTE'
+;
+
+STR_OP_NE : 'NEQ'
+;
+
+STR_OP_MATCHES : 'MATCHES'
 ;
     
-FILTEROP : STRFILTEROP | NUMFILTEROP
+NUM_OP_EQ : '=='
+;
+
+NUM_OP_LT : '<'
 ;
 
+NUM_OP_LTE : '<='
+;
+
+NUM_OP_GT : '>'
+;
+
+NUM_OP_GTE : '>=' 
+;
+
+NUM_OP_NE : '!='
+;
+    
 fragment DIGIT : '0'..'9'
 ;
 

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=1049153&r1=1049152&r2=1049153&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/QueryParser.g (original)
+++ pig/trunk/src/org/apache/pig/parser/QueryParser.g Tue Dec 14 16:41:51 2010
@@ -19,7 +19,7 @@
 /**
  * Parser file for Pig Parser
  *
- * NOTE: THIS FILE IS THE BASE FOR A FEW TREE PARSER FILES, such as DefaultDataTypeInserter.g, 
+ * NOTE: THIS FILE IS THE BASE FOR A FEW TREE PARSER FILES, such as AstValidator.g, 
  *       SO IF YOU CHANGE THIS FILE, YOU WILL PROBABLY NEED TO MAKE CORRESPONDING CHANGES TO 
  *       THOSE FILES AS WELL.
  */
@@ -52,6 +52,8 @@ tokens {
     MAP_TYPE;
     TUPLE_TYPE;
     BAG_TYPE;
+    NEG;
+    EXPR_IN_PAREN;
 }
 
 @header {
@@ -226,8 +228,7 @@ func_clause : func_name LEFT_PAREN func_
            -> ^( FUNC func_alias )
 ;
 
-func_name : eid ( PERIOD eid )*
-         -> eid+
+func_name : eid ( ( PERIOD | DOLLAR ) eid )*
 ;
 
 func_alias : IDENTIFIER
@@ -277,7 +278,7 @@ and_cond : unary_cond ( AND^ unary_cond 
 ;
 
 unary_cond : LEFT_PAREN! cond RIGHT_PAREN!
-           | expr FILTEROP^ expr
+           | expr rel_op^ expr
            | func_eval
            | null_check_cond
            | not_cond
@@ -314,7 +315,10 @@ cast_expr : ( LEFT_PAREN type RIGHT_PARE
           | unary_expr
 ;
 
-unary_expr : expr_eval | ( LEFT_PAREN! expr RIGHT_PAREN! )  | neg_expr
+unary_expr : expr_eval 
+           | LEFT_PAREN expr RIGHT_PAREN
+          -> ^( EXPR_IN_PAREN expr )
+           | neg_expr
 ;
 
 expr_eval : const_expr | var_expr
@@ -337,7 +341,8 @@ bin_expr : LEFT_PAREN cond QMARK exp1 = 
         -> ^( BIN_EXPR cond $exp1 $exp2 )
 ;
 
-neg_expr : MINUS^ cast_expr
+neg_expr : MINUS cast_expr
+        -> ^( NEG cast_expr )
 ;
 
 limit_clause : LIMIT^ rel ( INTEGER | LONGINTEGER )
@@ -467,33 +472,43 @@ alias_col_ref : GROUP | IDENTIFIER
 dollar_col_ref : DOLLAR^ INTEGER
 ;
 
-const_expr : scalar | map | bag | tuple
+const_expr : literal
 ;
 
+literal : scalar | map | bag | tuple
+;
+
+
 scalar : INTEGER | LONGINEGER | FLOATNUMBER | DOUBLENUMBER | QUOTEDSTRING | NULL
 ;
 
 map : LEFT_BRACKET ( keyvalue ( COMMA keyvalue )* )? RIGHT_BRACKET
    -> ^( MAP_VAL keyvalue+ )
+    | LEFT_BRACKET RIGHT_BRACKET
+   -> ^( MAP_VAL )
 ;
 
-keyvalue : string_val POUND const_expr
-        -> ^( KEY_VAL_PAIR string_val const_expr )
+keyvalue : map_key POUND const_expr
+        -> ^( KEY_VAL_PAIR map_key const_expr )
 ;
 
-string_val : QUOTEDSTRING | NULL
+map_key : QUOTEDSTRING | NULL
 ;
 
 bag : LEFT_CURLY ( tuple ( COMMA tuple )* )? RIGHT_CURLY
    -> ^( BAG_VAL tuple+ )
+    | LEFT_CURLY RIGHT_CURLY
+   -> ^( BAG_VAL )
 ;
 
-tuple : LEFT_PAREN ( const_expr ( COMMA const_expr )* )? RIGHT_PAREN
-     -> ^( TUPLE_VAL const_expr+ )
+tuple : LEFT_PAREN ( literal ( COMMA const_expr )* )? RIGHT_PAREN
+     -> ^( TUPLE_VAL literal+ )
+      | LEFT_PAREN RIGHT_PAREN
+     -> ^( TUPLE_VAL )
 ;
 
 // extended identifier, handling the keyword and identifier conflicts. Ugly but there is no other choice.
-eid : FILTEROP
+eid : rel_str_op
     | DEFINE
     | LOAD
     | FILTER
@@ -554,3 +569,41 @@ eid : FILTEROP
     | FULL
     | IDENTIFIER
 ;
+
+// relational operator
+rel_op : rel_op_eq
+       | rel_op_ne
+       | rel_op_gt
+       | rel_op_gte
+       | rel_op_lt
+       | rel_op_lte
+       | STR_OP_MATCHES
+;
+
+rel_op_eq : STR_OP_EQ | NUM_OP_EQ
+;
+
+rel_op_ne : STR_OP_NE | NUM_OP_NE
+;
+
+rel_op_gt : STR_OP_GT | NUM_OP_GT
+;
+
+rel_op_gte : STR_OP_GTE | NUM_OP_GTE
+;
+
+rel_op_lt : STR_OP_LT | NUM_OP_LT
+;
+
+rel_op_lte : STR_OP_LTE | NUM_OP_LTE
+;
+
+rel_str_op : STR_OP_EQ
+           | STR_OP_NE
+           | STR_OP_GT
+           | STR_OP_LT
+           | STR_OP_GTE
+           | STR_OP_LTE
+           | STR_OP_MATCHES
+;
+

Modified: pig/trunk/test/org/apache/pig/parser/ParserTestingUtils.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/parser/ParserTestingUtils.java?rev=1049153&r1=1049152&r2=1049153&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/parser/ParserTestingUtils.java (original)
+++ pig/trunk/test/org/apache/pig/parser/ParserTestingUtils.java Tue Dec 14 16:41:51 2010
@@ -26,6 +26,7 @@ import org.antlr.runtime.RecognitionExce
 import org.antlr.runtime.tree.CommonTree;
 import org.antlr.runtime.tree.CommonTreeNodeStream;
 import org.antlr.runtime.tree.Tree;
+import org.apache.pig.newplan.logical.relational.LogicalPlan;
 
 public class ParserTestingUtils {
     public static CommonTokenStream tokenize(String query) throws IOException, ParsingFailureException {
@@ -65,4 +66,20 @@ public class ParserTestingUtils {
         return newAst;
     }
     
+    public static LogicalPlan generateLogicalPlan(String query)
+    throws RecognitionException, ParsingFailureException, IOException {
+        Tree ast = validateAst( query );
+        
+        LogicalPlanGenerator walker = new LogicalPlanGenerator( new CommonTreeNodeStream( ast ) );
+        walker.query();
+        
+        if( 0 < walker.getNumberOfSyntaxErrors() ) 
+            throw new ParsingFailureException( LogicalPlanGenerator.class );
+        
+        LogicalPlan plan = walker.getLogicalPlan();
+        System.out.println( "Generated logical plan: " + plan.toString() );
+        
+        return plan;
+    }
+
 }

Added: pig/trunk/test/org/apache/pig/parser/TestLogicalPlanGenerator.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/parser/TestLogicalPlanGenerator.java?rev=1049153&view=auto
==============================================================================
--- pig/trunk/test/org/apache/pig/parser/TestLogicalPlanGenerator.java (added)
+++ pig/trunk/test/org/apache/pig/parser/TestLogicalPlanGenerator.java Tue Dec 14 16:41:51 2010
@@ -0,0 +1,44 @@
+/*
+ * 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.parser;
+
+import java.io.IOException;
+
+import junit.framework.Assert;
+
+import org.antlr.runtime.RecognitionException;
+import org.junit.Test;
+
+public class TestLogicalPlanGenerator {
+    @Test
+    public void test1() throws RecognitionException, IOException, ParsingFailureException {
+        ParserTestingUtils.generateLogicalPlan( "A = load 'x' using org.apache.pig.TextLoader( 'a', 'b' ) as ( u:int, v:long, w:bytearray); B = limit A 100; C = filter B by 2 > 1; D = store C into 'output';" );
+    }
+
+    @Test
+    public void testNegative2() throws RecognitionException, IOException {
+        try {
+            ParserTestingUtils.validateAst( "A = load 'x' as ( u:int, v:long, w:tuple( w:long, u:chararray, w:bytearray) );" );
+        } catch(ParsingFailureException ex) {
+            Assert.assertEquals( AstValidator.class, ex.getParsingClass() );
+            return;
+        }
+        Assert.assertTrue( false ); // should never come here.
+    }
+}

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=1049153&r1=1049152&r2=1049153&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/parser/TestQueryParser.java (original)
+++ pig/trunk/test/org/apache/pig/parser/TestQueryParser.java Tue Dec 14 16:41:51 2010
@@ -91,6 +91,13 @@ public class TestQueryParser {
     }
 
     @Test
+    public void test3() throws IOException, RecognitionException {
+        int errorCount = parse( 
+ "a = load '1.txt' as (a0); b = foreach a generate flatten((bag{T:tuple(m:map[])})a0) as b0:map[];c = foreach b generate (long)b0#'key1';" );
+        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);