You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by da...@apache.org on 2010/08/05 21:38:54 UTC

svn commit: r982747 [2/4] - in /hadoop/pig/trunk: src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/newplan/ src/org/apache/pig/newplan/logical/ src/org/apache/pig/newplan/logical/expression/ src/org/apache/pig/n...

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/ExpToPhyTranslationVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/ExpToPhyTranslationVisitor.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/ExpToPhyTranslationVisitor.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/ExpToPhyTranslationVisitor.java Thu Aug  5 19:38:51 2010
@@ -59,7 +59,9 @@ import org.apache.pig.backend.hadoop.exe
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.Subtract;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.data.DataType;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.plan.NodeIdGenerator;
 import org.apache.pig.impl.plan.OperatorKey;
 import org.apache.pig.impl.plan.PlanException;
@@ -77,7 +79,7 @@ public class ExpToPhyTranslationVisitor 
     // This value points to the current LogicalRelationalOperator we are working on
     protected LogicalRelationalOperator currentOp;
     
-    public ExpToPhyTranslationVisitor(OperatorPlan plan, LogicalRelationalOperator op, PhysicalPlan phyPlan, Map<Operator, PhysicalOperator> map) {
+    public ExpToPhyTranslationVisitor(OperatorPlan plan, LogicalRelationalOperator op, PhysicalPlan phyPlan, Map<Operator, PhysicalOperator> map) throws FrontendException {
         super(plan, new DependencyOrderWalker(plan));
         currentOp = op;
         logToPhyMap = map;
@@ -85,7 +87,7 @@ public class ExpToPhyTranslationVisitor 
         currentPlans = new Stack<PhysicalPlan>();
     }
     
-    public ExpToPhyTranslationVisitor(OperatorPlan plan, PlanWalker walker, LogicalRelationalOperator op, PhysicalPlan phyPlan, Map<Operator, PhysicalOperator> map) {
+    public ExpToPhyTranslationVisitor(OperatorPlan plan, PlanWalker walker, LogicalRelationalOperator op, PhysicalPlan phyPlan, Map<Operator, PhysicalOperator> map) throws FrontendException {
         super(plan, walker);
         currentOp = op;
         logToPhyMap = map;
@@ -112,7 +114,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     private void attachBinaryComparisonOperator( BinaryExpression op, 
-            BinaryComparisonOperator exprOp ) throws IOException {
+            BinaryComparisonOperator exprOp ) throws FrontendException {
         // We dont have aliases in ExpressionOperators
         // exprOp.setAlias(op.getAlias());
         
@@ -142,7 +144,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     private void attachBinaryExpressionOperator( BinaryExpression op, 
-            BinaryExpressionOperator exprOp ) throws IOException {
+            BinaryExpressionOperator exprOp ) throws FrontendException {
         // We dont have aliases in ExpressionOperators
         // exprOp.setAlias(op.getAlias());
         
@@ -172,7 +174,7 @@ public class ExpToPhyTranslationVisitor 
     }
 
     @Override
-    public void visit( AndExpression op ) throws IOException {
+    public void visit( AndExpression op ) throws FrontendException {
         
 //        System.err.println("Entering And");
         BinaryComparisonOperator exprOp = new POAnd(new OperatorKey(DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));
@@ -181,7 +183,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit( OrExpression op ) throws IOException {
+    public void visit( OrExpression op ) throws FrontendException {
         
 //        System.err.println("Entering Or");
         BinaryComparisonOperator exprOp = new POOr(new OperatorKey(DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));
@@ -190,7 +192,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit( EqualExpression op ) throws IOException {
+    public void visit( EqualExpression op ) throws FrontendException {
         
         BinaryComparisonOperator exprOp = new EqualToExpr(new OperatorKey(
                 DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));
@@ -199,7 +201,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit( NotEqualExpression op ) throws IOException {
+    public void visit( NotEqualExpression op ) throws FrontendException {
         
         BinaryComparisonOperator exprOp = new NotEqualToExpr(new OperatorKey(
                 DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));
@@ -208,7 +210,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit( GreaterThanExpression op ) throws IOException {
+    public void visit( GreaterThanExpression op ) throws FrontendException {
         
         BinaryComparisonOperator exprOp = new GreaterThanExpr(new OperatorKey(
                 DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));
@@ -217,7 +219,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit( GreaterThanEqualExpression op ) throws IOException {
+    public void visit( GreaterThanEqualExpression op ) throws FrontendException {
         
         BinaryComparisonOperator exprOp = new GTOrEqualToExpr(new OperatorKey(
                 DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));
@@ -226,7 +228,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit( LessThanExpression op ) throws IOException {
+    public void visit( LessThanExpression op ) throws FrontendException {
         
         BinaryComparisonOperator exprOp = new LessThanExpr(new OperatorKey(
                 DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));
@@ -236,7 +238,7 @@ public class ExpToPhyTranslationVisitor 
     
     
     @Override
-    public void visit( LessThanEqualExpression op ) throws IOException {
+    public void visit( LessThanEqualExpression op ) throws FrontendException {
         
         BinaryComparisonOperator exprOp = new LTOrEqualToExpr(new OperatorKey(
                 DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));
@@ -245,7 +247,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit(ProjectExpression op) throws IOException {
+    public void visit(ProjectExpression op) throws FrontendException {
         POProject exprOp;
        
         if(op.getAttachedRelationalOp() instanceof LOGenerate && op.getPlan().getSuccessors(op)==null &&
@@ -267,7 +269,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit( MapLookupExpression op ) throws IOException {
+    public void visit( MapLookupExpression op ) throws FrontendException {
         ExpressionOperator physOp = new POMapLookUp(new OperatorKey(DEFAULT_SCOPE,
                 nodeGen.getNextNodeId(DEFAULT_SCOPE)));
         ((POMapLookUp)physOp).setLookUpKey(op.getLookupKey() );
@@ -289,7 +291,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit(org.apache.pig.newplan.logical.expression.ConstantExpression op) throws IOException {
+    public void visit(org.apache.pig.newplan.logical.expression.ConstantExpression op) throws FrontendException {
         
 //        System.err.println("Entering Constant");
         ConstantExpression ce = new ConstantExpression(new OperatorKey(DEFAULT_SCOPE,
@@ -305,7 +307,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit( CastExpression op ) throws IOException {
+    public void visit( CastExpression op ) throws FrontendException {
         POCast pCast = new POCast(new OperatorKey(DEFAULT_SCOPE, nodeGen
                 .getNextNodeId(DEFAULT_SCOPE)));
 //        physOp.setAlias(op.getAlias());
@@ -318,7 +320,15 @@ public class ExpToPhyTranslationVisitor 
         pCast.setFieldSchema(new ResourceSchema.ResourceFieldSchema(Util.translateFieldSchema(op.getFieldSchema())));
         FuncSpec lfSpec = op.getFuncSpec();
         if(null != lfSpec) {
-            pCast.setFuncSpec(lfSpec);
+            try {
+                pCast.setFuncSpec(lfSpec);
+            } catch (IOException e) {
+                int errCode = 1053;
+                String msg = "Cannot resolve load function to use for casting" +
+                        " from " + DataType.findTypeName(op.getExpression().
+                                getType()) + " to " + DataType.findTypeName(op.getType());
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
+            }
         }
         try {
             currentPlan.connect(from, pCast);
@@ -330,7 +340,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit( NotExpression op ) throws IOException {
+    public void visit( NotExpression op ) throws FrontendException {
         
         PONot pNot = new PONot(new OperatorKey(DEFAULT_SCOPE, nodeGen
                 .getNextNodeId(DEFAULT_SCOPE)));
@@ -353,7 +363,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit( IsNullExpression op ) throws IOException {
+    public void visit( IsNullExpression op ) throws FrontendException {
         POIsNull pIsNull = new POIsNull(new OperatorKey(DEFAULT_SCOPE, nodeGen
                 .getNextNodeId(DEFAULT_SCOPE)));
 //        physOp.setAlias(op.getAlias());
@@ -375,7 +385,7 @@ public class ExpToPhyTranslationVisitor 
     }
 
     @Override
-    public void visit( NegativeExpression op ) throws IOException {
+    public void visit( NegativeExpression op ) throws FrontendException {
         PONegative pNegative = new PONegative(new OperatorKey(DEFAULT_SCOPE, nodeGen
                 .getNextNodeId(DEFAULT_SCOPE)));
 //        physOp.setAlias(op.getAlias());
@@ -395,49 +405,49 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit( AddExpression op ) throws IOException {        
+    public void visit( AddExpression op ) throws FrontendException {        
         BinaryExpressionOperator exprOp = new Add(new OperatorKey(DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));        
         
         attachBinaryExpressionOperator(op, exprOp);
     }
     
     @Override
-    public void visit( RegexExpression op ) throws IOException {        
+    public void visit( RegexExpression op ) throws FrontendException {        
         BinaryExpressionOperator exprOp = new PORegexp(new OperatorKey(DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));        
         
         attachBinaryExpressionOperator(op, exprOp);
     }
     
     @Override
-    public void visit( SubtractExpression op ) throws IOException {        
+    public void visit( SubtractExpression op ) throws FrontendException {        
         BinaryExpressionOperator exprOp = new Subtract(new OperatorKey(DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));        
         
         attachBinaryExpressionOperator(op, exprOp);
     }
     
     @Override
-    public void visit( MultiplyExpression op ) throws IOException {        
+    public void visit( MultiplyExpression op ) throws FrontendException {        
         BinaryExpressionOperator exprOp = new Multiply(new OperatorKey(DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));        
         
         attachBinaryExpressionOperator(op, exprOp);
     }
     
     @Override
-    public void visit( DivideExpression op ) throws IOException {        
+    public void visit( DivideExpression op ) throws FrontendException {        
         BinaryExpressionOperator exprOp = new Divide(new OperatorKey(DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));        
         
         attachBinaryExpressionOperator(op, exprOp);
     }
     
     @Override
-    public void visit( ModExpression op ) throws IOException {        
+    public void visit( ModExpression op ) throws FrontendException {        
         BinaryExpressionOperator exprOp = new Mod(new OperatorKey(DEFAULT_SCOPE, nodeGen.getNextNodeId(DEFAULT_SCOPE)));        
         
         attachBinaryExpressionOperator(op, exprOp);
     }
     
     @Override
-    public void visit( BinCondExpression op ) throws IOException {
+    public void visit( BinCondExpression op ) throws FrontendException {
         
         POBinCond exprOp = new POBinCond( new OperatorKey(DEFAULT_SCOPE,
                 nodeGen.getNextNodeId(DEFAULT_SCOPE)) );
@@ -469,7 +479,7 @@ public class ExpToPhyTranslationVisitor 
     
     @SuppressWarnings("unchecked")
     @Override
-    public void visit( UserFuncExpression op ) throws IOException {       
+    public void visit( UserFuncExpression op ) throws FrontendException {       
         Object f = PigContext.instantiateFuncFromSpec(op.getFuncSpec());
         PhysicalOperator p;
         if (f instanceof EvalFunc) {
@@ -500,7 +510,7 @@ public class ExpToPhyTranslationVisitor 
     }
     
     @Override
-    public void visit( DereferenceExpression op ) throws IOException {
+    public void visit( DereferenceExpression op ) throws FrontendException {
         POProject exprOp = new POProject(new OperatorKey(DEFAULT_SCOPE, nodeGen
                 .getNextNodeId(DEFAULT_SCOPE)));
 

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/GreaterThanEqualExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/GreaterThanEqualExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/GreaterThanEqualExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/GreaterThanEqualExpression.java Thu Aug  5 19:38:51 2010
@@ -17,9 +17,8 @@
  */
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -44,29 +43,25 @@ public class GreaterThanEqualExpression 
      * @link org.apache.pig.experimental.plan.Operator#accept(org.apache.pig.experimental.plan.PlanVisitor)
      */
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
     
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof GreaterThanEqualExpression) {
             GreaterThanEqualExpression eo = (GreaterThanEqualExpression)other;
-            try {
-                return eo.getLhs().isEqual(getLhs()) && eo.getRhs().isEqual(getRhs());
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+            return eo.getLhs().isEqual(getLhs()) && eo.getRhs().isEqual(getRhs());
         } else {
             return false;
         }
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         fieldSchema = new LogicalSchema.LogicalFieldSchema(null, null, DataType.BOOLEAN);

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/GreaterThanExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/GreaterThanExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/GreaterThanExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/GreaterThanExpression.java Thu Aug  5 19:38:51 2010
@@ -17,9 +17,8 @@
  */
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -44,29 +43,25 @@ public class GreaterThanExpression exten
      * @link org.apache.pig.experimental.plan.Operator#accept(org.apache.pig.experimental.plan.PlanVisitor)
      */
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
     
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof GreaterThanExpression) {
             GreaterThanExpression eo = (GreaterThanExpression)other;
-            try {
-                return eo.getLhs().isEqual(getLhs()) && eo.getRhs().isEqual(getRhs());
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+            return eo.getLhs().isEqual(getLhs()) && eo.getRhs().isEqual(getRhs());
         } else {
             return false;
         }
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         fieldSchema = new LogicalSchema.LogicalFieldSchema(null, null, DataType.BOOLEAN);

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/IsNullExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/IsNullExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/IsNullExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/IsNullExpression.java Thu Aug  5 19:38:51 2010
@@ -18,9 +18,8 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -33,29 +32,25 @@ public class IsNullExpression extends Un
     }
 
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
 
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof IsNullExpression) { 
             IsNullExpression of = (IsNullExpression)other;
-            try {
-                return plan.isEqual(of.plan) && getExpression().isEqual( of.getExpression() );
-            } catch (IOException e) {
-                return false;
-            }
+            return plan.isEqual(of.plan) && getExpression().isEqual( of.getExpression() );
         } else {
             return false;
         }
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         fieldSchema = new LogicalSchema.LogicalFieldSchema(null, null, DataType.BOOLEAN);

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LessThanEqualExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LessThanEqualExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LessThanEqualExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LessThanEqualExpression.java Thu Aug  5 19:38:51 2010
@@ -17,9 +17,8 @@
  */
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -44,29 +43,25 @@ public class LessThanEqualExpression ext
      * @link org.apache.pig.experimental.plan.Operator#accept(org.apache.pig.experimental.plan.PlanVisitor)
      */
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
     
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof LessThanEqualExpression) {
             LessThanEqualExpression eo = (LessThanEqualExpression)other;
-            try {
-                return eo.getLhs().isEqual(getLhs()) && eo.getRhs().isEqual(getRhs());
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+            return eo.getLhs().isEqual(getLhs()) && eo.getRhs().isEqual(getRhs());
         } else {
             return false;
         }
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         fieldSchema = new LogicalSchema.LogicalFieldSchema(null, null, DataType.BOOLEAN);

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LessThanExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LessThanExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LessThanExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LessThanExpression.java Thu Aug  5 19:38:51 2010
@@ -17,9 +17,8 @@
  */
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -44,29 +43,25 @@ public class LessThanExpression extends 
      * @link org.apache.pig.experimental.plan.Operator#accept(org.apache.pig.experimental.plan.PlanVisitor)
      */
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
     
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof LessThanExpression) {
             LessThanExpression eo = (LessThanExpression)other;
-            try {
-                return eo.getLhs().isEqual(getLhs()) && eo.getRhs().isEqual(getRhs());
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+            return eo.getLhs().isEqual(getLhs()) && eo.getRhs().isEqual(getRhs());
         } else {
             return false;
         }
     }
 
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         fieldSchema = new LogicalSchema.LogicalFieldSchema(null, null, DataType.BOOLEAN);

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LogicalExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LogicalExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LogicalExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LogicalExpression.java Thu Aug  5 19:38:51 2010
@@ -18,9 +18,8 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.logical.relational.LogicalSchema;
@@ -54,7 +53,15 @@ public abstract class LogicalExpression 
         super(name, plan);
     }
     
-    abstract public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException;
+    /**
+     * Get the field schema for the output of this expression operator.  This does
+     * not merely return the field schema variable.  If schema is not yet set, this
+     * will attempt to construct it.  Therefore it is abstract since each
+     * operator will need to construct its field schema differently.
+     * @return the FieldSchema
+     * @throws FrontendException
+     */
+    abstract public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException;
     
     public void resetFieldSchema() {
         fieldSchema = null;
@@ -64,37 +71,30 @@ public abstract class LogicalExpression 
      * Get the data type for this expression.
      * @return data type, one of the static bytes of DataType
      */
-    public byte getType() {
-        try {
-            if (getFieldSchema()!=null)
-                return getFieldSchema().type;
-        } catch (IOException e) {
-        }
+    public byte getType() throws FrontendException {
+        if (getFieldSchema()!=null)
+            return getFieldSchema().type;
         return DataType.UNKNOWN;
     }
     
     public String toString() {
         StringBuilder msg = new StringBuilder();
-        try {
-            msg.append("(Name: " + name + " Type: ");
-            if (fieldSchema!=null)
-                msg.append(DataType.findTypeName(getFieldSchema().type));
-            else
-                msg.append("null");
-            msg.append(" Uid: ");
-            if (fieldSchema!=null)
-                msg.append(getFieldSchema().uid);
-            else
-                msg.append("null");
-            msg.append(")");
-        } catch (IOException e) {
-            e.printStackTrace();
-        }
+        msg.append("(Name: " + name + " Type: ");
+        if (fieldSchema!=null)
+            msg.append(DataType.findTypeName(fieldSchema.type));
+        else
+            msg.append("null");
+        msg.append(" Uid: ");
+        if (fieldSchema!=null)
+            msg.append(fieldSchema.uid);
+        else
+            msg.append("null");
+        msg.append(")");
 
         return msg.toString();
     }
     
-    public void neverUseForRealSetFieldSchema(LogicalFieldSchema fs) throws IOException {
+    public void neverUseForRealSetFieldSchema(LogicalFieldSchema fs) throws FrontendException {
         fieldSchema = fs;
         uidOnlyFieldSchema = fieldSchema.mergeUid(uidOnlyFieldSchema);
     }

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LogicalExpressionPlan.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LogicalExpressionPlan.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LogicalExpressionPlan.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LogicalExpressionPlan.java Thu Aug  5 19:38:51 2010
@@ -18,10 +18,10 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
 import java.io.PrintStream;
 import java.util.List;
 
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.BaseOperatorPlan;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
@@ -33,14 +33,14 @@ import org.apache.pig.newplan.logical.op
 public class LogicalExpressionPlan extends BaseOperatorPlan {
     
     @Override
-    public boolean isEqual(OperatorPlan other) {
+    public boolean isEqual(OperatorPlan other) throws FrontendException {
         if (other != null && other instanceof LogicalExpressionPlan) {
             LogicalExpressionPlan otherPlan = (LogicalExpressionPlan)other;
             List<Operator> roots = getSources();
             List<Operator> otherRoots = otherPlan.getSources();
             if (roots.size() == 0 && otherRoots.size() == 0) return true;
             if (roots.size() > 1 || otherRoots.size() > 1) {
-                throw new RuntimeException("Found LogicalExpressionPlan with more than one root.  Unexpected.");
+                throw new FrontendException("Found LogicalExpressionPlan with more than one root.  Unexpected.", 2224);
             }
             return roots.get(0).isEqual(otherRoots.get(0));            
         } else {
@@ -50,7 +50,7 @@ public class LogicalExpressionPlan exten
     
     @Override
     public void explain(PrintStream ps, String format, boolean verbose) 
-    throws IOException {
+    throws FrontendException {
         ps.println("#-----------------------------------------------");
         ps.println("# New Logical Expression Plan:");
         ps.println("#-----------------------------------------------");

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LogicalExpressionVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LogicalExpressionVisitor.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LogicalExpressionVisitor.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/LogicalExpressionVisitor.java Thu Aug  5 19:38:51 2010
@@ -18,8 +18,7 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
 import org.apache.pig.newplan.PlanWalker;
@@ -30,85 +29,85 @@ import org.apache.pig.newplan.PlanWalker
 public abstract class LogicalExpressionVisitor extends PlanVisitor {
 
     protected LogicalExpressionVisitor(OperatorPlan p,
-                                       PlanWalker walker) {
+                                       PlanWalker walker) throws FrontendException {
         super(p, walker);
         
         if (!(plan instanceof LogicalExpressionPlan)) {
-            throw new RuntimeException(
+            throw new FrontendException(
                 "LogicalExpressionVisitor expects to visit " +
-                "expression plans.");
+                "expression plans.", 2227);
         }
     }
     
-    public void visit(AndExpression op) throws IOException {
+    public void visit(AndExpression op) throws FrontendException {
     }
     
-    public void visit(OrExpression op) throws IOException { 
+    public void visit(OrExpression op) throws FrontendException { 
     }
 
-    public void visit(EqualExpression op) throws IOException {
+    public void visit(EqualExpression op) throws FrontendException {
     }
     
-    public void visit(ProjectExpression op) throws IOException {
+    public void visit(ProjectExpression op) throws FrontendException {
     }
     
-    public void visit(ConstantExpression op) throws IOException {
+    public void visit(ConstantExpression op) throws FrontendException {
     }
     
-    public void visit(CastExpression op) throws IOException {
+    public void visit(CastExpression op) throws FrontendException {
     }
 
-    public void visit(GreaterThanExpression op) throws IOException {
+    public void visit(GreaterThanExpression op) throws FrontendException {
     }
     
-    public void visit(GreaterThanEqualExpression op) throws IOException {
+    public void visit(GreaterThanEqualExpression op) throws FrontendException {
     }
 
-    public void visit(LessThanExpression op) throws IOException { 
+    public void visit(LessThanExpression op) throws FrontendException { 
     }
     
-    public void visit(LessThanEqualExpression op) throws IOException {
+    public void visit(LessThanEqualExpression op) throws FrontendException {
     }
 
-    public void visit(NotEqualExpression op) throws IOException { 
+    public void visit(NotEqualExpression op) throws FrontendException { 
     }
 
-    public void visit(NotExpression op ) throws IOException {
+    public void visit(NotExpression op ) throws FrontendException {
     }
 
-    public void visit(IsNullExpression op) throws IOException {
+    public void visit(IsNullExpression op) throws FrontendException {
     }
     
-    public void visit(NegativeExpression op) throws IOException {
+    public void visit(NegativeExpression op) throws FrontendException {
     }
     
-    public void visit(AddExpression op) throws IOException {
+    public void visit(AddExpression op) throws FrontendException {
     }
     
-    public void visit(SubtractExpression op) throws IOException {
+    public void visit(SubtractExpression op) throws FrontendException {
     }
     
-    public void visit(MultiplyExpression op) throws IOException {
+    public void visit(MultiplyExpression op) throws FrontendException {
     }
     
-    public void visit(ModExpression op) throws IOException {
+    public void visit(ModExpression op) throws FrontendException {
     }
     
-    public void visit(DivideExpression op) throws IOException {
+    public void visit(DivideExpression op) throws FrontendException {
     }
 
-    public void visit(MapLookupExpression op) throws IOException {
+    public void visit(MapLookupExpression op) throws FrontendException {
     }
 
-    public void visit(BinCondExpression op) throws IOException {        
+    public void visit(BinCondExpression op) throws FrontendException {        
     }
 
-    public void visit(UserFuncExpression op) throws IOException {
+    public void visit(UserFuncExpression op) throws FrontendException {
     }
 
-    public void visit(DereferenceExpression op) throws IOException {
+    public void visit(DereferenceExpression op) throws FrontendException {
     }
 
-    public void visit(RegexExpression op) throws IOException {
+    public void visit(RegexExpression op) throws FrontendException {
     }
 }

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/MapLookupExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/MapLookupExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/MapLookupExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/MapLookupExpression.java Thu Aug  5 19:38:51 2010
@@ -18,10 +18,10 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
 import java.util.List;
 
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -48,15 +48,15 @@ public class MapLookupExpression extends
      * @link org.apache.pig.experimental.plan.Operator#accept(org.apache.pig.experimental.plan.PlanVisitor)
      */
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
     
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof MapLookupExpression) {
             MapLookupExpression po = (MapLookupExpression)other;
             return ( po.mMapKey.compareTo(mMapKey) == 0 ) && 
@@ -66,7 +66,7 @@ public class MapLookupExpression extends
         }
     }
     
-    public LogicalExpression getMap() throws IOException {
+    public LogicalExpression getMap() throws FrontendException {
         List<Operator> preds = plan.getSuccessors(this);
         if(preds == null) {
             return null;
@@ -78,7 +78,7 @@ public class MapLookupExpression extends
         return mMapKey;
     }
     
-    public LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         if (mValueSchema!=null)
@@ -91,22 +91,18 @@ public class MapLookupExpression extends
 
     public String toString() {
         StringBuilder msg = new StringBuilder();
-        try {
-            msg.append("(Name: " + name + " Type: ");
-            if (fieldSchema!=null)
-                msg.append(DataType.findTypeName(getFieldSchema().type));
-            else
-                msg.append("null");
-            msg.append(" Uid: ");
-            if (fieldSchema!=null)
-                msg.append(getFieldSchema().uid);
-            else
-                msg.append("null");
-            msg.append(" Key: " + mMapKey);
-            msg.append(")");
-        } catch (IOException e) {
-            e.printStackTrace();
-        }
+        msg.append("(Name: " + name + " Type: ");
+        if (fieldSchema!=null)
+            msg.append(DataType.findTypeName(fieldSchema.type));
+        else
+            msg.append("null");
+        msg.append(" Uid: ");
+        if (fieldSchema!=null)
+            msg.append(fieldSchema.uid);
+        else
+            msg.append("null");
+        msg.append(" Key: " + mMapKey);
+        msg.append(")");
 
         return msg.toString();
     }

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/ModExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/ModExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/ModExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/ModExpression.java Thu Aug  5 19:38:51 2010
@@ -18,8 +18,7 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -47,29 +46,25 @@ public class ModExpression extends Binar
      * @link org.apache.pig.experimental.plan.Operator#accept(org.apache.pig.experimental.plan.PlanVisitor)
      */
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
     
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof ModExpression) {
             ModExpression ao = (ModExpression)other;
-            try {
-                return ao.getLhs().isEqual(getLhs()) && ao.getRhs().isEqual(getRhs());
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+            return ao.getLhs().isEqual(getLhs()) && ao.getRhs().isEqual(getRhs());
         } else {
             return false;
         }
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         fieldSchema = new LogicalSchema.LogicalFieldSchema(null, null, getLhs().getType());

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/MultiplyExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/MultiplyExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/MultiplyExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/MultiplyExpression.java Thu Aug  5 19:38:51 2010
@@ -18,8 +18,7 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -47,29 +46,25 @@ public class MultiplyExpression extends 
      * @link org.apache.pig.experimental.plan.Operator#accept(org.apache.pig.experimental.plan.PlanVisitor)
      */
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
     
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof MultiplyExpression) {
             MultiplyExpression ao = (MultiplyExpression)other;
-            try {
-                return ao.getLhs().isEqual(getLhs()) && ao.getRhs().isEqual(getRhs());
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+            return ao.getLhs().isEqual(getLhs()) && ao.getRhs().isEqual(getRhs());
         } else {
             return false;
         }
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         fieldSchema = new LogicalSchema.LogicalFieldSchema(null, null, getLhs().getType());

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/NegativeExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/NegativeExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/NegativeExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/NegativeExpression.java Thu Aug  5 19:38:51 2010
@@ -18,8 +18,7 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -32,29 +31,25 @@ public class NegativeExpression extends 
     }
 
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
 
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof NegativeExpression) { 
             NegativeExpression of = (NegativeExpression)other;
-            try {
-                return plan.isEqual(of.plan) && getExpression().isEqual( of.getExpression() );
-            } catch (IOException e) {
-                return false;
-            }
+            return plan.isEqual(of.plan) && getExpression().isEqual( of.getExpression() );
         } else {
             return false;
         }
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         LogicalExpression exp = (LogicalExpression)plan.getSuccessors(this).get(0);

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/NotEqualExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/NotEqualExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/NotEqualExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/NotEqualExpression.java Thu Aug  5 19:38:51 2010
@@ -18,9 +18,8 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -48,31 +47,25 @@ public class NotEqualExpression extends 
      * @link org.apache.pig.experimental.plan.Operator#accept(org.apache.pig.experimental.plan.PlanVisitor)
      */
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
     
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof NotEqualExpression) {
             NotEqualExpression eo = (NotEqualExpression)other;
-            try {
-                return eo.getLhs().isEqual(
-                        getLhs()) && 
-                eo.getRhs().isEqual(getRhs());
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+            return eo.getLhs().isEqual(getLhs()) && eo.getRhs().isEqual(getRhs());
         } else {
             return false;
         }
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         fieldSchema = new LogicalSchema.LogicalFieldSchema(null, null, DataType.BOOLEAN);

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/NotExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/NotExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/NotExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/NotExpression.java Thu Aug  5 19:38:51 2010
@@ -18,9 +18,8 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -33,29 +32,25 @@ public class NotExpression extends Unary
     }
 
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
 
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof NotExpression) { 
             NotExpression of = (NotExpression)other;
-            try {
-                return plan.isEqual(of.plan) && getExpression().isEqual( of.getExpression() );
-            } catch (IOException e) {
-                return false;
-            }
+            return plan.isEqual(of.plan) && getExpression().isEqual( of.getExpression() );
         } else {
             return false;
         }
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         fieldSchema = new LogicalSchema.LogicalFieldSchema(null, null, DataType.BOOLEAN);

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/OrExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/OrExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/OrExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/OrExpression.java Thu Aug  5 19:38:51 2010
@@ -17,9 +17,8 @@
  */
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -48,29 +47,25 @@ public class OrExpression extends Binary
      * @link org.apache.pig.experimental.plan.Operator#accept(org.apache.pig.experimental.plan.PlanVisitor)
      */
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
     
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof OrExpression) {
             OrExpression ao = (OrExpression)other;
-            try {
-                return ao.getLhs().isEqual(getLhs()) && ao.getRhs().isEqual(getRhs());
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+            return ao.getLhs().isEqual(getLhs()) && ao.getRhs().isEqual(getRhs());
         } else {
             return false;
         }
     }
 
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         fieldSchema = new LogicalSchema.LogicalFieldSchema(null, null, DataType.BOOLEAN);

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/ProjectExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/ProjectExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/ProjectExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/ProjectExpression.java Thu Aug  5 19:38:51 2010
@@ -18,10 +18,10 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
 import java.util.List;
 
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -69,9 +69,9 @@ public class ProjectExpression extends C
      * @link org.apache.pig.experimental.plan.Operator#accept(org.apache.pig.experimental.plan.PlanVisitor)
      */
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
 
@@ -116,7 +116,7 @@ public class ProjectExpression extends C
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         LogicalRelationalOperator referent = findReferent();
@@ -185,27 +185,26 @@ public class ProjectExpression extends C
     
     /**
      * Find the LogicalRelationalOperator that this projection refers to.
-     * @param currentOp Current operator this projection is attached to
      * @return LRO this projection refers to
-     * @throws IOException
+     * @throws FrontendException
      */
-    public LogicalRelationalOperator findReferent() throws IOException {
+    public LogicalRelationalOperator findReferent() throws FrontendException {
         List<Operator> preds;
         preds = attachedRelationalOp.getPlan().getPredecessors(attachedRelationalOp);
         if (preds == null || input >= preds.size()) {
-            throw new IOException("Projection with nothing to reference!");
+            throw new FrontendException("Projection with nothing to reference!", 2225);
         }
             
         LogicalRelationalOperator pred =
             (LogicalRelationalOperator)preds.get(input);
         if (pred == null) {
-            throw new IOException("Found bad operator in logical plan");
+            throw new FrontendException("Cannot fine reference for " + this, 2226);
         }
         return pred;
     }
     
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof ProjectExpression) {
             ProjectExpression po = (ProjectExpression)other;
             return po.input == input && po.col == col;
@@ -216,26 +215,22 @@ public class ProjectExpression extends C
     
     public String toString() {
         StringBuilder msg = new StringBuilder();
-        try {
-            msg.append("(Name: " + name + " Type: ");
-            if (fieldSchema!=null)
-                msg.append(DataType.findTypeName(getFieldSchema().type));
-            else
-                msg.append("null");
-            msg.append(" Uid: ");
-            if (fieldSchema!=null)
-                msg.append(getFieldSchema().uid);
-            else
-                msg.append("null");
-            msg.append(" Input: " + input + " Column: ");
-            if (isProjectStar())
-                msg.append("(*)");
-            else
-                msg.append(col);
-            msg.append(")");
-        } catch (IOException e) {
-            e.printStackTrace();
-        }
+        msg.append("(Name: " + name + " Type: ");
+        if (fieldSchema!=null)
+            msg.append(DataType.findTypeName(fieldSchema.type));
+        else
+            msg.append("null");
+        msg.append(" Uid: ");
+        if (fieldSchema!=null)
+            msg.append(fieldSchema.uid);
+        else
+            msg.append("null");
+        msg.append(" Input: " + input + " Column: ");
+        if (isProjectStar())
+            msg.append("(*)");
+        else
+            msg.append(col);
+        msg.append(")");
 
         return msg.toString();
     }

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/RegexExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/RegexExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/RegexExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/RegexExpression.java Thu Aug  5 19:38:51 2010
@@ -18,9 +18,8 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -48,29 +47,25 @@ public class RegexExpression extends Bin
      * @link org.apache.pig.experimental.plan.Operator#accept(org.apache.pig.experimental.plan.PlanVisitor)
      */
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
     
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof RegexExpression) {
             RegexExpression ao = (RegexExpression)other;
-            try {
-                return ao.getLhs().isEqual(getLhs()) && ao.getRhs().isEqual(getRhs());
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+            return ao.getLhs().isEqual(getLhs()) && ao.getRhs().isEqual(getRhs());
         } else {
             return false;
         }
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         fieldSchema = new LogicalSchema.LogicalFieldSchema(null, null, DataType.BOOLEAN);

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/SubtractExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/SubtractExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/SubtractExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/SubtractExpression.java Thu Aug  5 19:38:51 2010
@@ -18,8 +18,7 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
-
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanVisitor;
@@ -47,29 +46,25 @@ public class SubtractExpression extends 
      * @link org.apache.pig.experimental.plan.Operator#accept(org.apache.pig.experimental.plan.PlanVisitor)
      */
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
     
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if (other != null && other instanceof SubtractExpression) {
             SubtractExpression ao = (SubtractExpression)other;
-            try {
-                return ao.getLhs().isEqual(getLhs()) && ao.getRhs().isEqual(getRhs());
-            } catch (IOException e) {
-                throw new RuntimeException(e);
-            }
+            return ao.getLhs().isEqual(getLhs()) && ao.getRhs().isEqual(getRhs());
         } else {
             return false;
         }
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         fieldSchema = new LogicalSchema.LogicalFieldSchema(null, null, getLhs().getType());

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/UnaryExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/UnaryExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/UnaryExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/UnaryExpression.java Thu Aug  5 19:38:51 2010
@@ -18,9 +18,9 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
 import java.util.List;
 
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 
@@ -48,9 +48,9 @@ public abstract class UnaryExpression ex
     /**
      * Get the expression that this unary expression operators on.
      * @return expression on the left hand side
-     * @throws IOException 
+     * @throws FrontendException 
      */
-    public LogicalExpression getExpression() throws IOException {
+    public LogicalExpression getExpression() throws FrontendException {
         List<Operator> preds = plan.getSuccessors(this);
         if(preds == null) {
             return null;

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/UserFuncExpression.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/UserFuncExpression.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/UserFuncExpression.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/expression/UserFuncExpression.java Thu Aug  5 19:38:51 2010
@@ -18,7 +18,6 @@
 
 package org.apache.pig.newplan.logical.expression;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -26,6 +25,7 @@ import org.apache.pig.EvalFunc;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.data.DataType;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
@@ -48,15 +48,15 @@ public class UserFuncExpression extends 
     }
     
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalExpressionVisitor)) {
-            throw new IOException("Expected LogicalExpressionVisitor");
+            throw new FrontendException("Expected LogicalExpressionVisitor", 2222);
         }
         ((LogicalExpressionVisitor)v).visit(this);
     }
 
     @Override
-    public boolean isEqual(Operator other) {
+    public boolean isEqual(Operator other) throws FrontendException {
         if( other instanceof UserFuncExpression ) {
             UserFuncExpression exp = (UserFuncExpression)other;
             return plan.isEqual(exp.plan) && mFuncSpec.equals(exp.mFuncSpec );
@@ -65,10 +65,10 @@ public class UserFuncExpression extends 
         }
     }
 
-    public List<LogicalExpression> getArguments() {
+    public List<LogicalExpression> getArguments() throws FrontendException {
         List<Operator> successors = null;
         List<LogicalExpression> args = new ArrayList<LogicalExpression>();
-        try {
+//        try {
             successors = plan.getSuccessors(this);
 
             if(successors == null)
@@ -77,9 +77,9 @@ public class UserFuncExpression extends 
             for(Operator lo : successors){
                 args.add((LogicalExpression)lo);
             }
-        } catch (IOException e) {
-           return args;
-        }
+//        } catch (FrontendException e) {
+//           return args;
+//        }
         return args;
     }
 
@@ -91,7 +91,7 @@ public class UserFuncExpression extends 
     }
     
     @Override
-    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws IOException {
+    public LogicalSchema.LogicalFieldSchema getFieldSchema() throws FrontendException {
         if (fieldSchema!=null)
             return fieldSchema;
         LogicalSchema inputSchema = new LogicalSchema();

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/AllExpressionVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/AllExpressionVisitor.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/AllExpressionVisitor.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/AllExpressionVisitor.java Thu Aug  5 19:38:51 2010
@@ -18,9 +18,9 @@
 
 package org.apache.pig.newplan.logical.optimizer;
 
-import java.io.IOException;
 import java.util.Collection;
 
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.util.MultiMap;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanWalker;
@@ -52,7 +52,7 @@ public abstract class AllExpressionVisit
      * @param walker Walker to use to visit the plan.
      */
     public AllExpressionVisitor(OperatorPlan plan,
-                                PlanWalker walker) {
+                                PlanWalker walker) throws FrontendException {
         super(plan, walker);
     }
     
@@ -62,17 +62,17 @@ public abstract class AllExpressionVisit
      * @param expr LogicalExpressionPlan that will be visited
      * @return a new LogicalExpressionVisitor for that expression
      */
-    abstract protected LogicalExpressionVisitor getVisitor(LogicalExpressionPlan expr);
+    abstract protected LogicalExpressionVisitor getVisitor(LogicalExpressionPlan expr) throws FrontendException;
     
     @Override
-    public void visit(LOFilter filter) throws IOException {
+    public void visit(LOFilter filter) throws FrontendException {
         currentOp = filter;
         LogicalExpressionVisitor v = getVisitor(filter.getFilterPlan());
         v.visit();
     }
     
     @Override
-    public void visit(LOJoin join) throws IOException {
+    public void visit(LOJoin join) throws FrontendException {
         currentOp = join;
         Collection<LogicalExpressionPlan> c = join.getExpressionPlans();
         for (LogicalExpressionPlan plan : c) {
@@ -82,7 +82,7 @@ public abstract class AllExpressionVisit
     }
     
     @Override
-    public void visit(LOCogroup cg) throws IOException {
+    public void visit(LOCogroup cg) throws FrontendException {
         currentOp = cg;
         MultiMap<Integer, LogicalExpressionPlan> expressionPlans = cg.getExpressionPlans();
         for( Integer key : expressionPlans.keySet() ) {
@@ -95,7 +95,7 @@ public abstract class AllExpressionVisit
     }
     
     @Override
-    public void visit(LOForEach foreach) throws IOException {
+    public void visit(LOForEach foreach) throws FrontendException {
         currentOp = foreach;
         // We have an Inner OperatorPlan in ForEach, so we go ahead
         // and work on that plan
@@ -107,7 +107,7 @@ public abstract class AllExpressionVisit
     }
     
     @Override
-    public void visit(LOGenerate gen ) throws IOException {
+    public void visit(LOGenerate gen ) throws FrontendException {
         currentOp = gen;
         Collection<LogicalExpressionPlan> plans = gen.getOutputPlans();
         for( LogicalExpressionPlan plan : plans ) {
@@ -117,7 +117,7 @@ public abstract class AllExpressionVisit
     }
     
     @Override
-    public void visit(LOInnerLoad load) throws IOException {
+    public void visit(LOInnerLoad load) throws FrontendException {
         // the expression in LOInnerLoad contains info relative from LOForEach
         // so use LOForeach as currentOp
         currentOp = load.getLOForEach();
@@ -128,14 +128,14 @@ public abstract class AllExpressionVisit
     }
     
     @Override
-    public void visit(LOSplitOutput splitOutput) throws IOException {
+    public void visit(LOSplitOutput splitOutput) throws FrontendException {
         currentOp = splitOutput;
         LogicalExpressionVisitor v = getVisitor(splitOutput.getFilterPlan());
         v.visit();
     }
     
     @Override
-    public void visit(LOSort sort) throws IOException {
+    public void visit(LOSort sort) throws FrontendException {
         currentOp = sort;
         Collection<LogicalExpressionPlan> c = sort.getSortColPlans();
         for (LogicalExpressionPlan plan : c) {

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/AllSameRalationalNodesVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/AllSameRalationalNodesVisitor.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/AllSameRalationalNodesVisitor.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/AllSameRalationalNodesVisitor.java Thu Aug  5 19:38:51 2010
@@ -18,8 +18,7 @@
 
 package org.apache.pig.newplan.logical.optimizer;
 
-import java.io.IOException;
-
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.PlanWalker;
 import org.apache.pig.newplan.logical.relational.LOCogroup;
@@ -50,7 +49,7 @@ public abstract class AllSameRalationalN
      * @param plan OperatorPlan to visit
      * @param walker Walker to use to visit the plan
      */
-    public AllSameRalationalNodesVisitor(OperatorPlan plan, PlanWalker walker) {
+    public AllSameRalationalNodesVisitor(OperatorPlan plan, PlanWalker walker) throws FrontendException {
         super(plan, walker);
     }
     
@@ -58,70 +57,70 @@ public abstract class AllSameRalationalN
      * Method to call on every node in the logical plan.
      * @param op Node that is currently being visited.
      */
-    abstract protected void execute(LogicalRelationalOperator op) throws IOException;
+    abstract protected void execute(LogicalRelationalOperator op) throws FrontendException;
     
     @Override
-    public void visit(LOFilter filter) throws IOException {
+    public void visit(LOFilter filter) throws FrontendException {
         execute(filter);
     }
 
     @Override
-    public void visit(LOJoin join) throws IOException {
+    public void visit(LOJoin join) throws FrontendException {
         execute(join);
     }
 
     @Override
-    public void visit(LOCogroup cg) throws IOException {
+    public void visit(LOCogroup cg) throws FrontendException {
         execute(cg);
     }
 
     @Override
-    public void visit(LOLoad load) throws IOException {
+    public void visit(LOLoad load) throws FrontendException {
         execute(load);
     }
     
     @Override
-    public void visit(LOStore store) throws IOException {
+    public void visit(LOStore store) throws FrontendException {
         execute(store);
     }
     
     @Override
-    public void visit(LOForEach foreach) throws IOException {
+    public void visit(LOForEach foreach) throws FrontendException {
         execute(foreach);
     }
     
     @Override
-    public void visit(LOSplit split) throws IOException {
+    public void visit(LOSplit split) throws FrontendException {
         execute(split);
     }
     
     @Override
-    public void visit(LOSplitOutput splitOutput) throws IOException {
+    public void visit(LOSplitOutput splitOutput) throws FrontendException {
         execute(splitOutput);
     }
     
     @Override
-    public void visit(LOUnion union) throws IOException {
+    public void visit(LOUnion union) throws FrontendException {
         execute(union);
     }
     
     @Override
-    public void visit(LOSort sort) throws IOException {
+    public void visit(LOSort sort) throws FrontendException {
         execute(sort);
     }
     
     @Override
-    public void visit(LODistinct distinct) throws IOException {
+    public void visit(LODistinct distinct) throws FrontendException {
         execute(distinct);
     }
     
     @Override
-    public void visit(LOCross cross) throws IOException {
+    public void visit(LOCross cross) throws FrontendException {
         execute(cross);
     }
     
     @Override
-    public void visit(LOStream stream) throws IOException {
+    public void visit(LOStream stream) throws FrontendException {
         execute(stream);
     }
 }

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/ExprPrinter.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/ExprPrinter.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/ExprPrinter.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/ExprPrinter.java Thu Aug  5 19:38:51 2010
@@ -17,9 +17,9 @@
  */
 package org.apache.pig.newplan.logical.optimizer;
 
-import java.io.IOException;
 import java.io.PrintStream;
 
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.DepthFirstMemoryWalker;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.logical.expression.AddExpression;
@@ -53,12 +53,12 @@ public class ExprPrinter extends Logical
 
     protected PrintStream stream = null;
     
-    public ExprPrinter(OperatorPlan plan, int startingLevel, PrintStream ps) {
+    public ExprPrinter(OperatorPlan plan, int startingLevel, PrintStream ps) throws FrontendException {
         super(plan, new DepthFirstMemoryWalker(plan, startingLevel));
         stream = ps;
     }
     
-    public ExprPrinter(OperatorPlan plan, PrintStream ps) {
+    public ExprPrinter(OperatorPlan plan, PrintStream ps) throws FrontendException {
         super(plan, new DepthFirstMemoryWalker(plan, 0));
         stream = ps;
     }
@@ -69,122 +69,122 @@ public class ExprPrinter extends Logical
     }
 
     @Override
-    public void visit(AndExpression exp) throws IOException {
+    public void visit(AndExpression exp) throws FrontendException {
         simplevisit(exp);
     }
 
     @Override
-    public void visit(OrExpression exp) throws IOException {
+    public void visit(OrExpression exp) throws FrontendException {
         simplevisit(exp);
     }
 
     @Override
-    public void visit(EqualExpression exp) throws IOException {
+    public void visit(EqualExpression exp) throws FrontendException {
         simplevisit(exp);
     }
 
     @Override
-    public void visit(ProjectExpression exp) throws IOException {
+    public void visit(ProjectExpression exp) throws FrontendException {
         simplevisit(exp);
     }
     
     @Override
-    public void visit(MapLookupExpression exp) throws IOException {
+    public void visit(MapLookupExpression exp) throws FrontendException {
         simplevisit(exp);
     }
 
     @Override
-    public void visit(ConstantExpression exp) throws IOException {
+    public void visit(ConstantExpression exp) throws FrontendException {
         simplevisit(exp);
     }
 
     @Override
-    public void visit(CastExpression exp) throws IOException {
+    public void visit(CastExpression exp) throws FrontendException {
         simplevisit(exp);
     }
 
     @Override
-    public void visit(GreaterThanExpression exp) throws IOException {
+    public void visit(GreaterThanExpression exp) throws FrontendException {
         simplevisit(exp);
     }
 
     @Override
-    public void visit(GreaterThanEqualExpression exp) throws IOException {
+    public void visit(GreaterThanEqualExpression exp) throws FrontendException {
         simplevisit(exp);
     }
 
     @Override
-    public void visit(LessThanExpression exp) throws IOException {
+    public void visit(LessThanExpression exp) throws FrontendException {
         simplevisit(exp);
     }
 
     @Override
-    public void visit(LessThanEqualExpression exp) throws IOException {
+    public void visit(LessThanEqualExpression exp) throws FrontendException {
         simplevisit(exp);
     }
     
     @Override
-    public void visit(NotEqualExpression exp) throws IOException { 
+    public void visit(NotEqualExpression exp) throws FrontendException { 
         simplevisit(exp);
     }
 
     @Override
-    public void visit(NotExpression exp ) throws IOException {
+    public void visit(NotExpression exp ) throws FrontendException {
         simplevisit(exp);
     }
 
     @Override
-    public void visit(IsNullExpression exp) throws IOException {
+    public void visit(IsNullExpression exp) throws FrontendException {
         simplevisit(exp);
     }
     
     @Override
-    public void visit(NegativeExpression exp) throws IOException {
+    public void visit(NegativeExpression exp) throws FrontendException {
         simplevisit(exp);
     }
     
     @Override
-    public void visit(AddExpression exp) throws IOException {
+    public void visit(AddExpression exp) throws FrontendException {
         simplevisit(exp);
     }
     
     @Override
-    public void visit(SubtractExpression exp) throws IOException {
+    public void visit(SubtractExpression exp) throws FrontendException {
         simplevisit(exp);
     }
     
     @Override
-    public void visit(MultiplyExpression exp) throws IOException {
+    public void visit(MultiplyExpression exp) throws FrontendException {
         simplevisit(exp);
     }
     
     @Override
-    public void visit(ModExpression exp) throws IOException {
+    public void visit(ModExpression exp) throws FrontendException {
         simplevisit(exp);
     }
     
     @Override
-    public void visit(DivideExpression exp) throws IOException {
+    public void visit(DivideExpression exp) throws FrontendException {
         simplevisit(exp);
     }
     
     @Override
-    public void visit(BinCondExpression exp ) throws IOException {
+    public void visit(BinCondExpression exp ) throws FrontendException {
         simplevisit(exp);
     }
     
     @Override
-    public void visit(UserFuncExpression exp) throws IOException {
+    public void visit(UserFuncExpression exp) throws FrontendException {
         simplevisit(exp);
     }
     
     @Override
-    public void visit(DereferenceExpression exp) throws IOException {
+    public void visit(DereferenceExpression exp) throws FrontendException {
         simplevisit(exp);
     }
     
     @Override
-    public void visit(RegexExpression op) throws IOException {
+    public void visit(RegexExpression op) throws FrontendException {
         simplevisit(op);
     }
 }

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/LogicalPlanOptimizer.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/LogicalPlanOptimizer.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/LogicalPlanOptimizer.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/optimizer/LogicalPlanOptimizer.java Thu Aug  5 19:38:51 2010
@@ -35,9 +35,11 @@ import org.apache.pig.newplan.optimizer.
 import org.apache.pig.newplan.optimizer.Rule;
 
 public class LogicalPlanOptimizer extends PlanOptimizer {
-
-    public LogicalPlanOptimizer(OperatorPlan p, int iterations) {    	
+    private Set<String> mRulesOff = null;
+    
+    public LogicalPlanOptimizer(OperatorPlan p, int iterations, Set<String> turnOffRules) {    	
         super(p, null, iterations);
+        this.mRulesOff = turnOffRules;
         ruleSets = buildRuleSets();
         addListeners();
     }
@@ -45,67 +47,94 @@ public class LogicalPlanOptimizer extend
     protected List<Set<Rule>> buildRuleSets() {
         List<Set<Rule>> ls = new ArrayList<Set<Rule>>();	    
 
-        // TypeCastInserter
+        // TypeCastInserter set
         // This set of rules Insert Foreach dedicated for casting after load
         Set<Rule> s = new HashSet<Rule>();
-        ls.add(s);
         // add split filter rule
         Rule r = new TypeCastInserter("TypeCastInserter", LOLoad.class.getName());
-        s.add(r);
+        checkAndAddRule(s, r);
+        if (!s.isEmpty())
+            ls.add(s);
         
         // Split Set
         // This set of rules does splitting of operators only.
         // It does not move operators
         s = new HashSet<Rule>();
-        ls.add(s);
         // add split filter rule
         r = new SplitFilter("SplitFilter");
-        s.add(r);
-                
-         
+        checkAndAddRule(s, r);
+        if (!s.isEmpty())
+            ls.add(s);
         
         
         // Push Set,
         // This set does moving of operators only.
         s = new HashSet<Rule>();
-        ls.add(s);
-        // add push up filter rule
         r = new PushUpFilter("PushUpFilter");
-        s.add(r);
+        checkAndAddRule(s, r);
         r = new FilterAboveForeach("FilterAboveForEachWithFlatten");
-        s.add(r);
-        
-        
-        
+        checkAndAddRule(s, r);
+        if (!s.isEmpty())
+            ls.add(s);
         
         // Merge Set
         // This Set merges operators but does not move them.
         s = new HashSet<Rule>();
-        ls.add(s);
+        checkAndAddRule(s, r);
         // add merge filter rule
         r = new MergeFilter("MergeFilter");        
-        s.add(r);	    
+        checkAndAddRule(s, r);
+        if (!s.isEmpty())
+            ls.add(s);
         
-        
-        // Prune Set Marker
+        // Prune Set
         // This set is used for pruning columns and maps
-      
         s = new HashSet<Rule>();
-        ls.add(s);
         // Add the PruneMap Filter
         r = new ColumnMapKeyPrune("ColumnMapKeyPrune");
-        s.add(r);
+        checkAndAddRule(s, r);
+        if (!s.isEmpty())
+            ls.add(s);
         
-        // Add LOForEach operator to trim off columns
+        // Add LOForEach set
         s = new HashSet<Rule>();
-        ls.add(s);
         // Add the AddForEach
         r = new AddForEach("AddForEach");
-        s.add(r);
-
+        checkAndAddRule(s, r);
+        if (!s.isEmpty())
+            ls.add(s);
         
         return ls;
     }
+        
+    private void checkAndAddRule(Set<Rule> ruleSet, Rule rule) {
+        if (rule.isMandatory()) {
+            ruleSet.add(rule);
+            return;
+        }
+        
+        boolean turnAllRulesOff = false;
+        if (mRulesOff != null) {
+            for (String ruleName : mRulesOff) {
+                if ("all".equalsIgnoreCase(ruleName)) {
+                    turnAllRulesOff = true;
+                    break;
+                }
+            }
+        }
+        
+        if (turnAllRulesOff) return;
+        
+        if(mRulesOff != null) {
+            for(String ruleOff: mRulesOff) {
+                String ruleName = rule.getName();
+                if(ruleName == null) continue;
+                if(ruleName.equalsIgnoreCase(ruleOff)) return;
+            }
+        }
+        
+        ruleSet.add(rule);
+    }
     
     private void addListeners() {
         addPlanTransformListener(new SchemaPatcher());