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 [4/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/rules/ColumnPruneHelper.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/ColumnPruneHelper.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/ColumnPruneHelper.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/ColumnPruneHelper.java Thu Aug  5 19:38:51 2010
@@ -17,7 +17,6 @@
  */
 package org.apache.pig.newplan.logical.rules;
 
-import java.io.IOException;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -26,6 +25,7 @@ import java.util.Map;
 import java.util.Set;
 
 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.OperatorSubPlan;
@@ -73,7 +73,7 @@ public class ColumnPruneHelper {
         this.currentPlan = currentPlan;
     }    
     
-    private OperatorSubPlan getSubPlan() throws IOException {
+    private OperatorSubPlan getSubPlan() throws FrontendException {
         OperatorSubPlan p = null;
         if (currentPlan instanceof OperatorSubPlan) {
             p = new OperatorSubPlan(((OperatorSubPlan)currentPlan).getBasePlan());
@@ -92,7 +92,7 @@ public class ColumnPruneHelper {
         return p;
     }
     
-    private void addOperator(Operator op, OperatorSubPlan subplan) throws IOException {
+    private void addOperator(Operator op, OperatorSubPlan subplan) throws FrontendException {
         if (op == null) {
             return;
         }
@@ -111,7 +111,7 @@ public class ColumnPruneHelper {
     
         
     @SuppressWarnings("unchecked")
-    public boolean check() throws IOException {
+    public boolean check() throws FrontendException {
         List<Operator> sources = currentPlan.getSources();
         // if this rule has run before, just return false
         if (sources.get(0).getAnnotation(INPUTUIDS) != null) {
@@ -151,7 +151,7 @@ public class ColumnPruneHelper {
     }
 
     // get a set of column indexes from a set of uids
-    protected Set<Integer> getColumns(LogicalSchema schema, Set<Long> uids) throws IOException {
+    protected Set<Integer> getColumns(LogicalSchema schema, Set<Long> uids) throws FrontendException {
         if (schema == null) {
             throw new SchemaNotDefinedException("Schema is not defined.");
         }
@@ -162,7 +162,7 @@ public class ColumnPruneHelper {
             long uid = iter.next();
             int index = schema.findField(uid);
             if (index == -1) {
-                throw new IOException("UID " + uid + " is not found in the schema");
+                throw new FrontendException("UID " + uid + " is not found in the schema " + schema, 2241);
             }
               
             cols.add(index);
@@ -182,12 +182,12 @@ public class ColumnPruneHelper {
     // that have their own schema.
     static private class ColumnDependencyVisitor extends LogicalRelationalNodesVisitor {    	
         
-        public ColumnDependencyVisitor(OperatorPlan plan) {
+        public ColumnDependencyVisitor(OperatorPlan plan) throws FrontendException {
             super(plan, new ReverseDependencyOrderWalker(plan));            
         }
         
         @Override
-        public void visit(LOLoad load) throws IOException {
+        public void visit(LOLoad load) throws FrontendException {
             Set<Long> output = setOutputUids(load);
             
             // for load, input uids are same as output uids
@@ -195,7 +195,7 @@ public class ColumnPruneHelper {
         }
 
         @Override
-        public void visit(LOFilter filter) throws IOException {
+        public void visit(LOFilter filter) throws FrontendException {
             Set<Long> output = setOutputUids(filter);
             
             // the input uids contains all the output uids and
@@ -209,7 +209,7 @@ public class ColumnPruneHelper {
         }
         
         @Override
-        public void visit(LOStore store) throws IOException {
+        public void visit(LOStore store) throws FrontendException {
             Set<Long> output = setOutputUids(store);            
             
             if (output.isEmpty()) {
@@ -229,7 +229,7 @@ public class ColumnPruneHelper {
         }
         
         @Override
-        public void visit(LOJoin join) throws IOException {
+        public void visit(LOJoin join) throws FrontendException {
             Set<Long> output = setOutputUids(join);
             
             // the input uids contains all the output uids and
@@ -247,7 +247,7 @@ public class ColumnPruneHelper {
         }
         
         @Override
-        public void visit(LOCogroup cg) throws IOException {
+        public void visit(LOCogroup cg) throws FrontendException {
             Set<Long> output = setOutputUids(cg);
             
             // the input uids contains all the output uids and
@@ -278,13 +278,13 @@ public class ColumnPruneHelper {
         }
         
         @Override
-        public void visit(LOLimit limit) throws IOException {
+        public void visit(LOLimit limit) throws FrontendException {
             Set<Long> output = setOutputUids(limit);
             limit.annotate(INPUTUIDS, output);
         }
         
         @Override
-        public void visit(LOStream stream) throws IOException {
+        public void visit(LOStream stream) throws FrontendException {
             Set<Long> input = new HashSet<Long>();
             
             // Every field is required
@@ -300,7 +300,7 @@ public class ColumnPruneHelper {
         }
         
         @Override
-        public void visit(LODistinct distinct) throws IOException {
+        public void visit(LODistinct distinct) throws FrontendException {
             Set<Long> input = new HashSet<Long>();
             
             // Every field is required
@@ -316,7 +316,7 @@ public class ColumnPruneHelper {
         }
         
         @Override
-        public void visit(LOCross cross) throws IOException {
+        public void visit(LOCross cross) throws FrontendException {
             Set<Long> output = setOutputUids(cross);
             // Since we do not change the topology of the plan, we keep
             // at least one input for each predecessor.
@@ -336,7 +336,7 @@ public class ColumnPruneHelper {
         }
         
         @Override
-        public void visit(LOUnion union) throws IOException {
+        public void visit(LOUnion union) throws FrontendException {
             Set<Long> output = setOutputUids(union);
             Set<Long> input = new HashSet<Long>();
             for (long uid : output) {
@@ -346,13 +346,13 @@ public class ColumnPruneHelper {
         }
         
         @Override
-        public void visit(LOSplit split) throws IOException {
+        public void visit(LOSplit split) throws FrontendException {
             Set<Long> output = setOutputUids(split);
             split.annotate(INPUTUIDS, output);
         }
         
         @Override
-        public void visit(LOSplitOutput splitOutput) throws IOException {
+        public void visit(LOSplitOutput splitOutput) throws FrontendException {
             Set<Long> output = setOutputUids(splitOutput);
             
             // the input uids contains all the output uids and
@@ -366,7 +366,7 @@ public class ColumnPruneHelper {
         }
         
         @Override
-        public void visit(LOSort sort) throws IOException {
+        public void visit(LOSort sort) throws FrontendException {
             Set<Long> output = setOutputUids(sort);
             
             Set<Long> input = new HashSet<Long>(output);
@@ -399,7 +399,7 @@ public class ColumnPruneHelper {
         }
         
         @Override
-        public void visit(LOForEach foreach) throws IOException {
+        public void visit(LOForEach foreach) throws FrontendException {
             Set<Long> output = setOutputUids(foreach);
             
             LogicalPlan innerPlan = foreach.getInnerPlan();
@@ -413,7 +413,7 @@ public class ColumnPruneHelper {
 
         @Override
         @SuppressWarnings("unchecked")
-        public void visit(LOGenerate gen) throws IOException {
+        public void visit(LOGenerate gen) throws FrontendException {
              Set<Long> output = (Set<Long>)gen.getAnnotation(OUTPUTUIDS);
              
              Set<Long> input = new HashSet<Long>();
@@ -507,12 +507,12 @@ public class ColumnPruneHelper {
         }
         
         @Override
-        public void visit(LOInnerLoad load) throws IOException {
+        public void visit(LOInnerLoad load) throws FrontendException {
             Set<Long> output = setOutputUids(load);
             load.annotate(INPUTUIDS, output);
         }
         
-        private void collectUids(LogicalRelationalOperator currentOp, LogicalExpressionPlan exp, Set<Long> uids) throws IOException {
+        private void collectUids(LogicalRelationalOperator currentOp, LogicalExpressionPlan exp, Set<Long> uids) throws FrontendException {
             List<Operator> ll = exp.getSinks();
             for(Operator op: ll) {
                 if (op instanceof ProjectExpression) {
@@ -534,7 +534,7 @@ public class ColumnPruneHelper {
         }
         
         @SuppressWarnings("unchecked")
-        private Set<Long> setOutputUids(LogicalRelationalOperator op) throws IOException {
+        private Set<Long> setOutputUids(LogicalRelationalOperator op) throws FrontendException {
             
             List<Operator> ll = plan.getSuccessors(op);
             Set<Long> uids = new HashSet<Long>();

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/ColumnPruneVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/ColumnPruneVisitor.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/ColumnPruneVisitor.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/ColumnPruneVisitor.java Thu Aug  5 19:38:51 2010
@@ -17,7 +17,6 @@
  */
 package org.apache.pig.newplan.logical.rules;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -64,7 +63,7 @@ public class ColumnPruneVisitor extends 
     private boolean columnPrune;
 
     public ColumnPruneVisitor(OperatorPlan plan, Map<LOLoad,Pair<Map<Integer,Set<String>>,Set<Integer>>> requiredItems,
-            boolean columnPrune) {
+            boolean columnPrune) throws FrontendException {
         super(plan, new ReverseDependencyOrderWalker(plan));
         this.columnPrune = columnPrune;
         this.requiredItems = requiredItems;
@@ -75,7 +74,7 @@ public class ColumnPruneVisitor extends 
     }
     
     @Override
-    public void visit(LOLoad load) throws IOException {
+    public void visit(LOLoad load) throws FrontendException {
         if(! requiredItems.containsKey( load ) ) {
             return;
         }
@@ -192,37 +191,37 @@ public class ColumnPruneVisitor extends 
     }
 
     @Override
-    public void visit(LOFilter filter) throws IOException {
+    public void visit(LOFilter filter) throws FrontendException {
     }
     
     @Override
-    public void visit(LOSplitOutput splitOutput) throws IOException {
+    public void visit(LOSplitOutput splitOutput) throws FrontendException {
     }
     
     @Override
-    public void visit(LOSort sort) throws IOException {
+    public void visit(LOSort sort) throws FrontendException {
     }
     
     @Override
-    public void visit(LOStore store) throws IOException {
+    public void visit(LOStore store) throws FrontendException {
     }
     
     @Override
-    public void visit( LOCogroup cg ) throws IOException {
+    public void visit( LOCogroup cg ) throws FrontendException {
         addForEachIfNecessary(cg);
     }
     
     @Override
-    public void visit(LOJoin join) throws IOException {
+    public void visit(LOJoin join) throws FrontendException {
     }
     
     @Override
-    public void visit(LOCross cross) throws IOException {
+    public void visit(LOCross cross) throws FrontendException {
     }
     
     @Override
     @SuppressWarnings("unchecked")
-    public void visit(LOForEach foreach) throws IOException {
+    public void visit(LOForEach foreach) throws FrontendException {
         if (!columnPrune) {
             return;
         }
@@ -355,7 +354,7 @@ public class ColumnPruneVisitor extends 
     }
     
     @Override
-    public void visit(LOUnion union) throws IOException {
+    public void visit(LOUnion union) throws FrontendException {
         // AddForEach before union if necessary.
         List<Operator> preds = new ArrayList<Operator>();
         preds.addAll(plan.getPredecessors(union));
@@ -366,7 +365,7 @@ public class ColumnPruneVisitor extends 
     }
     
     // remove all the operators starting from an operator
-    private void removeSubTree(LogicalRelationalOperator op) throws IOException {
+    private void removeSubTree(LogicalRelationalOperator op) throws FrontendException {
         LogicalPlan p = (LogicalPlan)op.getPlan();
         List<Operator> ll = p.getPredecessors(op);
         if (ll != null) {
@@ -387,7 +386,7 @@ public class ColumnPruneVisitor extends 
 
     // Add ForEach after op to prune unnecessary columns
     @SuppressWarnings("unchecked")
-    private void addForEachIfNecessary(LogicalRelationalOperator op) throws IOException {
+    private void addForEachIfNecessary(LogicalRelationalOperator op) throws FrontendException {
         Set<Long> outputUids = (Set<Long>)op.getAnnotation(ColumnPruneHelper.OUTPUTUIDS);
         LogicalSchema schema = op.getSchema();
         Set<Integer> columnsToDrop = new HashSet<Integer>();

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/FilterAboveForeach.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/FilterAboveForeach.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/FilterAboveForeach.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/FilterAboveForeach.java Thu Aug  5 19:38:51 2010
@@ -17,11 +17,11 @@
  */
 package org.apache.pig.newplan.logical.rules;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.util.Pair;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
@@ -45,7 +45,7 @@ import org.apache.pig.newplan.optimizer.
 public class FilterAboveForeach extends Rule {
 
     public FilterAboveForeach(String n) {
-        super(n);
+        super(n, false);
     }
 
     @Override
@@ -76,7 +76,7 @@ public class FilterAboveForeach extends 
         OperatorSubPlan subPlan = null;
         
         @Override
-        public boolean check(OperatorPlan matched) throws IOException {
+        public boolean check(OperatorPlan matched) throws FrontendException {
             Iterator<Operator> iter = matched.getOperators();
             while( iter.hasNext() ) {
                 Operator op = iter.next();
@@ -131,7 +131,7 @@ public class FilterAboveForeach extends 
          * @param filter
          * @return Set of uid
          */
-        private Pair<List<Long>, List<Byte>> getFilterProjectionUids(LOFilter filter) throws IOException {
+        private Pair<List<Long>, List<Byte>> getFilterProjectionUids(LOFilter filter) throws FrontendException {
             List<Long> uids = new ArrayList<Long>();
             List<Byte> types = new ArrayList<Byte>();
             if( filter != null ) {
@@ -157,7 +157,7 @@ public class FilterAboveForeach extends 
          * @return true if given LogicalRelationalOperator has all the given uids
          */
         private boolean hasAll(LogicalRelationalOperator op, Pair<List<Long>, 
-                List<Byte>> uidWithTypes) {
+                List<Byte>> uidWithTypes) throws FrontendException {
             LogicalSchema schema = op.getSchema();
             
             if (schema==null)
@@ -184,7 +184,7 @@ public class FilterAboveForeach extends 
         }
 
         @Override
-        public void transform(OperatorPlan matched) throws IOException {
+        public void transform(OperatorPlan matched) throws FrontendException {
             
             List<Operator> opSet = currentPlan.getPredecessors(filter);
             if( ! ( opSet != null && opSet.size() > 0 ) ) {

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/MapKeysPruneHelper.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/MapKeysPruneHelper.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/MapKeysPruneHelper.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/MapKeysPruneHelper.java Thu Aug  5 19:38:51 2010
@@ -18,7 +18,6 @@
 
 package org.apache.pig.newplan.logical.rules;
 
-import java.io.IOException;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -27,6 +26,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.DependencyOrderWalker;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
@@ -88,7 +88,7 @@ public class MapKeysPruneHelper {
   
 
     @SuppressWarnings("unchecked")
-    public boolean check() throws IOException {       
+    public boolean check() throws FrontendException {       
         
         // First check if we have a load with a map in it or not
         List<Operator> sources = currentPlan.getSources();
@@ -157,7 +157,7 @@ public class MapKeysPruneHelper {
      * @return true if it has a map, else false
      * @throws NullPointerException incase Schema is null
      */
-    private boolean hasMap(LogicalSchema schema ) throws NullPointerException {
+    private boolean hasMap(LogicalSchema schema ) {
         for( LogicalFieldSchema field : schema.getFields() ) {
             if( field.type == DataType.MAP ) {
                 return true;
@@ -197,13 +197,13 @@ public class MapKeysPruneHelper {
         
         Map<Long,Set<String>> inputUids = null;
 
-        protected MapMarker(OperatorPlan plan) {
+        protected MapMarker(OperatorPlan plan) throws FrontendException {
             super(plan, new ReverseDependencyOrderWalker(plan));
             inputUids = new HashMap<Long,Set<String>>();
         }
         
         @Override
-        public void visit(LOLoad load) throws IOException {
+        public void visit(LOLoad load) throws FrontendException {
             if( load.getSchema() != null ) {
                 Map<Integer,Set<String>> annotation = new HashMap<Integer,Set<String>>();
                 for( int i=0; i<load.getSchema().size(); i++) {
@@ -217,7 +217,7 @@ public class MapKeysPruneHelper {
         }
 
         @Override
-        public void visit(LOFilter filter) throws IOException {
+        public void visit(LOFilter filter) throws FrontendException {
             currentOp = filter;
             MapExprMarker v = (MapExprMarker) getVisitor(filter.getFilterPlan());
             v.visit();
@@ -225,7 +225,7 @@ public class MapKeysPruneHelper {
         }
         
         @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) {
@@ -236,7 +236,7 @@ public class MapKeysPruneHelper {
         }
         
         @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 ) {
@@ -247,7 +247,7 @@ public class MapKeysPruneHelper {
         }
         
         @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) {
@@ -269,7 +269,7 @@ public class MapKeysPruneHelper {
         }
 
         @Override
-        protected LogicalExpressionVisitor getVisitor(LogicalExpressionPlan expr) {
+        protected LogicalExpressionVisitor getVisitor(LogicalExpressionPlan expr) throws FrontendException {
             return new MapExprMarker(expr );
         }
         
@@ -277,13 +277,13 @@ public class MapKeysPruneHelper {
 
             Map<Long,Set<String>> inputUids = null;
             
-            protected MapExprMarker(OperatorPlan p) {
+            protected MapExprMarker(OperatorPlan p) throws FrontendException {
                 super(p, new DependencyOrderWalker(p));
                 inputUids = new HashMap<Long,Set<String>>();
             }
 
             @Override
-            public void visit(MapLookupExpression op) throws IOException {
+            public void visit(MapLookupExpression op) throws FrontendException {
                 Long uid = op.getMap().getFieldSchema().uid;
                 String key = op.getLookupKey();
                 

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/MergeFilter.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/MergeFilter.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/MergeFilter.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/MergeFilter.java Thu Aug  5 19:38:51 2010
@@ -17,10 +17,10 @@
  */
 package org.apache.pig.newplan.logical.rules;
 
-import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.util.Pair;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
@@ -38,7 +38,7 @@ import org.apache.pig.newplan.optimizer.
 public class MergeFilter extends Rule {
 
     public MergeFilter(String n) {
-        super(n);       
+        super(n, false);       
     }
 
     @Override
@@ -51,7 +51,7 @@ public class MergeFilter extends Rule {
         private OperatorSubPlan subPlan;
 
         @Override
-        public boolean check(OperatorPlan matched) throws IOException {           
+        public boolean check(OperatorPlan matched) throws FrontendException {           
             LOFilter filter = (LOFilter)matched.getSources().get(0);
             List<Operator> succeds = currentPlan.getSuccessors(filter);
             // if this filter is followed by another filter, we should combine them
@@ -64,7 +64,7 @@ public class MergeFilter extends Rule {
         }
 
         @Override
-        public void transform(OperatorPlan matched) throws IOException {     
+        public void transform(OperatorPlan matched) throws FrontendException {     
             subPlan = new OperatorSubPlan(currentPlan);
             
             LOFilter filter = (LOFilter)matched.getSources().get(0);
@@ -103,7 +103,7 @@ public class MergeFilter extends Rule {
         
         // combine the condition of two filters. The condition of second filter
         // is added into the condition of first filter with an AND operator.
-        private void combineFilterCond(LOFilter f1, LOFilter f2) throws IOException {
+        private void combineFilterCond(LOFilter f1, LOFilter f2) throws FrontendException {
             LogicalExpressionPlan p1 = f1.getFilterPlan();
             LogicalExpressionPlan p2 = f2.getFilterPlan();
             LogicalExpressionPlan andPlan = new LogicalExpressionPlan();

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/PushUpFilter.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/PushUpFilter.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/PushUpFilter.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/PushUpFilter.java Thu Aug  5 19:38:51 2010
@@ -17,12 +17,12 @@
  */
 package org.apache.pig.newplan.logical.rules;
 
-import java.io.IOException;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.util.Pair;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
@@ -40,7 +40,7 @@ import org.apache.pig.newplan.optimizer.
 public class PushUpFilter extends Rule {
     
     public PushUpFilter(String n) {
-        super(n);       
+        super(n, false);       
     }
 
     @Override
@@ -53,7 +53,7 @@ public class PushUpFilter extends Rule {
         private OperatorSubPlan subPlan;
 
         @Override
-        public boolean check(OperatorPlan matched) throws IOException {   
+        public boolean check(OperatorPlan matched) throws FrontendException {   
             // check if it is inner join
             LOJoin join = (LOJoin)matched.getSources().get(0);
             boolean[] innerFlags = join.getInnerFlags();
@@ -100,7 +100,7 @@ public class PushUpFilter extends Rule {
         }
 
         @Override
-        public void transform(OperatorPlan matched) throws IOException {
+        public void transform(OperatorPlan matched) throws FrontendException {
             subPlan = new OperatorSubPlan(currentPlan);
 
             LOJoin join = (LOJoin)matched.getSources().get(0);
@@ -170,7 +170,7 @@ public class PushUpFilter extends Rule {
         }
         
         // check if a relational operator contains all of the specified uids
-        private boolean hasAll(LogicalRelationalOperator op, Set<Long> uids) {
+        private boolean hasAll(LogicalRelationalOperator op, Set<Long> uids) throws FrontendException {
             LogicalSchema schema = op.getSchema();
             for(long uid: uids) {
                 if (schema.findField(uid) == -1) {

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/SplitFilter.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/SplitFilter.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/SplitFilter.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/SplitFilter.java Thu Aug  5 19:38:51 2010
@@ -17,10 +17,10 @@
  */
 package org.apache.pig.newplan.logical.rules;
 
-import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.util.Pair;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
@@ -38,7 +38,7 @@ import org.apache.pig.newplan.optimizer.
 public class SplitFilter extends Rule {    
 
     public SplitFilter(String n) {
-        super(n);       
+        super(n, false);       
     }
 
     @Override
@@ -50,7 +50,7 @@ public class SplitFilter extends Rule { 
         private OperatorSubPlan subPlan;
 
         @Override
-        public boolean check(OperatorPlan matched) throws IOException {
+        public boolean check(OperatorPlan matched) throws FrontendException {
             LOFilter filter = (LOFilter)matched.getSources().get(0);
             LogicalExpressionPlan cond = filter.getFilterPlan();
             LogicalExpression root = (LogicalExpression) cond.getSources().get(0);
@@ -62,7 +62,7 @@ public class SplitFilter extends Rule { 
         }
 
         @Override
-        public void transform(OperatorPlan matched) throws IOException {
+        public void transform(OperatorPlan matched) throws FrontendException {
             subPlan = new OperatorSubPlan(currentPlan);
             
             // split one LOFilter into 2 by "AND"
@@ -85,20 +85,16 @@ public class SplitFilter extends Rule { 
             currentPlan.add(filter2);
             
             Operator succed = null;
-            try {
-                List<Operator> succeds = currentPlan.getSuccessors(filter);
-                if (succeds != null) {
-                    succed = succeds.get(0);
-                    subPlan.add(succed);
-                    Pair<Integer, Integer> p = currentPlan.disconnect(filter, succed);
-                    currentPlan.connect(filter2, 0, succed, p.second);
-                    currentPlan.connect(filter, p.first, filter2, 0); 
-                } else {
-                    currentPlan.connect(filter, 0, filter2, 0); 
-                }
-            }catch(Exception e) {
-                throw new IOException(e);
-            }                       
+            List<Operator> succeds = currentPlan.getSuccessors(filter);
+            if (succeds != null) {
+                succed = succeds.get(0);
+                subPlan.add(succed);
+                Pair<Integer, Integer> p = currentPlan.disconnect(filter, succed);
+                currentPlan.connect(filter2, 0, succed, p.second);
+                currentPlan.connect(filter, p.first, filter2, 0); 
+            } else {
+                currentPlan.connect(filter, 0, filter2, 0); 
+            }
             
             subPlan.add(filter);
             subPlan.add(filter2);
@@ -117,7 +113,7 @@ public class SplitFilter extends Rule { 
         }
         
         private void fillSubPlan(OperatorPlan origPlan, 
-                OperatorPlan subPlan, Operator startOp) throws IOException {
+                OperatorPlan subPlan, Operator startOp) throws FrontendException {
                        
             List<Operator> l = origPlan.getSuccessors(startOp);
             if (l != null) {

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/TypeCastInserter.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/TypeCastInserter.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/TypeCastInserter.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/TypeCastInserter.java Thu Aug  5 19:38:51 2010
@@ -17,12 +17,12 @@
  */
 package org.apache.pig.newplan.logical.rules;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.pig.FuncSpec;
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.streaming.StreamingCommand;
 import org.apache.pig.impl.streaming.StreamingCommand.HandleSpec;
 import org.apache.pig.impl.util.Pair;
@@ -47,7 +47,7 @@ public class TypeCastInserter extends Ru
     private String operatorClassName;
     
     public TypeCastInserter(String n, String operatorClassName) {
-        super(n);
+        super(n, true);
         this.operatorClassName = operatorClassName;
     }
 
@@ -67,7 +67,7 @@ public class TypeCastInserter extends Ru
     
     public class TypeCastInserterTransformer extends Transformer {
         @Override
-        public boolean check(OperatorPlan matched) throws IOException {
+        public boolean check(OperatorPlan matched) throws FrontendException {
             LogicalRelationalOperator op = (LogicalRelationalOperator)matched.getSources().get(0);
             LogicalSchema s = op.getSchema();
             if (s == null) return false;
@@ -97,7 +97,7 @@ public class TypeCastInserter extends Ru
         }
 
         @Override
-        public void transform(OperatorPlan matched) throws IOException {
+        public void transform(OperatorPlan matched) throws FrontendException {
             LogicalRelationalOperator op = (LogicalRelationalOperator)matched.getSources().get(0);
             LogicalSchema s = op.getSchema();
             // For every field, build a logical plan.  If the field has a type
@@ -155,7 +155,7 @@ public class TypeCastInserter extends Ru
                         loadFuncSpec = new FuncSpec(streamOutputSpec.getSpec());
                     } else {
                         String msg = "TypeCastInserter invoked with an invalid operator class name: " + innerPlan.getClass().getSimpleName();
-                        throw new IOException(msg);
+                        throw new FrontendException(msg, 2242);
                     }
                     cast.setFuncSpec(loadFuncSpec);
                 }

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/WholePlanRule.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/WholePlanRule.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/WholePlanRule.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/logical/rules/WholePlanRule.java Thu Aug  5 19:38:51 2010
@@ -31,8 +31,8 @@ import org.apache.pig.newplan.optimizer.
  */
 public abstract class WholePlanRule extends Rule {
 
-    public WholePlanRule(String n) {
-        super(n);
+    public WholePlanRule(String n, boolean mandatory) {
+        super(n, mandatory);
     }
 
     public List<OperatorPlan> match(OperatorPlan plan) {

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/PlanOptimizer.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/PlanOptimizer.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/PlanOptimizer.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/PlanOptimizer.java Thu Aug  5 19:38:51 2010
@@ -18,12 +18,11 @@
 
 package org.apache.pig.newplan.optimizer;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.pig.impl.plan.optimizer.OptimizerException;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.OperatorPlan;
 
 /**
@@ -91,9 +90,9 @@ public abstract class PlanOptimizer {
      * method of the associated Transformer to give the it a chance to
      * check whether it really wants to do the optimization.  If that
      * returns true as well, then Transformer.transform is called. 
-     * @throws OptimizerException
+     * @throws FrontendException
      */
-    public void optimize() throws IOException {
+    public void optimize() throws FrontendException {
 
         for (Set<Rule> rs : ruleSets) {
             boolean sawMatch = false;

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/PlanTransformListener.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/PlanTransformListener.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/PlanTransformListener.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/PlanTransformListener.java Thu Aug  5 19:38:51 2010
@@ -18,8 +18,7 @@
 
 package org.apache.pig.newplan.optimizer;
 
-import java.io.IOException;
-
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.OperatorPlan;
 
 /**
@@ -30,8 +29,8 @@ public interface PlanTransformListener {
      * the listener that is notified after a plan is transformed
      * @param fp  the full plan that has been transformed
      * @param tp  a plan containing only the operators that have been transformed
-     * @throws IOException 
+     * @throws FrontendException 
      */
-    public void transformed(OperatorPlan fp, OperatorPlan tp) throws IOException;
+    public void transformed(OperatorPlan fp, OperatorPlan tp) throws FrontendException;
 
 }

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/Rule.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/Rule.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/Rule.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/Rule.java Thu Aug  5 19:38:51 2010
@@ -18,7 +18,6 @@
 
 package org.apache.pig.newplan.optimizer;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -28,6 +27,7 @@ import java.util.Stack;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.OperatorSubPlan;
@@ -45,14 +45,16 @@ public abstract class Rule {
     transient protected OperatorPlan currentPlan;
     protected static final Log log = LogFactory.getLog(Rule.class);
     private transient Set<Operator> matchedNodes = new HashSet<Operator>();
+    private boolean mandatory;
     
     /**
      * Create this rule by using the default pattern that this rule provided
      * @param n Name of this rule
      */
-    public Rule(String n) {
+    public Rule(String n, boolean mandatory) {
         name = n;    
         pattern = buildPattern();
+        this.mandatory = mandatory;
     }
     
     /**
@@ -100,7 +102,7 @@ public abstract class Rule {
      *        
      * @param plan the OperatorPlan to look for matches to the pattern
      */
-    public List<OperatorPlan> match(OperatorPlan plan) {
+    public List<OperatorPlan> match(OperatorPlan plan) throws FrontendException {
         currentPlan = plan;
         
         List<Operator> leaves = pattern.getSinks();
@@ -124,11 +126,7 @@ public abstract class Rule {
                     
                     
                     List<Operator> preds = null;
-                    try {
-                        preds = plan.getPredecessors(op);
-                    }catch(IOException e) {
-                        // not going to happen
-                    }
+                    preds = plan.getPredecessors(op);
                     
                     // if this node has no predecessor, it must be a root
                     if (preds == null) {
@@ -139,17 +137,13 @@ public abstract class Rule {
                     for(Operator s: preds) {
                         matched = true;
                         List<Operator> siblings = null;
-                        try {
-                            if (s != null) {
-                                siblings = plan.getSuccessors(s);
-                            }else{
-                                // for a root, we get its siblings by getting all roots
-                                siblings = plan.getSources();
-                            }
-                        }catch(IOException e) {
-                            // not going to happen
-                            throw new RuntimeException(e);
+                        if (s != null) {
+                            siblings = plan.getSuccessors(s);
+                        }else{
+                            // for a root, we get its siblings by getting all roots
+                            siblings = plan.getSources();
                         }
+
                         int index = siblings.indexOf(op);
                         if (siblings.size()-index < leaves.size()) {
                             continue;
@@ -181,21 +175,17 @@ public abstract class Rule {
                 
               
                 PatternMatchOperatorPlan match = new PatternMatchOperatorPlan(plan);
-                try {
-                    if (match.check(planOps)) {
-                        // we find a matched pattern,
-                        // add the operators into matchedNodes
-                        Iterator<Operator> iter2 = match.getOperators();                      
-                        while(iter2.hasNext()) {
-                            Operator opt = iter2.next();
-                            matchedNodes.add(opt);                        
-                        }
-                        
-                        // add pattern
-                        matchedList.add(match);                                                
+                if (match.check(planOps)) {
+                    // we find a matched pattern,
+                    // add the operators into matchedNodes
+                    Iterator<Operator> iter2 = match.getOperators();                      
+                    while(iter2.hasNext()) {
+                        Operator opt = iter2.next();
+                        matchedNodes.add(opt);                        
                     }
-                }catch(IOException e) {
-                    log.error("Failed to search for optmization pattern. ", e);
+                    
+                    // add pattern
+                    matchedList.add(match);                                                
                 }
             }
         }
@@ -207,6 +197,10 @@ public abstract class Rule {
         return name;
     }
     
+    public boolean isMandatory() {
+        return mandatory;
+    }
+    
     /** 
      * Check if two operators match each other, we want to find matches
      * that don't share nodes
@@ -222,7 +216,7 @@ public abstract class Rule {
             super(basePlan);
         }    	    	
         
-        protected boolean check(List<Operator> planOps) throws IOException {
+        protected boolean check(List<Operator> planOps) throws FrontendException {
             List<Operator> patternOps = pattern.getSinks();
             if (planOps.size() != patternOps.size()) {
                 return false;
@@ -257,7 +251,7 @@ public abstract class Rule {
          * if we are looking for join->load pattern, only one match will be returned instead
          * of two, so that the matched subsets don't share nodes.
          */ 
-        private boolean check(Operator planOp, Operator patternOp, Stack<Operator> opers) throws IOException {
+        private boolean check(Operator planOp, Operator patternOp, Stack<Operator> opers) throws FrontendException {
             if (!match(planOp, patternOp)) {
                 return false;
             }

Modified: hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/Transformer.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/Transformer.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/Transformer.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/newplan/optimizer/Transformer.java Thu Aug  5 19:38:51 2010
@@ -18,8 +18,7 @@
 
 package org.apache.pig.newplan.optimizer;
 
-import java.io.IOException;
-
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.OperatorPlan;
 
 public abstract class Transformer {
@@ -32,18 +31,18 @@ public abstract class Transformer {
      *        subset has the same graph as the pattern, but the operators
      *        point to the same objects as the plan to be matched.
      * @return true if the transform should be done.
-     * @throws IOException
+     * @throws Transformer
      */
-    public abstract boolean check(OperatorPlan matched) throws IOException;
+    public abstract boolean check(OperatorPlan matched) throws FrontendException;
 
     /**
      * Transform the tree
      * @param matched the sub-set of the plan that matches the pattern. This 
      *        subset has the same graph as the pattern, but the operators
      *        point to the same objects as the plan to be matched.
-     * @throws IOException
+     * @throws Transformer
      */
-    public abstract void transform(OperatorPlan matched) throws IOException;
+    public abstract void transform(OperatorPlan matched) throws FrontendException;
     
     /**
      * Report what parts of the tree were transformed.  This is so that 

Modified: hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/JobStats.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/JobStats.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/JobStats.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/JobStats.java Thu Aug  5 19:38:51 2010
@@ -49,6 +49,7 @@ import org.apache.pig.classification.Int
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.PlanVisitor;
 import org.apache.pig.impl.io.FileSpec;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.util.ObjectSerializer;
 import org.apache.pig.tools.pigstats.PigStats.JobGraph;
 import org.apache.pig.tools.pigstats.PigStats.JobGraphPrinter;
@@ -215,7 +216,7 @@ public final class JobStats extends Oper
     }
     
     @Override
-    public void accept(PlanVisitor v) throws IOException {
+    public void accept(PlanVisitor v) throws FrontendException {
         if (v instanceof JobGraphPrinter) {
             JobGraphPrinter jpp = (JobGraphPrinter)v;
             jpp.visit(this);

Modified: hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/PigStats.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/PigStats.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/PigStats.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/PigStats.java Thu Aug  5 19:38:51 2010
@@ -17,7 +17,6 @@
  */
 package org.apache.pig.tools.pigstats;
 
-import java.io.IOException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -44,6 +43,7 @@ import org.apache.pig.classification.Int
 import org.apache.pig.classification.InterfaceStability;
 import org.apache.pig.data.InternalCachedBag;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.plan.DependencyOrderWalker;
 import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.util.SpillableMemoryManager;
@@ -111,7 +111,7 @@ public final class PigStats {
             JobGraphPrinter jp = new JobGraphPrinter(this);
             try {
                 jp.visit();
-            } catch (IOException e) {
+            } catch (FrontendException e) {
                 LOG.warn("unable to print job plan", e);
             }
             return jp.toString();
@@ -135,11 +135,7 @@ public final class PigStats {
  
         boolean isConnected(Operator from, Operator to) {
             List<Operator> succs = null;
-            try {
-                succs = getSuccessors(from);
-            } catch (IOException e) {
-                LOG.warn("unable to get successors for operator");
-            }
+            succs = getSuccessors(from);
             if (succs != null) {
                 for (Operator succ: succs) {
                     if (succ.getName().equals(to.getName()) 
@@ -221,7 +217,7 @@ public final class PigStats {
             buf = new StringBuffer();
         }
         
-        public void visit(JobStats op) throws IOException {
+        public void visit(JobStats op) throws FrontendException {
             buf.append(op.getJobId());
             List<Operator> succs = plan.getSuccessors(op);
             if (succs != null) {

Modified: hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/ScriptState.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/ScriptState.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/ScriptState.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/tools/pigstats/ScriptState.java Thu Aug  5 19:38:51 2010
@@ -411,20 +411,15 @@ public class ScriptState {
             }
         }
         if (js != null) {
-            try {
-                List<Operator> preds = jg.getPredecessors(js);
-                if (preds != null) {
-                    StringBuilder sb = new StringBuilder();
-                    for (Operator op : preds) {
-                        JobStats job = (JobStats)op;
-                        if (sb.length() > 0) sb.append(",");
-                        sb.append(job.getJobId());
-                    }
-                    conf.set(PIG_PROPERTY.JOB_PARENTS.toString(), sb.toString());
+            List<Operator> preds = jg.getPredecessors(js);
+            if (preds != null) {
+                StringBuilder sb = new StringBuilder();
+                for (Operator op : preds) {
+                    JobStats job = (JobStats)op;
+                    if (sb.length() > 0) sb.append(",");
+                    sb.append(job.getJobId());
                 }
-            } catch (IOException e) {
-                LOG.warn("unable to get job predecessors for job "
-                        + js.getJobId(), e);
+                conf.set(PIG_PROPERTY.JOB_PARENTS.toString(), sb.toString());
             }
         }
     }

Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanColumnPrune.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanColumnPrune.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanColumnPrune.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanColumnPrune.java Thu Aug  5 19:38:51 2010
@@ -29,14 +29,17 @@ import junit.framework.TestCase;
 
 import org.apache.pig.ExecType;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
 import org.apache.pig.newplan.logical.LogicalPlanMigrationVistor;
 import org.apache.pig.newplan.logical.optimizer.LogicalPlanOptimizer;
+import org.apache.pig.newplan.logical.optimizer.SchemaResetter;
 import org.apache.pig.newplan.logical.relational.LOLoad;
 import org.apache.pig.newplan.logical.relational.LogicalPlan;
 import org.apache.pig.newplan.logical.relational.LogicalRelationalOperator;
+import org.apache.pig.newplan.logical.rules.AddForEach;
 import org.apache.pig.newplan.logical.rules.ColumnMapKeyPrune;
 import org.apache.pig.newplan.logical.rules.MapKeysPruneHelper;
 import org.apache.pig.newplan.optimizer.PlanOptimizer;
@@ -48,11 +51,14 @@ public class TestNewPlanColumnPrune exte
     LogicalPlan plan = null;
     PigContext pc = new PigContext(ExecType.LOCAL, new Properties());
   
-    private LogicalPlan migratePlan(org.apache.pig.impl.logicalLayer.LogicalPlan lp) throws VisitorException{
+    private LogicalPlan migratePlan(org.apache.pig.impl.logicalLayer.LogicalPlan lp) throws FrontendException{
         LogicalPlanMigrationVistor visitor = new LogicalPlanMigrationVistor(lp);        
         visitor.visit();
         org.apache.pig.newplan.logical.relational.LogicalPlan newPlan = visitor.getNewLogicalPlan();
         
+        SchemaResetter schemaResetter = new SchemaResetter(newPlan);
+        schemaResetter.visit();
+        
         return newPlan;
     }
     
@@ -127,7 +133,8 @@ public class TestNewPlanColumnPrune exte
         lpt = new LogicalPlanTester(pc);
         lpt.buildPlan("a = load 'd.txt' as (id, v1, v3, v2);");
         lpt.buildPlan("b = filter a by v1 != NULL AND (v2+v3)<100;");
-        lpt.buildPlan("c = foreach b generate id;");
+        lpt.buildPlan("b1 = foreach b generate id;");
+        lpt.buildPlan("c = foreach b1 generate id;");
         plan = lpt.buildPlan("store c into 'empty';"); 
         expected = migratePlan(plan);
         assertTrue(expected.isEqual(newLogicalPlan));
@@ -186,7 +193,8 @@ public class TestNewPlanColumnPrune exte
         lpt.buildPlan("a = load 'd.txt' as (v5, v4, v2);");
         lpt.buildPlan("b = foreach a generate v2, v5, v4;");
         lpt.buildPlan("c = filter b by v2 != NULL;");
-        lpt.buildPlan("d = foreach c generate v5, v4;");
+        lpt.buildPlan("c1 = foreach c generate v5, v4;");
+        lpt.buildPlan("d = foreach c1 generate v5, v4;");
         plan = lpt.buildPlan("store d into 'empty';");  
         expected = migratePlan(plan);
         assertTrue(expected.isEqual(newLogicalPlan));
@@ -206,8 +214,9 @@ public class TestNewPlanColumnPrune exte
         lpt = new LogicalPlanTester(pc);
         lpt.buildPlan("a = load 'd.txt' as (id, v3);");
         lpt.buildPlan("b = load 'c.txt' as (id, v4, v5);");
-        lpt.buildPlan("c = join a by id, b by id;");       
-        lpt.buildPlan("d = foreach c generate a::id, v5, v3, v4;");
+        lpt.buildPlan("c = join a by id, b by id;");
+        lpt.buildPlan("c1 = foreach c generate a::id, v3, v4, v5;");  
+        lpt.buildPlan("d = foreach c1 generate a::id, v5, v3, v4;");
         plan = lpt.buildPlan("store d into 'empty';");  
         expected = migratePlan(plan);
         assertTrue(expected.isEqual(newLogicalPlan));
@@ -227,7 +236,8 @@ public class TestNewPlanColumnPrune exte
         lpt.buildPlan("a = load 'd.txt' using BinStorage() as (id, v1, v5, v3, v4, v2);");
         lpt.buildPlan("b = foreach a generate v5, v4, v2;");
         lpt.buildPlan("c = filter b by v2 != NULL;");
-        lpt.buildPlan("d = foreach c generate v5, v4;");
+        lpt.buildPlan("c1 = foreach c generate v5, v4;");
+        lpt.buildPlan("d = foreach c1 generate v5, v4;");
         plan = lpt.buildPlan("store d into 'empty';");  
         expected = migratePlan(plan);
         assertTrue(expected.isEqual(newLogicalPlan));
@@ -248,7 +258,8 @@ public class TestNewPlanColumnPrune exte
         lpt.buildPlan("a = load 'd.txt' using BinStorage() as (id, v1, v5, v3, v4, v2);");
         lpt.buildPlan("b = foreach a generate v5, v2;");
         lpt.buildPlan("c = filter b by v2 != NULL;");
-        lpt.buildPlan("d = foreach c generate v5;");
+        lpt.buildPlan("c1 = foreach c generate v5;");
+        lpt.buildPlan("d = foreach c1 generate v5;");
         plan = lpt.buildPlan("store d into 'empty';");  
         expected = migratePlan(plan);
         assertTrue(expected.isEqual(newLogicalPlan));
@@ -269,7 +280,8 @@ public class TestNewPlanColumnPrune exte
         lpt.buildPlan("a = load 'd.txt' using BinStorage() as (id, v1, v5, v3, v4, v2);");
         lpt.buildPlan("b = foreach a generate v5, v2, 10;");
         lpt.buildPlan("c = filter b by v2 != NULL;");
-        lpt.buildPlan("d = foreach c generate v5;");
+        lpt.buildPlan("c1 = foreach c generate v5;");
+        lpt.buildPlan("d = foreach c1 generate v5;");
         plan = lpt.buildPlan("store d into 'empty';");  
         expected = migratePlan(plan);
         assertTrue(expected.isEqual(newLogicalPlan));
@@ -380,7 +392,6 @@ public class TestNewPlanColumnPrune exte
         assertTrue(expected.isEqual(newLogicalPlan));
     }
     
-    /*
     public void testAddForeach() throws Exception  {
         // filter above foreach
         LogicalPlanTester lpt = new LogicalPlanTester(pc);
@@ -396,7 +407,8 @@ public class TestNewPlanColumnPrune exte
         lpt = new LogicalPlanTester(pc);
         lpt.buildPlan("a = load 'd.txt' as (id, v1);");
         lpt.buildPlan("b = filter a by v1>10;");
-        lpt.buildPlan("c = foreach b generate id;");      
+        lpt.buildPlan("b1 = foreach b generate id;"); 
+        lpt.buildPlan("c = foreach b1 generate id;");      
         plan = lpt.buildPlan("store c into 'empty';");  
         LogicalPlan expected = migratePlan(plan);
         
@@ -421,17 +433,18 @@ public class TestNewPlanColumnPrune exte
         lpt.buildPlan("c = join a by id, b by id;");
         lpt.buildPlan("d = foreach c generate a::id, a::v1, b::v1;");        
         lpt.buildPlan("e = filter d by a::v1>b::v1;");
-        lpt.buildPlan("f = foreach e generate a::id;");        
+        lpt.buildPlan("e1 = foreach e generate a::id;");  
+        lpt.buildPlan("f = foreach e1 generate a::id;");        
         plan = lpt.buildPlan("store f into 'empty';");  
         expected = migratePlan(plan);
         
         assertTrue(expected.isEqual(newLogicalPlan));
-    }*/
+    }
     
     public class MyPlanOptimizer extends LogicalPlanOptimizer {
 
         protected MyPlanOptimizer(OperatorPlan p,  int iterations) {
-            super(p, iterations);			
+            super(p, iterations, null);			
         }
         
         protected List<Set<Rule>> buildRuleSets() {            
@@ -442,6 +455,11 @@ public class TestNewPlanColumnPrune exte
             s.add(r);            
             ls.add(s);
             
+            r = new AddForEach("AddForEach");
+            s = new HashSet<Rule>();
+            s.add(r);            
+            ls.add(s);
+            
             return ls;
         }
     }    

Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanColumnPrune2.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanColumnPrune2.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanColumnPrune2.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanColumnPrune2.java Thu Aug  5 19:38:51 2010
@@ -31,18 +31,13 @@ import org.apache.log4j.FileAppender;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.log4j.SimpleLayout;
-import org.apache.pig.ExecType;
 import org.apache.pig.FilterFunc;
-import org.apache.pig.LoadPushDown;
 import org.apache.pig.PigServer;
-import org.apache.pig.builtin.BinStorage;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.FrontendException;
-import org.apache.pig.impl.logicalLayer.optimizer.PruneColumns;
 import org.apache.pig.newplan.logical.expression.LogicalExpression;
 import org.apache.pig.newplan.logical.rules.ColumnPruneVisitor;
-import org.apache.pig.newplan.optimizer.Rule;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;

Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanFilterRule.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanFilterRule.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanFilterRule.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanFilterRule.java Thu Aug  5 19:38:51 2010
@@ -18,9 +18,9 @@
 
 package org.apache.pig.test;
 
-import java.io.IOException;
 import java.util.*;
 import org.apache.pig.data.DataType;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.util.MultiMap;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
@@ -482,7 +482,7 @@ public class TestNewPlanFilterRule exten
 
         @Override
         public void transformed(OperatorPlan fp, OperatorPlan tp)
-                throws IOException {
+                throws FrontendException {
             this.tp = tp;
         }
         

Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanListener.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanListener.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanListener.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanListener.java Thu Aug  5 19:38:51 2010
@@ -18,10 +18,10 @@
 
 package org.apache.pig.test;
 
-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.impl.util.MultiMap;
 import org.apache.pig.newplan.DepthFirstWalker;
 import org.apache.pig.newplan.Operator;
@@ -147,12 +147,12 @@ public class TestNewPlanListener extends
     private static class SillySameVisitor extends AllSameRalationalNodesVisitor {
         StringBuffer buf = new StringBuffer();
 
-        SillySameVisitor(OperatorPlan plan) {
+        SillySameVisitor(OperatorPlan plan) throws FrontendException {
             super(plan, new DepthFirstWalker(plan));
         }
         
         @Override
-        protected void execute(LogicalRelationalOperator op) throws IOException {
+        protected void execute(LogicalRelationalOperator op) throws FrontendException {
             buf.append(op.getName());
             buf.append(" ");
         }
@@ -167,7 +167,7 @@ public class TestNewPlanListener extends
     // Test that the AllSameVisitor calls execute on every node
     // in the plan.
     @Test
-    public void testAllSameVisitor() throws IOException {
+    public void testAllSameVisitor() throws FrontendException {
         SillySameVisitor v = new SillySameVisitor(lp);
         v.visit();
         assertTrue("LOLoad LOJoin LOLoad LOFilter ".equals(v.toString()) ||
@@ -178,28 +178,28 @@ public class TestNewPlanListener extends
     private static class SillyExpressionVisitor extends LogicalExpressionVisitor {
         StringBuffer buf;
 
-        protected SillyExpressionVisitor(OperatorPlan p, StringBuffer b) {
+        protected SillyExpressionVisitor(OperatorPlan p, StringBuffer b) throws FrontendException {
             super(p, new DepthFirstWalker(p));
             buf = b;
         }
         
         @Override
-        public void visit(AndExpression andExpr) throws IOException {
+        public void visit(AndExpression andExpr) throws FrontendException {
             buf.append("and ");
         }
         
         @Override
-        public void visit(EqualExpression equal) throws IOException {
+        public void visit(EqualExpression equal) throws FrontendException {
             buf.append("equal ");
         }
         
         @Override
-        public void visit(ProjectExpression p) throws IOException {
+        public void visit(ProjectExpression p) throws FrontendException {
             buf.append("proj ");
         }
         
         @Override
-        public void visit(ConstantExpression c) throws IOException {
+        public void visit(ConstantExpression c) throws FrontendException {
             buf.append("const ");
         }
     }
@@ -207,13 +207,13 @@ public class TestNewPlanListener extends
     private static class SillyAllExpressionVisitor extends AllExpressionVisitor {
         StringBuffer buf = new StringBuffer();
 
-        public SillyAllExpressionVisitor(OperatorPlan plan) {
+        public SillyAllExpressionVisitor(OperatorPlan plan) throws FrontendException {
             super(plan, new DepthFirstWalker(plan));
         }
      
 
         @Override
-        protected LogicalExpressionVisitor getVisitor(LogicalExpressionPlan expr) {
+        protected LogicalExpressionVisitor getVisitor(LogicalExpressionPlan expr) throws FrontendException {
             return new SillyExpressionVisitor(expr, buf);
         }   
         
@@ -226,7 +226,7 @@ public class TestNewPlanListener extends
     // Test that the AllExpressionVisitor executes on every
     // expression in the plan
     @Test
-    public void testAllExpressionVisitor() throws IOException {
+    public void testAllExpressionVisitor() throws FrontendException {
         SillyAllExpressionVisitor v = new SillyAllExpressionVisitor(lp);
         v.visit();
         assertTrue("proj proj equal proj const ".equals(v.toString()) ||
@@ -235,7 +235,7 @@ public class TestNewPlanListener extends
     
     // Test that schemas are patched up after a transform
     @Test
-    public void testSchemaPatcher() throws IOException {
+    public void testSchemaPatcher() throws FrontendException {
         SchemaPatcher patcher = new SchemaPatcher();
         patcher.transformed(lp, changedPlan);
         
@@ -253,7 +253,7 @@ public class TestNewPlanListener extends
     
     // Test that projections are patched up after a transform
     @Test
-    public void testProjectionPatcher() throws IOException {
+    public void testProjectionPatcher() throws FrontendException {
         ProjectionPatcher patcher = new ProjectionPatcher();
         patcher.transformed(lp, changedPlan);
         

Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanLogicalOptimizer.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanLogicalOptimizer.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanLogicalOptimizer.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanLogicalOptimizer.java Thu Aug  5 19:38:51 2010
@@ -127,7 +127,7 @@ public class TestNewPlanLogicalOptimizer
         }
         
         System.out.println(lp);
-        LogicalPlanOptimizer optimizer = new LogicalPlanOptimizer(lp, 500);
+        LogicalPlanOptimizer optimizer = new LogicalPlanOptimizer(lp, 500, null);
         optimizer.optimize();
         
         LogicalPlan expected = new LogicalPlan();

Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanOperatorPlan.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanOperatorPlan.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanOperatorPlan.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanOperatorPlan.java Thu Aug  5 19:38:51 2010
@@ -18,13 +18,13 @@
 
 package org.apache.pig.test;
 
-import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 
 import org.apache.pig.FuncSpec;
 import org.apache.pig.data.DataType;
 import org.apache.pig.impl.io.FileSpec;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.util.MultiMap;
 import org.apache.pig.impl.util.Pair;
 import org.apache.pig.newplan.BaseOperatorPlan;
@@ -170,7 +170,7 @@ public class TestNewPlanOperatorPlan ext
     
     // Test OperatorPlan
     @Test
-    public void testOperatorPlan() throws IOException {
+    public void testOperatorPlan() throws FrontendException {
         SillyPlan plan = new SillyPlan();
         SillyOperator fred = new SillyOperator("fred", plan);
         SillyOperator joe = new SillyOperator("joe", plan);
@@ -291,7 +291,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testDisconnectAndRemove() throws IOException {
+    public void testDisconnectAndRemove() throws FrontendException {
         SillyPlan plan = new SillyPlan();
         SillyOperator fred = new SillyOperator("fred", plan);
         SillyOperator joe = new SillyOperator("joe", plan);
@@ -337,7 +337,7 @@ public class TestNewPlanOperatorPlan ext
         boolean caught = false;
         try {
             plan.remove(fred);
-        } catch (IOException e) {
+        } catch (FrontendException e) {
             caught = true;
         }
         assertTrue(caught);
@@ -345,7 +345,7 @@ public class TestNewPlanOperatorPlan ext
         caught = false;
         try {
             plan.remove(joe);
-        } catch (IOException e) {
+        } catch (FrontendException e) {
             caught = true;
         }
         assertTrue(caught);
@@ -364,7 +364,7 @@ public class TestNewPlanOperatorPlan ext
         boolean caught = false;
         try {
             plan.disconnect(fred, joe);
-        } catch (IOException e) {
+        } catch (FrontendException e) {
             caught = true;
         }
         assertTrue(caught);
@@ -374,7 +374,7 @@ public class TestNewPlanOperatorPlan ext
     // Tests for DependencyOrderWalker
     
     @Test
-    public void testDependencyOrderWalkerLinear() throws IOException {
+    public void testDependencyOrderWalkerLinear() throws FrontendException {
         SillyPlan plan = new SillyPlan();
         SillyOperator fred = new SillyOperator("fred", plan);
         SillyOperator joe = new SillyOperator("joe", plan);
@@ -398,7 +398,7 @@ public class TestNewPlanOperatorPlan ext
     }
 
     @Test
-    public void testDependencyOrderWalkerTree() throws IOException {
+    public void testDependencyOrderWalkerTree() throws FrontendException {
         SillyPlan plan = new SillyPlan();
         SillyOperator fred = new SillyOperator("fred", plan);
         SillyOperator joe = new SillyOperator("joe", plan);
@@ -434,7 +434,7 @@ public class TestNewPlanOperatorPlan ext
     }
 
     @Test
-    public void testDependencyOrderWalkerGraph() throws IOException {
+    public void testDependencyOrderWalkerGraph() throws FrontendException {
         SillyPlan plan = new SillyPlan();
         SillyOperator fred = new SillyOperator("fred", plan);
         SillyOperator joe = new SillyOperator("joe", plan);
@@ -472,7 +472,7 @@ public class TestNewPlanOperatorPlan ext
     // Tests for DepthFirstWalker
     
     @Test
-    public void testDepthFirstWalkerLinear() throws IOException {
+    public void testDepthFirstWalkerLinear() throws FrontendException {
         SillyPlan plan = new SillyPlan();
         SillyOperator fred = new SillyOperator("fred", plan);
         SillyOperator joe = new SillyOperator("joe", plan);
@@ -496,7 +496,7 @@ public class TestNewPlanOperatorPlan ext
     }
 
     @Test
-    public void testDepthFirstWalkerTree() throws IOException {
+    public void testDepthFirstWalkerTree() throws FrontendException {
         SillyPlan plan = new SillyPlan();
         SillyOperator fred = new SillyOperator("fred", plan);
         SillyOperator joe = new SillyOperator("joe", plan);
@@ -526,7 +526,7 @@ public class TestNewPlanOperatorPlan ext
     }
 
     @Test
-    public void testDepthFirstWalkerGraph() throws IOException {
+    public void testDepthFirstWalkerGraph() throws FrontendException {
         SillyPlan plan = new SillyPlan();
         SillyOperator fred = new SillyOperator("fred", plan);
         SillyOperator joe = new SillyOperator("joe", plan);
@@ -562,7 +562,7 @@ public class TestNewPlanOperatorPlan ext
     // Tests for ReverseDependencyOrderWalker
     
     @Test
-    public void testReverseDependencyOrderWalkerLinear() throws IOException {
+    public void testReverseDependencyOrderWalkerLinear() throws FrontendException {
         SillyPlan plan = new SillyPlan();
         SillyOperator fred = new SillyOperator("fred", plan);
         SillyOperator joe = new SillyOperator("joe", plan);
@@ -586,7 +586,7 @@ public class TestNewPlanOperatorPlan ext
     }
 
     @Test
-    public void testReverseDependencyOrderWalkerTree() throws IOException {
+    public void testReverseDependencyOrderWalkerTree() throws FrontendException {
         SillyPlan plan = new SillyPlan();
         SillyOperator fred = new SillyOperator("fred", plan);
         SillyOperator joe = new SillyOperator("joe", plan);
@@ -626,7 +626,7 @@ public class TestNewPlanOperatorPlan ext
     }
 
     @Test
-    public void testReverseDependencyOrderWalkerGraph() throws IOException {
+    public void testReverseDependencyOrderWalkerGraph() throws FrontendException {
         SillyPlan plan = new SillyPlan();
         SillyOperator fred = new SillyOperator("fred", plan);
         SillyOperator joe = new SillyOperator("joe", plan);
@@ -665,7 +665,7 @@ public class TestNewPlanOperatorPlan ext
         
         StringBuffer bf = new StringBuffer();
 
-        protected TestLogicalVisitor(OperatorPlan plan) {
+        protected TestLogicalVisitor(OperatorPlan plan) throws FrontendException {
             super(plan, new DepthFirstWalker(plan));
         }
         
@@ -681,7 +681,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testLogicalPlanVisitor() throws IOException {
+    public void testLogicalPlanVisitor() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
         LOLoad load = new LOLoad(null, null, lp, null);
         /*lp.add((LogicalRelationalOperator)null, load,
@@ -695,7 +695,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testBinaryOperatorOrder() throws IOException {
+    public void testBinaryOperatorOrder() throws FrontendException {
         LogicalExpressionPlan ep = new LogicalExpressionPlan();
         ConstantExpression c = new ConstantExpression(ep, new Integer(5), new LogicalFieldSchema(null, null, DataType.INTEGER));
         ProjectExpression p = new ProjectExpression(ep, 0, 0, null);
@@ -709,7 +709,7 @@ public class TestNewPlanOperatorPlan ext
         
         StringBuffer bf = new StringBuffer();
 
-        protected TestExpressionVisitor(OperatorPlan plan) {
+        protected TestExpressionVisitor(OperatorPlan plan) throws FrontendException {
             super(plan, new DepthFirstWalker(plan));
         }
         
@@ -740,7 +740,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testExpressionPlanVisitor() throws IOException {
+    public void testExpressionPlanVisitor() throws FrontendException {
         LogicalExpressionPlan ep = new LogicalExpressionPlan();
         ConstantExpression c = new ConstantExpression(ep, new Integer(5), new LogicalFieldSchema(null, null, DataType.INTEGER));
         ProjectExpression p = new ProjectExpression(ep, 0, 0, null);
@@ -754,7 +754,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testExpressionEquality() {
+    public void testExpressionEquality() throws FrontendException {
         LogicalExpressionPlan ep1 = new LogicalExpressionPlan();
         ConstantExpression c1 = new ConstantExpression(ep1, new Integer(5), new LogicalFieldSchema(null, null, DataType.INTEGER));
         ProjectExpression p1 = new ProjectExpression(ep1, 0, 0, null);
@@ -798,7 +798,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testRelationalEquality() throws IOException {
+    public void testRelationalEquality() throws FrontendException {
         // Build a plan that is the logical plan for
         // A = load 'bla' as (x);
         // B = load 'morebla' as (y);
@@ -910,7 +910,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testLoadEqualityDifferentFuncSpecCtorArgs() {
+    public void testLoadEqualityDifferentFuncSpecCtorArgs() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
         
         LogicalSchema aschema1 = new LogicalSchema();
@@ -928,7 +928,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testLoadEqualityDifferentNumFuncSpecCstorArgs() {
+    public void testLoadEqualityDifferentNumFuncSpecCstorArgs() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
         
         LogicalSchema aschema1 = new LogicalSchema();
@@ -946,7 +946,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testLoadEqualityDifferentFunctionNames() {
+    public void testLoadEqualityDifferentFunctionNames() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
         
         LogicalSchema aschema1 = new LogicalSchema();
@@ -965,7 +965,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testLoadEqualityDifferentFileName() {
+    public void testLoadEqualityDifferentFileName() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
         LogicalSchema aschema1 = new LogicalSchema();
         aschema1.addField(new LogicalSchema.LogicalFieldSchema(
@@ -983,7 +983,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testRelationalEqualityDifferentSchema() {
+    public void testRelationalEqualityDifferentSchema() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
         LogicalSchema aschema1 = new LogicalSchema();
         aschema1.addField(new LogicalSchema.LogicalFieldSchema(
@@ -1005,7 +1005,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testRelationalEqualityNullSchemas() {
+    public void testRelationalEqualityNullSchemas() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
         // Test that two loads with no schema are still equal
         LOLoad load7 = new LOLoad(new FileSpec("/abc",
@@ -1020,7 +1020,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testRelationalEqualityOneNullOneNotNullSchema() {
+    public void testRelationalEqualityOneNullOneNotNullSchema() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
         LogicalSchema aschema1 = new LogicalSchema();
         aschema1.addField(new LogicalSchema.LogicalFieldSchema(
@@ -1038,7 +1038,7 @@ public class TestNewPlanOperatorPlan ext
     }
         
     @Test
-    public void testFilterDifferentPredicates() {
+    public void testFilterDifferentPredicates() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
             
         LogicalExpressionPlan fp1 = new LogicalExpressionPlan();
@@ -1071,7 +1071,7 @@ public class TestNewPlanOperatorPlan ext
     // func, and I don't want to mess with that here.
     
     @Test
-    public void testJoinDifferentJoinTypes() throws IOException {
+    public void testJoinDifferentJoinTypes() throws FrontendException {
        LogicalPlan lp = new LogicalPlan();
        LogicalSchema jaschema1 = new LogicalSchema();
        jaschema1.addField(new LogicalSchema.LogicalFieldSchema(
@@ -1148,7 +1148,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testJoinDifferentInner() throws IOException {
+    public void testJoinDifferentInner() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
                LogicalSchema jaschema1 = new LogicalSchema();
        jaschema1.addField(new LogicalSchema.LogicalFieldSchema(
@@ -1228,7 +1228,7 @@ public class TestNewPlanOperatorPlan ext
     }
  
     @Test
-    public void testJoinDifferentNumInputs() throws IOException {
+    public void testJoinDifferentNumInputs() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
                LogicalSchema jaschema1 = new LogicalSchema();
        jaschema1.addField(new LogicalSchema.LogicalFieldSchema(
@@ -1317,7 +1317,7 @@ public class TestNewPlanOperatorPlan ext
     }
         
     @Test
-    public void testJoinDifferentJoinKeys() throws IOException {
+    public void testJoinDifferentJoinKeys() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
         
         // Test different join keys
@@ -1406,7 +1406,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testJoinDifferentNumJoinKeys() throws IOException {
+    public void testJoinDifferentNumJoinKeys() throws FrontendException {
         LogicalPlan lp = new LogicalPlan();
         
         // Test different join keys
@@ -1492,7 +1492,7 @@ public class TestNewPlanOperatorPlan ext
     }
     
     @Test
-    public void testRelationalSameOpDifferentPreds() throws IOException {
+    public void testRelationalSameOpDifferentPreds() throws FrontendException {
         LogicalPlan lp1 = new LogicalPlan();
         LogicalSchema aschema1 = new LogicalSchema();
         aschema1.addField(new LogicalSchema.LogicalFieldSchema(

Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanRule.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanRule.java?rev=982747&r1=982746&r2=982747&view=diff
==============================================================================
--- hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanRule.java (original)
+++ hadoop/pig/trunk/test/org/apache/pig/test/TestNewPlanRule.java Thu Aug  5 19:38:51 2010
@@ -20,6 +20,7 @@ package org.apache.pig.test;
 
 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;
@@ -188,7 +189,7 @@ public class TestNewPlanRule extends Tes
          assertNotSame(match.getSources().get(0), match.getSources().get(1));
     }
     
-    public void testSingleNodeMatch() {
+    public void testSingleNodeMatch() throws FrontendException {
         // search for Load 
         OperatorPlan pattern = new SillyPlan();
         pattern.add(new OP_Load("mmm", pattern));
@@ -264,7 +265,7 @@ public class TestNewPlanRule extends Tes
       
     }
     
-    public void testTwoNodeMatch() {
+    public void testTwoNodeMatch() throws FrontendException {
         // search for 2 Loads at the same time 
         OperatorPlan pattern = new SillyPlan();
         pattern.add(new OP_Load("mmm1", pattern));