You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ol...@apache.org on 2009/02/13 02:59:30 UTC

svn commit: r743952 [2/4] - in /hadoop/pig/trunk: ./ src/org/apache/pig/ src/org/apache/pig/backend/executionengine/ src/org/apache/pig/backend/hadoop/ src/org/apache/pig/backend/hadoop/datastorage/ src/org/apache/pig/backend/hadoop/executionengine/ sr...

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/SortPartitioner.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/SortPartitioner.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/SortPartitioner.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/SortPartitioner.java Fri Feb 13 01:59:27 2009
@@ -27,6 +27,7 @@
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Partitioner;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.hadoop.HDataType;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.backend.executionengine.ExecException;
@@ -95,7 +96,7 @@
     private void addToQuantiles(
             JobConf job,
             ArrayList<PigNullableWritable> q,
-            Tuple t) {
+            Tuple t) throws ExecException {
         try {
             if ("true".equals(job.get("pig.usercomparator")) || t.size() > 1) {
                 q.add(new NullableTuple(t));
@@ -103,8 +104,10 @@
                 Object o = t.get(0);
                 String kts = job.get("pig.reduce.key.type");
                 if (kts == null) {
-                    throw new RuntimeException("Didn't get reduce key type "
-                        + "from config file.");
+                    int errCode = 2095;
+                    String msg = "Did not get reduce key type "
+                        + "from job configuration.";
+                    throw new ExecException(msg, errCode, PigException.BUG);
                 }
                 q.add(HDataType.getWritableComparableTypes(o,
                     Byte.valueOf(kts)));
@@ -116,7 +119,7 @@
 
     private void convertToArray(
             JobConf job,
-            ArrayList<PigNullableWritable> q) {
+            ArrayList<PigNullableWritable> q) throws ExecException {
         if ("true".equals(job.get("pig.usercomparator")) ||
                 q.get(0).getClass().equals(NullableTuple.class)) {
             quantiles = q.toArray(new NullableTuple[0]);
@@ -133,7 +136,9 @@
         } else if (q.get(0).getClass().equals(NullableText.class)) {
             quantiles = q.toArray(new NullableText[0]);
         } else {
-            throw new RuntimeException("Unexpected class in SortPartitioner");
+            int errCode = 2096;
+            String msg = "Unexpected class in SortPartitioner: " + q.get(0).getClass().getName();
+            throw new ExecException(msg, errCode, PigException.BUG);
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/plans/POPackageAnnotator.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/plans/POPackageAnnotator.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/plans/POPackageAnnotator.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/plans/POPackageAnnotator.java Fri Feb 13 01:59:27 2009
@@ -22,6 +22,7 @@
 import java.util.List;
 import java.util.Map;
 
+import org.apache.pig.PigException;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceOper;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
@@ -32,6 +33,7 @@
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POCombinerPackage;
 import org.apache.pig.impl.plan.DepthFirstWalker;
 import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.impl.plan.optimizer.OptimizerException;
 import org.apache.pig.impl.util.Pair;
 
 /**
@@ -77,8 +79,10 @@
                 // just look for the corresponding LocalRearrange(s) in the combine plan
                 if(pkg instanceof POCombinerPackage) {
                     if(patchPackage(mr.combinePlan, pkg) != pkg.getNumInps()) {
-                        throw new VisitorException("Unexpected problem while trying " +
-                        		"to optimize (could not find LORearrange in combine plan)");
+                        int errCode = 2085;
+                        String msg = "Unexpected problem during optimization." +
+                        " Could not find LocalRearrange in combine plan.";
+                        throw new OptimizerException(msg, errCode, PigException.BUG);
                     }
                 } else {
                     handlePackage(mr, pkg);
@@ -106,8 +110,11 @@
                 }     
             }
         }
-        if(lrFound != pkg.getNumInps())
-            throw new VisitorException("Unexpected problem while trying to optimize (Could not find all LocalRearranges)");
+        if(lrFound != pkg.getNumInps()) {
+            int errCode = 2086;
+            String msg = "Unexpected problem during optimization. Could not find all LocalRearrange operators.";
+            throw new OptimizerException(msg, errCode, PigException.BUG);
+        }
     }
 
     private int patchPackage(PhysicalPlan plan, POPackage pkg) throws VisitorException {
@@ -199,9 +206,11 @@
             if(keyInfo.get(lrearrange.getIndex()) != null) {
                 // something is wrong - we should not be getting key info 
                 // for the same index from two different Local Rearranges
-                throw new VisitorException("Unexpected problem while trying " +
-                                "to optimize (found same index:" + lrearrange.getIndex() + 
-                                " in multiple Local Rearrange operators");
+                int errCode = 2087;
+                String msg = "Unexpected problem during optimization." +
+                " Found index:" + lrearrange.getIndex() + 
+                " in multiple LocalRearrange operators.";
+                throw new OptimizerException(msg, errCode, PigException.BUG);
                 
             }
             keyInfo.put(new Integer(lrearrange.getIndex()), 

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogToPhyTranslationVisitor.java Fri Feb 13 01:59:27 2009
@@ -677,7 +677,13 @@
 
             }
             currentPlan = currentPlans.pop();
-            physOp.setPlans(exprPlans);
+            try {
+                physOp.setPlans(exprPlans);
+            } catch (PlanException pe) {
+                int errCode = 2071;
+                String msg = "Problem with setting up local rearrange's plans.";
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, pe);
+            }
             try {
                 physOp.setIndex(count++);
             } catch (ExecException e1) {

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/PhysicalOperator.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/PhysicalOperator.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/PhysicalOperator.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/PhysicalOperator.java Fri Feb 13 01:59:27 2009
@@ -297,7 +297,6 @@
     public PhysicalOperator clone() throws CloneNotSupportedException {
         String s = new String("This physical operator does not " +
             "implement clone.");
-        log.error(s);
         throw new CloneNotSupportedException(s);
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/EqualToExpr.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/EqualToExpr.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/EqualToExpr.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/EqualToExpr.java Fri Feb 13 01:59:27 2009
@@ -20,6 +20,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
@@ -100,9 +101,13 @@
                             }
 
 
-        default:
-            throw new RuntimeException("Equal to doesn't know how to " +
-                "handle type " + DataType.findTypeName(operandType));
+        default: {
+            int errCode = 2067;
+            String msg = this.getClass().getSimpleName() + " does not know how to " +
+            "handle type: " + DataType.findTypeName(operandType);
+            throw new ExecException(msg, errCode, PigException.BUG);
+        }
+        
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/GTOrEqualToExpr.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/GTOrEqualToExpr.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/GTOrEqualToExpr.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/GTOrEqualToExpr.java Fri Feb 13 01:59:27 2009
@@ -20,6 +20,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
@@ -100,9 +101,13 @@
                             }
 
 
-        default:
-            throw new RuntimeException("Greater than or equal to doesn't know how to " +
-                "handle type " + DataType.findTypeName(operandType));
+        default: {
+            int errCode = 2067;
+            String msg = this.getClass().getSimpleName() + " does not know how to " +
+            "handle type: " + DataType.findTypeName(operandType);
+            throw new ExecException(msg, errCode, PigException.BUG);
+        }
+        
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/GreaterThanExpr.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/GreaterThanExpr.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/GreaterThanExpr.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/GreaterThanExpr.java Fri Feb 13 01:59:27 2009
@@ -20,6 +20,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
@@ -100,9 +101,13 @@
                             }
 
 
-        default:
-            throw new RuntimeException("Greater than doesn't know how to " +
-                "handle type " + DataType.findTypeName(operandType));
+        default: {
+            int errCode = 2067;
+            String msg = this.getClass().getSimpleName() + " does not know how to " +
+            "handle type: " + DataType.findTypeName(operandType);
+            throw new ExecException(msg, errCode, PigException.BUG);
+        }
+        
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/LTOrEqualToExpr.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/LTOrEqualToExpr.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/LTOrEqualToExpr.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/LTOrEqualToExpr.java Fri Feb 13 01:59:27 2009
@@ -20,6 +20,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
@@ -100,9 +101,13 @@
                             }
 
 
-        default:
-            throw new RuntimeException("Less than doesn't know how to " +
-                "handle type " + DataType.findTypeName(operandType));
+        default: {
+            int errCode = 2067;
+            String msg = this.getClass().getSimpleName() + " does not know how to " +
+            "handle type: " + DataType.findTypeName(operandType);
+            throw new ExecException(msg, errCode, PigException.BUG);
+        }
+        
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/LessThanExpr.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/LessThanExpr.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/LessThanExpr.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/LessThanExpr.java Fri Feb 13 01:59:27 2009
@@ -20,6 +20,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import org.apache.pig.PigException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
@@ -100,9 +101,13 @@
                             }
 
 
-        default:
-            throw new RuntimeException("Less than doesn't know how to " +
-                "handle type " + DataType.findTypeName(operandType));
+        default: {
+            int errCode = 2067;
+            String msg = this.getClass().getSimpleName() + " does not know how to " +
+            "handle type: " + DataType.findTypeName(operandType);
+            throw new ExecException(msg, errCode, PigException.BUG);
+        }
+        
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/NotEqualToExpr.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/NotEqualToExpr.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/NotEqualToExpr.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/NotEqualToExpr.java Fri Feb 13 01:59:27 2009
@@ -20,6 +20,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
@@ -100,9 +101,13 @@
                             }
 
 
-        default:
-            throw new RuntimeException("Not equal to doesn't know how to " +
-                "handle type " + DataType.findTypeName(operandType));
+        default: {
+            int errCode = 2067;
+            String msg = this.getClass().getSimpleName() + " does not know how to " +
+            "handle type: " + DataType.findTypeName(operandType);
+            throw new ExecException(msg, errCode, PigException.BUG);
+        }
+        
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POCast.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POCast.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POCast.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POCast.java Fri Feb 13 01:59:27 2009
@@ -25,6 +25,7 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.LoadFunc;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
@@ -138,7 +139,9 @@
                     } else {
                         // the input is a differen type
                         // rethrow the exception
-                        throw e;
+                        int errCode = 1081;
+                        String msg = "Cannot cast to int. Expected bytearray but received: " + DataType.findTypeName(res.result);
+                        throw new ExecException(msg, errCode, PigException.INPUT, e);
                     }
 
                 }
@@ -146,13 +149,9 @@
                     if (null != load) {
                         res.result = load.bytesToInteger(dba.get());
                     } else {
-                        String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to int."
-                                + " castToType: "
-                                + castToType
-                                + " name: "
-                                + DataType.findTypeName(castToType);
-                        log.error(msg);
-                        throw new ExecException(msg);
+                        int errCode = 1075;
+                        String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to int.";
+                        throw new ExecException(msg, errCode, PigException.INPUT);
                     }
                 } catch (ExecException ee) {
                     throw ee;
@@ -280,7 +279,9 @@
                     } else {
                         // the input is a differen type
                         // rethrow the exception
-                        throw e;
+                        int errCode = 1081;
+                        String msg = "Cannot cast to long. Expected bytearray but received: " + DataType.findTypeName(res.result);
+                        throw new ExecException(msg, errCode, PigException.INPUT, e);
                     }
 
                 }
@@ -288,9 +289,9 @@
                     if (null != load) {
                         res.result = load.bytesToLong(dba.get());
                     } else {
+                        int errCode = 1075;
                         String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to long.";
-                        log.error(msg);
-                        throw new ExecException(msg);
+                        throw new ExecException(msg, errCode, PigException.INPUT);
                     }
                 } catch (ExecException ee) {
                     throw ee;
@@ -413,7 +414,9 @@
                     } else {
                         // the input is a differen type
                         // rethrow the exception
-                        throw e;
+                        int errCode = 1081;
+                        String msg = "Cannot cast to double. Expected bytearray but received: " + DataType.findTypeName(res.result);
+                        throw new ExecException(msg, errCode, PigException.INPUT, e);
                     }
 
                 }
@@ -421,9 +424,9 @@
                     if (null != load) {
                         res.result = load.bytesToDouble(dba.get());
                     } else {
+                        int errCode = 1075;
                         String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to double.";
-                        log.error(msg);
-                        throw new ExecException(msg);
+                        throw new ExecException(msg, errCode, PigException.INPUT);
                     }
                 } catch (ExecException ee) {
                     throw ee;
@@ -545,7 +548,9 @@
                     } else {
                         // the input is a differen type
                         // rethrow the exception
-                        throw e;
+                        int errCode = 1081;
+                        String msg = "Cannot cast to float. Expected bytearray but received: " + DataType.findTypeName(res.result);
+                        throw new ExecException(msg, errCode, PigException.INPUT, e);
                     }
 
                 }
@@ -553,9 +558,9 @@
                     if (null != load) {
                         res.result = load.bytesToFloat(dba.get());
                     } else {
+                        int errCode = 1075;
                         String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to float.";
-                        log.error(msg);
-                        throw new ExecException(msg);
+                        throw new ExecException(msg, errCode, PigException.INPUT);
                     }
                 } catch (ExecException ee) {
                     throw ee;
@@ -679,7 +684,9 @@
                     } else {
                         // the input is a differen type
                         // rethrow the exception
-                        throw e;
+                        int errCode = 1081;
+                        String msg = "Cannot cast to chararray. Expected bytearray but received: " + DataType.findTypeName(res.result);
+                        throw new ExecException(msg, errCode, PigException.INPUT, e);
                     }
 
                 }
@@ -687,9 +694,9 @@
                     if (null != load) {
                         res.result = load.bytesToCharArray(dba.get());
                     } else {
+                        int errCode = 1075;
                         String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to string.";
-                        log.error(msg);
-                        throw new ExecException(msg);
+                        throw new ExecException(msg, errCode, PigException.INPUT);
                     }
                 } catch (ExecException ee) {
                     throw ee;
@@ -802,7 +809,9 @@
                     } else {
                         // the input is a differen type
                         // rethrow the exception
-                        throw e;
+                        int errCode = 1081;
+                        String msg = "Cannot cast to tuple. Expected bytearray but received: " + DataType.findTypeName(res.result);
+                        throw new ExecException(msg, errCode, PigException.INPUT, e);
                     }
 
                 }
@@ -810,9 +819,9 @@
                     if (null != load) {
                         res.result = load.bytesToTuple(dba.get());
                     } else {
+                        int errCode = 1075;
                         String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to tuple.";
-                        log.error(msg);
-                        throw new ExecException(msg);
+                        throw new ExecException(msg, errCode, PigException.INPUT);
                     }
                 } catch (ExecException ee) {
                     throw ee;
@@ -888,7 +897,9 @@
                     } else {
                         // the input is a differen type
                         // rethrow the exception
-                        throw e;
+                        int errCode = 1081;
+                        String msg = "Cannot cast to bag. Expected bytearray but received: " + DataType.findTypeName(res.result);
+                        throw new ExecException(msg, errCode, PigException.INPUT, e);
                     }
 
                 }
@@ -896,9 +907,9 @@
                     if (null != load) {
                         res.result = load.bytesToBag(dba.get());
                     } else {
+                        int errCode = 1075;
                         String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to bag.";
-                        log.error(msg);
-                        throw new ExecException(msg);
+                        throw new ExecException(msg, errCode, PigException.INPUT);
                     }
                 } catch (ExecException ee) {
                     throw ee;
@@ -974,7 +985,9 @@
                     } else {
                         // the input is a differen type
                         // rethrow the exception
-                        throw e;
+                        int errCode = 1081;
+                        String msg = "Cannot cast to map. Expected bytearray but received: " + DataType.findTypeName(res.result);
+                        throw new ExecException(msg, errCode, PigException.INPUT, e);
                     }
 
                 }
@@ -982,9 +995,9 @@
                     if (null != load) {
                         res.result = load.bytesToMap(dba.get());
                     } else {
+                        int errCode = 1075;
                         String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to map.";
-                        log.error(msg);
-                        throw new ExecException(msg);
+                        throw new ExecException(msg, errCode, PigException.INPUT);
                     }
                 } catch (ExecException ee) {
                     throw ee;

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POIsNull.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POIsNull.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POIsNull.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POIsNull.java Fri Feb 13 01:59:27 2009
@@ -19,6 +19,7 @@
 
 import java.util.Map;
 
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
@@ -165,9 +166,13 @@
                 }
             }
             return res;        
-        default:
-            throw new RuntimeException("'is null' to doesn't know how to " +
-                "handle type " + DataType.findTypeName(operandType));
+        default: {
+            int errCode = 2067;
+            String msg = this.getClass().getSimpleName() + " does not know how to " +
+            "handle type: " + DataType.findTypeName(operandType);
+            throw new ExecException(msg, errCode, PigException.BUG);
+        }
+        
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POProject.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POProject.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POProject.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POProject.java Fri Feb 13 01:59:27 2009
@@ -21,6 +21,7 @@
 import java.util.Map;
 import java.util.ArrayList;
 
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.DataBag;
@@ -313,11 +314,12 @@
         return columns;
     }
 
-    public int getColumn() {
+    public int getColumn() throws ExecException {
         if(columns.size() != 1) {
-            throw new RuntimeException(
-            "Internal error: improper use of getColumn in "
-            + POProject.class.getName());
+            int errCode = 2068;
+            String msg = "Internal error. Improper use of method getColumn() in "
+                + POProject.class.getSimpleName(); 
+            throw new ExecException(msg, errCode, PigException.BUG);
         }
         return columns.get(0);
     }

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java Fri Feb 13 01:59:27 2009
@@ -29,6 +29,7 @@
 import org.apache.pig.Algebraic;
 import org.apache.pig.EvalFunc;
 import org.apache.pig.FuncSpec;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
@@ -192,15 +193,14 @@
 				return result;
 			}
 			return result;
-			
-		} catch (IOException e1) {
-		    errMsg = "Caught error from UDF " + funcSpec.getClassName() + 
-            "[" + e1.getMessage() + "]";
-			log.error(errMsg);
+		} catch (ExecException ee) {
+		    throw ee;
+		} catch (IOException ioe) {
+		    int errCode = 2078;
+		    String msg = "Caught error from UDF: " + funcSpec.getClassName() + 
+            "[" + ioe.getMessage() + "]";
+			throw new ExecException(msg, errCode, PigException.BUG, ioe);
 		}
-		result.result = errMsg;
-		result.returnStatus = POStatus.STATUS_ERR;
-		return result;
 	}
 
 	@Override
@@ -260,7 +260,7 @@
 		return getNext();
 	}
 
-	public void setAlgebraicFunction(byte Function) {
+	public void setAlgebraicFunction(byte Function) throws ExecException {
 		// This will only be used by the optimizer for putting correct functions
 		// in the mapper,
 		// combiner and reduce. This helps in maintaining the physical plan as
@@ -285,39 +285,39 @@
         setResultType(DataType.findType(((EvalFunc<?>) func).getReturnType()));
 	}
 
-	public String getInitial() {
+	public String getInitial() throws ExecException {
 	    instantiateFunc(origFSpec);
 		if (func instanceof Algebraic) {
 			return ((Algebraic) func).getInitial();
 		} else {
-			String msg = new String("Attempt to run a non-algebraic function"
-                + " as an algebraic function");
-            log.error(msg);
-            throw new RuntimeException(msg);
+		    int errCode = 2072;
+			String msg = "Attempt to run a non-algebraic function"
+                + " as an algebraic function";
+            throw new ExecException(msg, errCode, PigException.BUG);
 		}
 	}
 
-	public String getIntermed() {
+	public String getIntermed() throws ExecException {
         instantiateFunc(origFSpec);
 		if (func instanceof Algebraic) {
 			return ((Algebraic) func).getIntermed();
 		} else {
-			String msg = new String("Attempt to run a non-algebraic function"
-                + " as an algebraic function");
-            log.error(msg);
-            throw new RuntimeException(msg);
+            int errCode = 2072;
+            String msg = "Attempt to run a non-algebraic function"
+                + " as an algebraic function";
+            throw new ExecException(msg, errCode, PigException.BUG);
 		}
 	}
 
-	public String getFinal() {
+	public String getFinal() throws ExecException {
         instantiateFunc(origFSpec);
 		if (func instanceof Algebraic) {
 			return ((Algebraic) func).getFinal();
 		} else {
-			String msg = new String("Attempt to run a non-algebraic function"
-                + " as an algebraic function");
-            log.error(msg);
-            throw new RuntimeException(msg);
+            int errCode = 2072;
+            String msg = "Attempt to run a non-algebraic function"
+                + " as an algebraic function";
+            throw new ExecException(msg, errCode, PigException.BUG);
 		}
 	}
 

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java Fri Feb 13 01:59:27 2009
@@ -195,8 +195,7 @@
             if (cloneFrom == null) {
                 String msg = new String("Unable to find clone for op "
                     + op.name());
-                log.error(msg);
-                throw new RuntimeException(msg);
+                throw new CloneNotSupportedException(msg);
             }
             Collection<PhysicalOperator> toOps = mFromEdges.get(op);
             for (PhysicalOperator toOp : toOps) {
@@ -204,13 +203,14 @@
                 if (cloneTo == null) {
                     String msg = new String("Unable to find clone for op "
                         + toOp.name());
-                    log.error(msg);
-                    throw new RuntimeException(msg);
+                    throw new CloneNotSupportedException(msg);
                 }
                 try {
                     clone.connect(cloneFrom, cloneTo);
                 } catch (PlanException pe) {
-                    throw new RuntimeException(pe);
+                    CloneNotSupportedException cnse = new CloneNotSupportedException();
+                    cnse.initCause(pe);
+                    throw cnse;
                 }
             }
         }
@@ -225,16 +225,14 @@
             if (cloneOp == null) {
                 String msg = new String("Unable to find clone for op "
                     + cloneOp.name());
-                log.error(msg);
-                throw new RuntimeException(msg);
+                throw new CloneNotSupportedException(msg);
             }
             for (PhysicalOperator iOp : inputs) {
                 PhysicalOperator cloneIOp = matches.get(iOp);
                 if (cloneIOp == null) {
                     String msg = new String("Unable to find clone for op "
                         + cloneIOp.name());
-                    log.error(msg);
-                    throw new RuntimeException(msg);
+                    throw new CloneNotSupportedException(msg);
                 }
                 newInputs.add(cloneIOp);
             }

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PlanPrinter.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PlanPrinter.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PlanPrinter.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PlanPrinter.java Fri Feb 13 01:59:27 2009
@@ -28,6 +28,7 @@
 import java.util.List;
 import java.util.Set;
 
+import org.apache.pig.PigException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.*;
 import org.apache.pig.impl.plan.DepthFirstWalker;
 import org.apache.pig.impl.plan.Operator;
@@ -65,8 +66,10 @@
     public void visit() throws VisitorException {
         try {
             stream.write(depthFirstPP().getBytes());
-        } catch (IOException e) {
-            throw new VisitorException(e.getMessage());
+        } catch (IOException ioe) {
+            int errCode = 2079;
+            String msg = "Unexpected error while printing physical plan.";
+            throw new VisitorException(msg, errCode, PigException.BUG, ioe);
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POCombinerPackage.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POCombinerPackage.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POCombinerPackage.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POCombinerPackage.java Fri Feb 13 01:59:27 2009
@@ -43,7 +43,7 @@
  * The package operator that packages the globally rearranged tuples into
  * output format after the combiner stage.  It differs from POPackage in that
  * it does not use the index in the NullableTuple to find the bag to put a
- * tuple in.  Intead, the inputs are put in a bag corresponding to their 
+ * tuple in.  Instead, the inputs are put in a bag corresponding to their 
  * offset in the tuple.
  */
 public class POCombinerPackage extends POPackage {

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POFRJoin.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POFRJoin.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POFRJoin.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POFRJoin.java Fri Feb 13 01:59:27 2009
@@ -28,6 +28,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.pig.ExecType;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
@@ -148,7 +149,13 @@
             lr.setIndex(i);
             lr.setResultType(DataType.TUPLE);
             lr.setKeyType(keyTypes.get(i));
-            lr.setPlans(ppLst);
+            try {
+                lr.setPlans(ppLst);
+            } catch (PlanException pe) {
+                int errCode = 2071;
+                String msg = "Problem with setting up local rearrange's plans.";
+                throw new ExecException(msg, errCode, PigException.BUG, pe);
+            }
             LRs[i]= lr;
             ConstantExpression ce = new ConstantExpression(genKey(old));
             ce.setResultType((i==fragment)?DataType.TUPLE:DataType.BAG);

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java Fri Feb 13 01:59:27 2009
@@ -27,6 +27,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
@@ -271,11 +272,12 @@
                 inputData = planLeafOps[i].getNext(dummyString);
                 break;
 
-                default:
-                    String msg = new String("Unknown type " +
-                        DataType.findTypeName(resultTypes[i]));
-                    log.error(msg);
-                    throw new ExecException(msg);
+                default: {
+                    int errCode = 2080;
+                    String msg = "Foreach currently does not handle type " + DataType.findTypeName(resultTypes[i]);
+                    throw new ExecException(msg, errCode, PigException.BUG);
+                }
+                
                 }
                 
                 if(inputData.returnStatus == POStatus.STATUS_EOP) {

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POLoad.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POLoad.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POLoad.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POLoad.java Fri Feb 13 01:59:27 2009
@@ -24,6 +24,7 @@
 import org.apache.commons.logging.LogFactory;
 
 import org.apache.pig.LoadFunc;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
@@ -121,10 +122,10 @@
         if(!setUpDone && lFile!=null){
             try {
                 setUp();
-            } catch (IOException e) {
-                ExecException ee = new ExecException("Unable to setup the loader because of the exception: " + e.getMessage());
-                ee.initCause(e);
-                throw ee;
+            } catch (IOException ioe) {
+                int errCode = 2081;
+                String msg = "Unable to setup the load function.";
+                throw new ExecException(msg, errCode, PigException.BUG, ioe);
             }
             setUpDone = true;
         }

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POLocalRearrange.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POLocalRearrange.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POLocalRearrange.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POLocalRearrange.java Fri Feb 13 01:59:27 2009
@@ -155,9 +155,11 @@
     }
 
     public void setIndex(int index) throws ExecException {
-        if (index > 0x40) {
-            throw new RuntimeException("Cogroups with more than 127 inputs "
-                + " not supported.");
+        if (index > 0x7F) {
+            int errCode = 1082;
+            String msg = "Cogroups with more than 127 inputs "
+                + " not supported.";
+            throw new ExecException(msg, errCode, PigException.INPUT);
         } else {
             this.index = (byte)index;
         }
@@ -349,7 +351,7 @@
         return plans;
     }
 
-    public void setPlans(List<PhysicalPlan> plans) {
+    public void setPlans(List<PhysicalPlan> plans) throws PlanException {
         this.plans = plans;
         leafOps.clear();
         int keyIndex = 0; // zero based index for fields in the key
@@ -383,7 +385,13 @@
                             log.debug("Project * in group by not being optimized in key-value transfer");
                         }
                     } else {
-                        mProjectedColsMap.put(project.getColumn(), keyIndex);
+                        try {
+                            mProjectedColsMap.put(project.getColumn(), keyIndex);
+                        } catch (ExecException e) {
+                            int errCode = 2070;
+                            String msg = "Problem in accessing column from project operator.";
+                            throw new PlanException(msg, errCode, PigException.BUG);
+                        }
                     }
                     if(project.getResultType() == DataType.TUPLE)
                         isKeyTuple = true;
@@ -416,13 +424,19 @@
             mKey.scope, 
             NodeIdGenerator.getGenerator().getNextNodeId(mKey.scope)),
             requestedParallelism);
-        clone.setPlans(clonePlans);
+        try {
+            clone.setPlans(clonePlans);
+        } catch (PlanException pe) {
+            CloneNotSupportedException cnse = new CloneNotSupportedException("Problem with setting plans of " + this.getClass().getSimpleName());
+            cnse.initCause(pe);
+            throw cnse;
+        }
         clone.keyType = keyType;
         clone.index = index;
         try {
             clone.lrOutput.set(0, index);
         } catch (ExecException e) {
-            CloneNotSupportedException cnse = new CloneNotSupportedException();
+            CloneNotSupportedException cnse = new CloneNotSupportedException("Problem with setting index of output.");
             cnse.initCause(e);
             throw cnse;
         }
@@ -491,7 +505,13 @@
                         String msg = "Internal error. Unexpected operator project(*) in local rearrange inner plan.";
                         throw new PlanException(msg, errCode, PigException.BUG);
                     } else {
-                        mProjectedColsMap.put(project.getColumn(), keyIndex);
+                        try {
+                            mProjectedColsMap.put(project.getColumn(), keyIndex);
+                        } catch (ExecException e) {
+                            int errCode = 2070;
+                            String msg = "Problem in accessing column from project operator.";
+                            throw new PlanException(msg, errCode, PigException.BUG);
+                        }
                     }
                     if(project.getResultType() == DataType.TUPLE)
                         isKeyTuple = true;

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POLocalRearrangeForIllustrate.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POLocalRearrangeForIllustrate.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POLocalRearrangeForIllustrate.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POLocalRearrangeForIllustrate.java Fri Feb 13 01:59:27 2009
@@ -30,6 +30,7 @@
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
 import org.apache.pig.impl.plan.OperatorKey;
 import org.apache.pig.impl.plan.NodeIdGenerator;
+import org.apache.pig.impl.plan.PlanException;
 import org.apache.pig.impl.plan.VisitorException;
 
 /**
@@ -128,7 +129,13 @@
             mKey.scope, 
             NodeIdGenerator.getGenerator().getNextNodeId(mKey.scope)),
             requestedParallelism);
-        clone.setPlans(clonePlans);
+        try {
+            clone.setPlans(clonePlans);
+        } catch (PlanException pe) {
+            CloneNotSupportedException cnse = new CloneNotSupportedException("Problem with setting plans of " + this.getClass().getSimpleName());
+            cnse.initCause(pe);
+            throw cnse;
+        }
         clone.keyType = keyType;
         clone.index = index;
         // Needs to be called as setDistinct so that the fake index tuple gets

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POSort.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POSort.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POSort.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POSort.java Fri Feb 13 01:59:27 2009
@@ -25,6 +25,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
@@ -197,11 +198,13 @@
                 res = Op.getNext(dummyTuple);
                 break;
 
-            default:
-                String msg = new String("Did not expect result of type " +
-                    DataType.findTypeName(resultType));
-                log.error(msg);
-                throw new RuntimeException(msg);
+            default: {
+                int errCode = 2082;
+                String msg = "Did not expect result of type: " +
+                        DataType.findTypeName(resultType);
+                    throw new ExecException(msg, errCode, PigException.BUG);                
+            }
+            
             }
 			return res;
 		}

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POStore.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POStore.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POStore.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POStore.java Fri Feb 13 01:59:27 2009
@@ -23,6 +23,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.pig.PigException;
 import org.apache.pig.StoreFunc;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.Tuple;
@@ -125,10 +126,10 @@
     public Result store() throws ExecException{
         try{
             setUp();
-        }catch (IOException e) {
-            ExecException ee = new ExecException("Unable to setup the storer because of the exception: " + e.getMessage());
-            ee.initCause(e);
-            throw ee;
+        }catch (IOException ioe) {
+            int errCode = 2081;
+            String msg = "Unable to setup the store function.";            
+            throw new ExecException(msg, errCode, PigException.BUG, ioe);
         }
         try{
             Result res;

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POStream.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POStream.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POStream.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POStream.java Fri Feb 13 01:59:27 2009
@@ -25,6 +25,7 @@
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 
+import org.apache.pig.PigException;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.plan.OperatorKey;
 import org.apache.pig.impl.plan.VisitorException;
@@ -224,7 +225,9 @@
             }
             
         } catch(Exception e) {
-            throw new ExecException("Error while trying to get next result in POStream", e);
+            int errCode = 2083;
+            String msg = "Error while trying to get next result in POStream.";
+            throw new ExecException(msg, errCode, PigException.BUG, e);
         }
             
         
@@ -267,8 +270,10 @@
                                 try {
                                     executableManager.configure(this);
                                     executableManager.run();
-                                } catch (IOException e) {
-                                    throw new ExecException("Error while running streaming binary", e);
+                                } catch (IOException ioe) {
+                                    int errCode = 2084;
+                                    String msg = "Error while running streaming binary.";
+                                    throw new ExecException(msg, errCode, PigException.BUG, ioe);
                                 }            
                                 initialized = true;
                             }
@@ -289,7 +294,9 @@
                 }
             }
         } catch (Exception e) {
-            throw new ExecException("Error while trying to get next result in POStream", e);
+            int errCode = 2083;
+            String msg = "Error while trying to get next result in POStream.";
+            throw new ExecException(msg, errCode, PigException.BUG, e);
         }
     }
     

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java Fri Feb 13 01:59:27 2009
@@ -142,10 +142,10 @@
                                                      getOutputName(partition))
                                             );
                     } catch (IOException ioe) {
-                        System.err.println("Failed to save secondary output '" + 
-                                           fileName + "' of task: " + taskId +
-                                           " with " + ioe);
-                        throw ioe;
+                        int errCode = 6014; 
+                        String msg = "Failed to save secondary output '" + 
+                        fileName + "' of task: " + taskId;
+                        throw new ExecException(msg, errCode, PigException.REMOTE_ENVIRONMENT, ioe);
                     }
                 }
         }

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/local/executionengine/physicalLayer/LocalLogToPhyTranslationVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/local/executionengine/physicalLayer/LocalLogToPhyTranslationVisitor.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/local/executionengine/physicalLayer/LocalLogToPhyTranslationVisitor.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/local/executionengine/physicalLayer/LocalLogToPhyTranslationVisitor.java Fri Feb 13 01:59:27 2009
@@ -23,6 +23,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.LogToPhyTranslationVisitor;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
@@ -90,7 +91,11 @@
 
             }
             currentPlan = currentPlans.pop();
-            physOp.setPlans(exprPlans);
+            try {
+                physOp.setPlans(exprPlans);
+            } catch (PlanException pe) {
+                throw new VisitorException(pe);
+            }
             try {
                 physOp.setIndex(count++);
             } catch (ExecException e1) {

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/ARITY.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/ARITY.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/ARITY.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/ARITY.java Fri Feb 13 01:59:27 2009
@@ -20,6 +20,8 @@
 import java.io.IOException;
 
 import org.apache.pig.EvalFunc;
+import org.apache.pig.PigException;
+import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
@@ -35,8 +37,10 @@
             Tuple t = (Tuple)input.get(0);
             if (t == null) return null;
             return new Integer(t.size());
-        }catch(Exception e){
-            throw WrappedIOException.wrap(e); 
+        } catch (Exception e) {
+            int errCode = 2106;
+            String msg = "Error while computing arity in " + this.getClass().getSimpleName();
+            throw new ExecException(msg, errCode, PigException.BUG, e);           
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/AVG.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/AVG.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/AVG.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/AVG.java Fri Feb 13 01:59:27 2009
@@ -25,6 +25,7 @@
 import org.apache.pig.Algebraic;
 import org.apache.pig.EvalFunc;
 import org.apache.pig.FuncSpec;
+import org.apache.pig.PigException;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.DataType;
@@ -61,9 +62,7 @@
     
             return avg;
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            oughtToBeEE.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
         }
     }
 
@@ -99,13 +98,15 @@
                     t.set(0, null);
                     t.set(1, 1L);
                 } catch (ExecException e) {
-                    throw WrappedIOException.wrap(e);
+                    throw e;
                 }
                 return t;
             } catch (ExecException ee) {
-                IOException oughtToBeEE = new IOException();
-                oughtToBeEE.initCause(ee);
-                throw oughtToBeEE;
+                throw ee;
+            } catch (Exception e) {
+                int errCode = 2106;
+                String msg = "Error while computing average in " + this.getClass().getSimpleName();
+                throw new ExecException(msg, errCode, PigException.BUG, e);            
             }
                 
         }
@@ -118,9 +119,12 @@
                 DataBag b = (DataBag)input.get(0);
                 return combine(b);
             } catch (ExecException ee) {
-                IOException oughtToBeEE = new IOException();
-                oughtToBeEE.initCause(ee);
-                throw oughtToBeEE;
+                throw ee;
+            } catch (Exception e) {
+                int errCode = 2106;
+                String msg = "Error while computing average in " + this.getClass().getSimpleName();
+                throw new ExecException(msg, errCode, PigException.BUG, e);           
+            
             }
         }
     }
@@ -144,9 +148,11 @@
                 }
                 return avg;
             } catch (ExecException ee) {
-                IOException oughtToBeEE = new IOException();
-                oughtToBeEE.initCause(ee);
-                throw oughtToBeEE;
+                throw ee;
+            } catch (Exception e) {
+                int errCode = 2106;
+                String msg = "Error while computing average in " + this.getClass().getSimpleName();
+                throw new ExecException(msg, errCode, PigException.BUG, e);           
             }
         }
     }
@@ -211,9 +217,9 @@
                 sawNonNull = true;
                 sum += d;
             }catch(RuntimeException exp) {
-                ExecException newE =  new ExecException("Error processing: " +
-                    t.toString() + exp.getMessage(), exp);
-                throw newE;
+                int errCode = 2103;
+                String msg = "Problem while computing sum of doubles.";
+                throw new ExecException(msg, errCode, PigException.BUG, exp);
             }
         }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/BagSize.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/BagSize.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/BagSize.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/BagSize.java Fri Feb 13 01:59:27 2009
@@ -19,6 +19,7 @@
 
 import java.io.IOException;
 import org.apache.pig.EvalFunc;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataType;
@@ -37,11 +38,11 @@
             DataBag bag = (DataBag)(input.get(0));
             return bag == null ? null : new Long(bag.size());
         } catch (ExecException exp) {
-            IOException oughtToBeEE = 
-                new IOException("Error processing: " +
-                input.toString() + exp.getMessage());
-            oughtToBeEE.initCause(exp);
-            throw oughtToBeEE;
+            throw exp;
+        } catch (Exception e) {
+            int errCode = 2106;
+            String msg = "Error while computing size in " + this.getClass().getSimpleName();
+            throw new ExecException(msg, errCode, PigException.BUG, e);            
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/BinStorage.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/BinStorage.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/BinStorage.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/BinStorage.java Fri Feb 13 01:59:27 2009
@@ -30,6 +30,7 @@
 import java.util.Map;
 
 import org.apache.pig.ExecType;
+import org.apache.pig.PigException;
 import org.apache.pig.ReversibleLoadStoreFunc;
 import org.apache.pig.backend.datastorage.DataStorage;
 import org.apache.pig.backend.executionengine.ExecException;
@@ -89,9 +90,7 @@
         try {
             return (Tuple)DataReaderWriter.readDatum(inData);
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            oughtToBeEE.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
         }
     }
 
@@ -124,9 +123,7 @@
         try {
             return (DataBag)DataReaderWriter.readDatum(dis);
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
         }        
     }
 
@@ -135,9 +132,7 @@
         try {
             return (String)DataReaderWriter.readDatum(dis);
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
         }
     }
 
@@ -146,9 +141,7 @@
         try {
             return (Double)DataReaderWriter.readDatum(dis);
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
         }
     }
 
@@ -157,9 +150,7 @@
         try {
             return (Float)DataReaderWriter.readDatum(dis);
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
         }
     }
 
@@ -168,9 +159,7 @@
         try {
             return (Integer)DataReaderWriter.readDatum(dis);
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
         }
     }
 
@@ -179,9 +168,7 @@
         try {
             return (Long)DataReaderWriter.readDatum(dis);
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
         }
     }
 
@@ -190,9 +177,7 @@
         try {
             return (Map<Object, Object>)DataReaderWriter.readDatum(dis);
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
         }
     }
 
@@ -201,9 +186,7 @@
         try {
             return (Tuple)DataReaderWriter.readDatum(dis);
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
         }
     }
 
@@ -229,7 +212,9 @@
             try {
                 s.add(DataType.determineFieldSchema(t.get(i)));
             } catch (Exception e) {
-                throw WrappedIOException.wrap(e);
+                int errCode = 2104;
+                String msg = "Error while determining schema of BinStorage data.";
+                throw new ExecException(msg, errCode, PigException.BUG, e);
             } 
         }
         return s;
@@ -246,9 +231,9 @@
         try {
             DataReaderWriter.writeDatum(dos, bag);
         } catch (Exception ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            int errCode = 2105;
+            String msg = "Error while converting bag to bytes.";
+            throw new ExecException(msg, errCode, PigException.BUG, ee);
         }
         return baos.toByteArray();
     }
@@ -259,9 +244,9 @@
         try {
             DataReaderWriter.writeDatum(dos, s);
         } catch (Exception ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            int errCode = 2105;
+            String msg = "Error while converting chararray to bytes.";
+            throw new ExecException(msg, errCode, PigException.BUG, ee);
         }
         return baos.toByteArray();
     }
@@ -272,9 +257,9 @@
         try {
             DataReaderWriter.writeDatum(dos, d);
         } catch (Exception ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            int errCode = 2105;
+            String msg = "Error while converting double to bytes.";
+            throw new ExecException(msg, errCode, PigException.BUG, ee);
         }
         return baos.toByteArray();
     }
@@ -285,9 +270,9 @@
         try {
             DataReaderWriter.writeDatum(dos, f);
         } catch (Exception ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            int errCode = 2105;
+            String msg = "Error while converting float to bytes.";
+            throw new ExecException(msg, errCode, PigException.BUG, ee);
         }
         return baos.toByteArray();
     }
@@ -298,9 +283,9 @@
         try {
             DataReaderWriter.writeDatum(dos, i);
         } catch (Exception ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            int errCode = 2105;
+            String msg = "Error while converting int to bytes.";
+            throw new ExecException(msg, errCode, PigException.BUG, ee);
         }
         return baos.toByteArray();
     }
@@ -311,9 +296,9 @@
         try {
             DataReaderWriter.writeDatum(dos, l);
         } catch (Exception ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            int errCode = 2105;
+            String msg = "Error while converting long to bytes.";
+            throw new ExecException(msg, errCode, PigException.BUG, ee);
         }
         return baos.toByteArray();
     }
@@ -324,9 +309,9 @@
         try {
             DataReaderWriter.writeDatum(dos, m);
         } catch (Exception ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            int errCode = 2105;
+            String msg = "Error while converting map to bytes.";
+            throw new ExecException(msg, errCode, PigException.BUG, ee);
         }
         return baos.toByteArray();
     }
@@ -337,9 +322,9 @@
         try {
             DataReaderWriter.writeDatum(dos, t);
         } catch (Exception ee) {
-            IOException oughtToBeEE = new IOException();
-            ee.initCause(ee);
-            throw oughtToBeEE;
+            int errCode = 2105;
+            String msg = "Error while converting tuple to bytes.";
+            throw new ExecException(msg, errCode, PigException.BUG, ee);
         }
         return baos.toByteArray();
     }

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/BinaryStorage.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/BinaryStorage.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/BinaryStorage.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/BinaryStorage.java Fri Feb 13 01:59:27 2009
@@ -127,9 +127,7 @@
         try {
             data = ((DataByteArray)f.get(0)).get();
         } catch (ExecException e) {
-            IOException ioe = new IOException("Unable to get field out of tuple");
-            ioe.initCause(e);
-            throw ioe;
+            throw e;
         }
         if (data.length > 0) {
             out.write(data);

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/CONCAT.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/CONCAT.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/CONCAT.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/CONCAT.java Fri Feb 13 01:59:27 2009
@@ -23,6 +23,7 @@
 
 import org.apache.pig.EvalFunc;
 import org.apache.pig.FuncSpec;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.DataType;
@@ -47,10 +48,11 @@
                 return null;
             }
         } catch (ExecException exp) {
-            IOException oughtToBeEE = new IOException("Error processing: " +
-                input.toString() + exp.getMessage());
-            oughtToBeEE.initCause(exp);
-            throw oughtToBeEE;
+            throw exp;
+        } catch (Exception e) {
+            int errCode = 2106;
+            String msg = "Error while computing concat in " + this.getClass().getSimpleName();
+            throw new ExecException(msg, errCode, PigException.BUG, e);          
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/COUNT.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/COUNT.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/COUNT.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/COUNT.java Fri Feb 13 01:59:27 2009
@@ -23,6 +23,7 @@
 
 import org.apache.pig.Algebraic;
 import org.apache.pig.EvalFunc;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataType;
@@ -44,7 +45,11 @@
             DataBag bag = (DataBag)input.get(0);
             return bag.size();
         } catch (ExecException ee) {
-            throw WrappedIOException.wrap("Caught exception in COUNT", ee);
+            throw ee;
+        } catch (Exception e) {
+            int errCode = 2106;                
+            String msg = "Error while computing count in " + this.getClass().getSimpleName();
+            throw new ExecException(msg, errCode, PigException.BUG, e);
         }
     }
 
@@ -79,8 +84,11 @@
             try {
                 return mTupleFactory.newTuple(sum(input));
             } catch (ExecException ee) {
-                throw WrappedIOException.wrap(
-                    "Caught exception in COUNT.Intermed", ee);
+                throw ee;
+            } catch (Exception e) {
+                int errCode = 2106;                
+                String msg = "Error while computing count in " + this.getClass().getSimpleName();
+                throw new ExecException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -91,8 +99,9 @@
             try {
                 return sum(input);
             } catch (Exception ee) {
-                throw WrappedIOException.wrap(
-                    "Caught exception in COUNT.Final", ee);
+                int errCode = 2106;
+                String msg = "Error while computing count in " + this.getClass().getSimpleName();
+                throw new ExecException(msg, errCode, PigException.BUG, ee);
             }
         }
     }

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/ConstantSize.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/ConstantSize.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/ConstantSize.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/ConstantSize.java Fri Feb 13 01:59:27 2009
@@ -20,6 +20,7 @@
 import java.io.IOException;
 
 import org.apache.pig.EvalFunc;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
@@ -36,10 +37,11 @@
         try {
            return input.get(0) == null ? null : new Long(1);
         } catch (ExecException exp) {
-            IOException oughtToBeEE = new IOException("Error processing: " +
-                input.toString() + exp.getMessage());
-            oughtToBeEE.initCause(exp);
-            throw oughtToBeEE;
+            throw exp;
+        } catch (Exception e) {
+            int errCode = 2106;
+            String msg = "Error while computing size in " + this.getClass().getSimpleName();
+            throw new ExecException(msg, errCode, PigException.BUG, e);           
         }
     }
     

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/DIFF.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/DIFF.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/DIFF.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/DIFF.java Fri Feb 13 01:59:27 2009
@@ -23,6 +23,7 @@
 import java.util.Set;
 
 import org.apache.pig.EvalFunc;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.DataBag;
@@ -48,8 +49,9 @@
     @Override
     public DataBag exec(Tuple input) throws IOException {
         if (input.size() != 2) {
-            throw new IOException("DIFF must compare two fields not " +
-                input.size());
+            int errCode = 2107;
+            String msg = "DIFF expected two inputs but received " + input.size() + " inputs.";
+            throw new ExecException(msg, errCode, PigException.BUG);
         }
         try {
             DataBag output = mBagFactory.newDefaultBag();
@@ -68,9 +70,7 @@
             }
             return output;
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            oughtToBeEE.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/Distinct.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/Distinct.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/Distinct.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/Distinct.java Fri Feb 13 01:59:27 2009
@@ -87,7 +87,7 @@
                 bag.add((Tuple)input.get(0));
                 return tupleFactory.newTuple(bag);
             } catch (ExecException e) {
-                throw WrappedIOException.wrap(e);
+                throw e;
             }
         }
     }
@@ -132,7 +132,7 @@
                 }
             }
         } catch (ExecException e) {
-           throw WrappedIOException.wrap(e);
+           throw e;
         }
         return result;
     }
@@ -151,7 +151,7 @@
             }
             return result;
         } catch (ExecException e) {
-             throw WrappedIOException.wrap(e);
+             throw e;
         }
     }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/DoubleAvg.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/DoubleAvg.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/DoubleAvg.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/DoubleAvg.java Fri Feb 13 01:59:27 2009
@@ -24,6 +24,7 @@
 import org.apache.pig.Algebraic;
 import org.apache.pig.EvalFunc;
 import org.apache.pig.FuncSpec;
+import org.apache.pig.PigException;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
@@ -58,9 +59,7 @@
     
             return avg;
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            oughtToBeEE.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
         }
     }
 
@@ -88,12 +87,12 @@
                 t.set(0, (Double)(tp.get(0)));
                 t.set(1, 1L);
                 return t;
-            } catch(RuntimeException t) {
-                throw new RuntimeException(t.getMessage() + ": " + input);
             } catch (ExecException ee) {
-                IOException oughtToBeEE = new IOException();
-                oughtToBeEE.initCause(ee);
-                throw oughtToBeEE;
+                throw ee;
+            } catch (Exception e) {
+                int errCode = 2106;
+                String msg = "Error while computing average in " + this.getClass().getSimpleName();
+                throw new ExecException(msg, errCode, PigException.BUG, e);           
             }
                 
         }
@@ -106,9 +105,11 @@
                 DataBag b = (DataBag)input.get(0);
                 return combine(b);
             } catch (ExecException ee) {
-                IOException oughtToBeEE = new IOException();
-                oughtToBeEE.initCause(ee);
-                throw oughtToBeEE;
+                throw ee;
+            } catch (Exception e) {
+                int errCode = 2106;
+                String msg = "Error while computing average in " + this.getClass().getSimpleName();
+                throw new ExecException(msg, errCode, PigException.BUG, e);            
             }
         }
     }
@@ -132,9 +133,11 @@
                 }
                 return avg;
             } catch (ExecException ee) {
-                IOException oughtToBeEE = new IOException();
-                oughtToBeEE.initCause(ee);
-                throw oughtToBeEE;
+                throw ee;
+            } catch (Exception e) {
+                int errCode = 2106;
+                String msg = "Error while computing average in " + this.getClass().getSimpleName();
+                throw new ExecException(msg, errCode, PigException.BUG, e);            
             }
         }
     }
@@ -198,9 +201,9 @@
                 sawNonNull = true;
                 sum += d;
             }catch(RuntimeException exp) {
-                ExecException newE =  new ExecException("Error processing: " +
-                    t.toString() + exp.getMessage(), exp);
-                throw newE;
+                int errCode = 2103;
+                String msg = "Problem while computing sum of doubles.";
+                throw new ExecException(msg, errCode, PigException.BUG, exp);
             }
         }
 

Modified: hadoop/pig/trunk/src/org/apache/pig/builtin/DoubleMax.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/builtin/DoubleMax.java?rev=743952&r1=743951&r2=743952&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/builtin/DoubleMax.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/builtin/DoubleMax.java Fri Feb 13 01:59:27 2009
@@ -22,6 +22,7 @@
 
 import org.apache.pig.Algebraic;
 import org.apache.pig.EvalFunc;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataType;
@@ -40,9 +41,11 @@
          try {
             return max(input);
         } catch (ExecException ee) {
-            IOException oughtToBeEE = new IOException();
-            oughtToBeEE.initCause(ee);
-            throw oughtToBeEE;
+            throw ee;
+        } catch (Exception e) {
+            int errCode = 2106;
+            String msg = "Error while computing max in " + this.getClass().getSimpleName();
+            throw new ExecException(msg, errCode, PigException.BUG, e);           
         }
     }
 
@@ -70,9 +73,11 @@
                 Tuple tp = bg.iterator().next();
                 return tfact.newTuple((Double)(tp.get(0)));
             } catch (ExecException ee) {
-                IOException oughtToBeEE = new IOException();
-                oughtToBeEE.initCause(ee);
-                throw oughtToBeEE;
+                throw ee;
+            } catch (Exception e) {
+                int errCode = 2106;
+                String msg = "Error while computing max in " + this.getClass().getSimpleName();
+                throw new ExecException(msg, errCode, PigException.BUG, e);            
             }
         }
     }
@@ -85,9 +90,11 @@
             try {
                 return tfact.newTuple(max(input));
             } catch (ExecException ee) {
-                IOException oughtToBeEE = new IOException();
-                oughtToBeEE.initCause(ee);
-                throw oughtToBeEE;
+                throw ee;
+            } catch (Exception e) {
+                int errCode = 2106;
+                String msg = "Error while computing max in " + this.getClass().getSimpleName();
+                throw new ExecException(msg, errCode, PigException.BUG, e);           
             }
         }
     }
@@ -97,9 +104,11 @@
             try {
                 return max(input);
             } catch (ExecException ee) {
-                IOException oughtToBeEE = new IOException();
-                oughtToBeEE.initCause(ee);
-                throw oughtToBeEE;
+                throw ee;
+            } catch (Exception e) {
+                int errCode = 2106;
+                String msg = "Error while computing max in " + this.getClass().getSimpleName();
+                throw new ExecException(msg, errCode, PigException.BUG, e);           
             }
         }
     }
@@ -124,9 +133,9 @@
                 sawNonNull = true;
                 curMax = java.lang.Math.max(curMax, d);
             } catch (RuntimeException exp) {
-                ExecException newE = new ExecException("Error processing: " +
-                    t.toString() + exp.getMessage(), exp);
-                throw newE;
+                int errCode = 2103;
+                String msg = "Problem while computing max of doubles.";
+                throw new ExecException(msg, errCode, PigException.BUG, exp);
             }
         }