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/01/24 00:07:32 UTC

svn commit: r737239 [2/6] - in /hadoop/pig/trunk: ./ src/org/apache/pig/ src/org/apache/pig/backend/ src/org/apache/pig/backend/datastorage/ src/org/apache/pig/backend/executionengine/ src/org/apache/pig/backend/executionengine/util/ src/org/apache/pig...

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java?rev=737239&r1=737238&r2=737239&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java Fri Jan 23 23:07:30 2009
@@ -45,20 +45,16 @@
 
 import org.apache.pig.ComparisonFunc;
 import org.apache.pig.FuncSpec;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.hadoop.HDataType;
-import org.apache.pig.backend.hadoop.DoubleWritable;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MROperPlan;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLoad;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
 import org.apache.pig.data.BagFactory;
-import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.DataType;
-import org.apache.pig.data.Tuple;
 import org.apache.pig.data.TupleFactory;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileLocalizer;
@@ -70,9 +66,7 @@
 import org.apache.pig.impl.io.NullableLongWritable;
 import org.apache.pig.impl.io.NullableText;
 import org.apache.pig.impl.io.NullableTuple;
-import org.apache.pig.impl.plan.DepthFirstWalker;
 import org.apache.pig.impl.plan.OperatorKey;
-import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.util.JarManager;
 import org.apache.pig.impl.util.ObjectSerializer;
 import org.apache.pig.impl.util.Pair;
@@ -189,9 +183,12 @@
             //Create a new Job with the obtained JobConf
             //and the compiled inputs as dependent jobs
             return new Job(currJC,(ArrayList<Job>)compiledInputs);
-        }catch(Exception e){
-            JobCreationException jce = new JobCreationException(e);
-            throw jce;
+        } catch (JobCreationException jce) {
+        	throw jce;
+        } catch(Exception e) {
+            int errCode = 2017;
+            String msg = "Internal error creating job configuration.";
+            throw new JobCreationException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -392,9 +389,12 @@
                 }
             }
             return jobConf;
-        }catch(Exception e){
-            JobCreationException jce = new JobCreationException(e);
-            throw jce;
+        } catch (JobCreationException jce) {
+        	throw jce;
+        } catch(Exception e) {
+            int errCode = 2017;
+            String msg = "Internal error creating job configuration.";
+            throw new JobCreationException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -514,16 +514,18 @@
                 break;
 
             case DataType.MAP:
-                log.error("Using Map as key not supported.");
-                throw new JobCreationException("Using Map as key not supported");
+                int errCode = 1068;
+                String msg = "Using Map as key not supported.";
+                throw new JobCreationException(msg, errCode, PigException.INPUT);
 
             case DataType.TUPLE:
                 jobConf.setOutputKeyComparatorClass(PigTupleRawComparator.class);
                 break;
 
             case DataType.BAG:
-                log.error("Using Bag as key not supported.");
-                throw new JobCreationException("Using Bag as key not supported");
+                errCode = 1068;
+                msg = "Using Bag as key not supported.";
+                throw new JobCreationException(msg, errCode, PigException.INPUT);
 
             default:
                 break;
@@ -557,20 +559,23 @@
             break;
 
         case DataType.MAP:
-            log.error("Using Map as key not supported.");
-            throw new JobCreationException("Using Map as key not supported");
+            int errCode = 1068;
+            String msg = "Using Map as key not supported.";
+            throw new JobCreationException(msg, errCode, PigException.INPUT);
 
         case DataType.TUPLE:
             jobConf.setOutputKeyComparatorClass(PigTupleWritableComparator.class);
             break;
 
         case DataType.BAG:
-            log.error("Using Bag as key not supported.");
-            throw new JobCreationException("Using Bag as key not supported");
+            errCode = 1068;
+            msg = "Using Bag as key not supported.";
+            throw new JobCreationException(msg, errCode, PigException.INPUT);
 
         default:
-            throw new RuntimeException("Forgot case for type " +
-                DataType.findTypeName(keyType));
+            errCode = 2036;
+            msg = "Unhandled key type " + DataType.findTypeName(keyType);
+            throw new JobCreationException(msg, errCode, PigException.BUG);
         }
     }
 
@@ -597,8 +602,10 @@
                     try {
                         srcURI = new URI(src.toString());
                     } catch (URISyntaxException ue) {
-                        throw new IOException("Invalid cache specification, " +
-                        		              "file doesn't exist: " + src);
+                        int errCode = 6003;
+                        String msg = "Invalid cache specification. " +
+                        "File doesn't exist: " + src;
+                        throw new PigException(msg, errCode, PigException.USER_ENVIRONMENT);
                     }
                     
                     // Ship it to the cluster if necessary and add to the
@@ -614,8 +621,22 @@
                         try {
                             dstURI = new URI(dst.toString() + "#" + src.getName());
                         } catch (URISyntaxException ue) {
-                            throw new IOException("Invalid ship specification, " +
-                                                  "file doesn't exist: " + dst);
+                            byte errSrc = pigContext.getErrorSource();
+                            int errCode = 0;
+                            switch(errSrc) {
+                            case PigException.REMOTE_ENVIRONMENT:
+                                errCode = 6004;
+                                break;
+                            case PigException.USER_ENVIRONMENT:
+                                errCode = 4004;
+                                break;
+                            default:
+                                errCode = 2037;
+                                break;
+                            }
+                            String msg = "Invalid ship specification. " +
+                            "File doesn't exist: " + dst;
+                            throw new PigException(msg, errCode, errSrc);
                         }
                         DistributedCache.addCacheFile(dstURI, conf);
                     } else {

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobCreationException.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobCreationException.java?rev=737239&r1=737238&r2=737239&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobCreationException.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobCreationException.java Fri Jan 23 23:07:30 2009
@@ -17,26 +17,143 @@
  */
 package org.apache.pig.backend.hadoop.executionengine.mapReduceLayer;
 
-import org.apache.pig.impl.logicalLayer.FrontendException;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 
 public class JobCreationException extends FrontendException{
 
     private static final long serialVersionUID = 1L;
-
-    public JobCreationException() {
-        super();
-    }
-
-    public JobCreationException(String arg0, Throwable arg1) {
-        super(arg0, arg1);
-    }
-
-    public JobCreationException(String arg0) {
-        super(arg0);
-    }
-
-    public JobCreationException(Throwable arg0) {
-        super(arg0);
+
+    /**
+     * Create a new JobCreationException with null as the error message.
+     */
+    public JobCreationException() {
+        super();
+    }
+    
+    /**
+     * Create a new JobCreationException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     */
+    public JobCreationException(String message) {
+        super(message);
+    }
+    
+    /**
+     * Create a new JobCreationException with the specified cause.
+     *
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown.
+     */
+    public JobCreationException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Create a new JobCreationException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown.
+     */
+    public JobCreationException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    /**
+     * Create a new JobCreationException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     */
+    public JobCreationException(String message, int errCode) {
+        super(message, errCode);
+    }
+
+    /**
+     * Create a new JobCreationException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown. 
+     */
+    public JobCreationException(String message, int errCode, Throwable cause) {
+        super(message, errCode, cause);
+    }
+
+    /**
+     * Create a new JobCreationException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     */
+    public JobCreationException(String message, int errCode, byte errSrc) {
+        super(message, errCode, errSrc);
+    }   
+
+    /**
+     * Create a new JobCreationException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown. 
+     */
+    public JobCreationException(String message, int errCode, byte errSrc,
+            Throwable cause) {
+        super(message, errCode, errSrc, cause);
+    }
+
+    /**
+     * Create a new JobCreationException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param retry - If the exception is retriable or not
+     */ 
+    public JobCreationException(String message, int errCode, boolean retry) {
+        super(message, errCode, retry);
+    }
+
+    /**
+     * Create a new JobCreationException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     * @param retry - If the exception is retriable or not
+     */
+    public JobCreationException(String message, int errCode, byte errSrc,
+            boolean retry) {
+        super(message, errCode, errSrc, retry);
+    }    
+
+    /**
+     * Create a new JobCreationException with the specified message, error code, error source, retriable or not, detalied message for the developer and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     * @param retry - If the exception is retriable or not
+     * @param detailedMsg - The detailed message shown to the developer 
+     */
+    public JobCreationException(String message, int errCode, byte errSrc,
+            boolean retry, String detailedMsg) {
+        super(message, errCode, errSrc, retry, detailedMsg);
+    }
+    
+    /**
+     * Create a new JobCreationException with the specified message, error code, error source, retriable or not, detalied message for the developer and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     * @param retry - If the exception is retriable or not
+     * @param detailedMsg - The detailed message shown to the developer 
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown.
+     */
+    public JobCreationException(String message, int errCode, byte errSrc,
+            boolean retry, String detailedMsg, Throwable cause) {
+        super(message, errCode, errSrc, retry, detailedMsg, cause);
     }
 
 }

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java?rev=737239&r1=737238&r2=737239&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompiler.java Fri Jan 23 23:07:30 2009
@@ -31,6 +31,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.pig.FuncSpec;
+import org.apache.pig.PigException;
 import org.apache.pig.builtin.BinStorage;
 import org.apache.pig.data.DataType;
 import org.apache.pig.impl.PigContext;
@@ -141,22 +142,28 @@
     
     private UDFFinder udfFinder;
     
-    public MRCompiler(PhysicalPlan plan) {
+    public MRCompiler(PhysicalPlan plan) throws MRCompilerException {
         this(plan,null);
     }
     
     public MRCompiler(PhysicalPlan plan,
-            PigContext pigContext) {
+            PigContext pigContext) throws MRCompilerException {
         super(plan, new DepthFirstWalker<PhysicalOperator, PhysicalPlan>(plan));
         this.plan = plan;
         this.pigContext = pigContext;
         splitsSeen = new HashMap<OperatorKey, MapReduceOper>();
         MRPlan = new MROperPlan();
         nig = NodeIdGenerator.getGenerator();
-        scope = plan.getRoots().get(0).getOperatorKey().getScope();
         r = new Random(1331);
         FileLocalizer.setR(r);
         udfFinder = new UDFFinder();
+        List<PhysicalOperator> roots = plan.getRoots();
+        if((roots == null) || (roots.size() <= 0)) {
+        	int errCode = 2053;
+        	String msg = "Internal error. Did not find roots in the physical plan.";
+        	throw new MRCompilerException(msg, errCode, PigException.BUG);
+        }
+        scope = roots.get(0).getOperatorKey().getScope();
     }
     
     public void randomizeFileLocalizer(){
@@ -190,6 +197,12 @@
      */
     public MROperPlan compile() throws IOException, PlanException, VisitorException {
         List<PhysicalOperator> leaves = plan.getLeaves();
+        if(!(leaves.get(0) instanceof POStore)) {
+            int errCode = 2025;
+            String msg = "Expected leaf of reduce plan to " +
+                "always be POStore. Found " + leaves.get(0).getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG);
+        }
         POStore store = (POStore)leaves.get(0);
         FileLocalizer.registerDeleteOnFail(store.getSFile().getFileName(), pigContext);
         compile(store);
@@ -305,7 +318,6 @@
      * connect the MROpers according to the dependencies.
      * @param op
      * @throws PlanException
-     * @throws IOException 
      * @throws IOException
      */
     private void nonBlocking(PhysicalOperator op) throws PlanException, IOException{
@@ -318,8 +330,9 @@
             } else if (mro.isMapDone() && !mro.isReduceDone()) {
                 mro.reducePlan.addAsLeaf(op);
             } else {
-                log.error("Both map and reduce phases have been done. This is unexpected while compiling!");
-                throw new PlanException("Both map and reduce phases have been done. This is unexpected while compiling!");
+                int errCode = 2022;
+                String msg = "Both map and reduce phases have been done. This is unexpected while compiling.";                
+                throw new PlanException(msg, errCode, PigException.BUG);
             }
             curMROp = mro;
         } else {
@@ -406,7 +419,7 @@
      * by adding appropriate loads
      * @param mergedPlans - The list of reduce MROpers
      * @param mro - The map MROper
-     * @throws IOException 
+     * @throws PlanException 
      * @throws IOException
      */
     private void connRedOper(List<MapReduceOper> mergedPlans, MapReduceOper mro) throws PlanException, IOException{
@@ -433,8 +446,9 @@
     
     private MapReduceOper endSingleInputPlanWithStr(FileSpec fSpec) throws PlanException{
         if(compiledInputs.length>1) {
-            log.error("Received a multi input plan when expecting only a single input one.");
-            throw new PlanException("Received a multi input plan when expecting only a single input one.");
+            int errCode = 2023;
+            String msg = "Received a multi input plan when expecting only a single input one.";
+            throw new PlanException(msg, errCode, PigException.BUG);
         }
         MapReduceOper mro = compiledInputs[0];
         POStore str = getStore();
@@ -446,8 +460,9 @@
             mro.reducePlan.addAsLeaf(str);
             mro.setReduceDone(true);
         } else {
-            log.error("Both map and reduce phases have been done. This is unexpected while compiling!");
-            throw new PlanException("Both map and reduce phases have been done. This is unexpected while compiling!");
+            int errCode = 2022;
+            String msg = "Both map and reduce phases have been done. This is unexpected while compiling.";
+            throw new PlanException(msg, errCode, PigException.BUG);
         }
         return mro;
     }
@@ -523,10 +538,9 @@
             } else if (mro.isMapDone() && !mro.isReduceDone()) {
                 ret.add(mro);
             } else {
-                log.error(
-                        "Both map and reduce phases have been done. This is unexpected for a merge!");
-                throw new PlanException(
-                        "Both map and reduce phases have been done. This is unexpected for a merge!");
+                int errCode = 2027;
+                String msg = "Both map and reduce phases have been done. This is unexpected for a merge."; 
+                throw new PlanException(msg, errCode, PigException.BUG);
             }
         }
         merge(ret.get(0).mapPlan, mpLst);
@@ -594,9 +608,9 @@
             splitsSeen.put(op.getOperatorKey(), mro);
             curMROp = startNew(fSpec, mro);
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage());
-            pe.initCause(e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -604,9 +618,9 @@
         try{
             nonBlocking(op);
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage());
-            pe.initCause(e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -614,9 +628,9 @@
         try{
             nonBlocking(op);
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage());
-            pe.initCause(e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -625,9 +639,9 @@
             nonBlocking(op);
             addUDFs(op.getPlan());
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage());
-            pe.initCause(e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -635,9 +649,9 @@
         try{
             nonBlocking(op);
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage());
-            pe.initCause(e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -656,7 +670,9 @@
         try {
             lr.setIndex(0);
         } catch (ExecException e) {
-            throw new PlanException("Unable to set index on the newly created POLocalRearrange.", e);
+        	int errCode = 2058;
+        	String msg = "Unable to set index on the newly created POLocalRearrange.";
+            throw new PlanException(msg, errCode, PigException.BUG, e);
         }
         lr.setKeyType(DataType.TUPLE);
         lr.setPlans(eps);
@@ -719,9 +735,9 @@
                 log.warn("Both map and reduce phases have been done. This is unexpected while compiling!");
             }
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage());
-            pe.initCause(e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
 
@@ -733,9 +749,9 @@
                 for(PhysicalPlan ep : plans)
                     addUDFs(ep);
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage());
-            pe.initCause(e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -748,9 +764,9 @@
                     addUDFs(plan);
                 }
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage());
-            pe.initCause(e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -758,9 +774,9 @@
         try{
             blocking(op);
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage());
-            pe.initCause(e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -768,9 +784,9 @@
         try{
             nonBlocking(op);
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage());
-            pe.initCause(e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -778,9 +794,9 @@
         try{
             nonBlocking(op);
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage());
-            pe.initCause(e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -824,8 +840,9 @@
                     mro.reducePlan.addAsLeaf(str);
                     mro.setReduceDone(true);
                 } else {
-                    log.error("Both map and reduce phases have been done. This is unexpected while compiling!");
-                    throw new PlanException("Both map and reduce phases have been done. This is unexpected while compiling!");
+                	int errCode = 2022;
+                    String msg = "Both map and reduce phases have been done. This is unexpected while compiling.";
+                    throw new PlanException(msg, errCode, PigException.BUG);
                 }
             }
             for(int i=0;i<compiledInputs.length;i++){
@@ -838,8 +855,9 @@
             } else if (curMROp.isMapDone() && !curMROp.isReduceDone()) {
                 curMROp.reducePlan.addAsLeaf(op);
             } else {
-                log.error("Both map and reduce phases have been done. This is unexpected while compiling!");
-                throw new PlanException("Both map and reduce phases have been done. This is unexpected while compiling!");
+            	int errCode = 2022;
+                String msg = "Both map and reduce phases have been done. This is unexpected while compiling.";
+                throw new PlanException(msg, errCode, PigException.BUG);
             }
             List<List<PhysicalPlan>> joinPlans = op.getJoinPlans();
             if(joinPlans!=null)
@@ -913,9 +931,9 @@
             curMROp.reducePlan.addAsLeaf(nfe1);
             curMROp.setNeedsDistinctCombiner(true);
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage());
-            pe.initCause(e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
 
@@ -934,8 +952,9 @@
                 curMROp.UDFs.add(op.getMSortFunc().getFuncSpec().toString());
             }
         }catch(Exception e){
-            VisitorException pe = new VisitorException(e.getMessage(), e);
-            throw pe;
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new MRCompilerException(msg, errCode, PigException.BUG, e);
         }
     }
     
@@ -950,8 +969,9 @@
             }
             return ret;
         }
-        log.error("No expression plan found in POSort");
-        throw new PlanException("No Expression Plan found in POSort");
+        int errCode = 2026;
+        String msg = "No expression plan found in POSort.";
+        throw new PlanException(msg, errCode, PigException.BUG);
     }
     
     public MapReduceOper getSortJob(
@@ -1019,7 +1039,9 @@
                 fktv.visit();
                 keyType = fktv.keyType;
             } catch (VisitorException ve) {
-                throw new PlanException(ve);
+                int errCode = 2035;
+                String msg = "Internal error. Could not compute key type of sort operator.";
+                throw new PlanException(msg, errCode, PigException.BUG, ve);
             }
         }
         
@@ -1027,7 +1049,9 @@
         try {
             lr.setIndex(0);
         } catch (ExecException e) {
-            throw new PlanException("Unable to set index on newly create POLocalRearrange.", e);
+        	int errCode = 2058;
+        	String msg = "Unable to set index on newly created POLocalRearrange.";
+            throw new PlanException(msg, errCode, PigException.BUG, e);
         }
         lr.setKeyType((fields == null || fields.length>1) ? DataType.TUPLE :
             keyType);
@@ -1072,7 +1096,9 @@
 	        try {
                 lr_c2.setIndex(0);
             } catch (ExecException e) {
-                throw new PlanException("Unable to set index on newly created POLocalRearrange.", e);
+            	int errCode = 2058;
+            	String msg = "Unable to set index on newly created POLocalRearrange.";            	
+                throw new PlanException(msg, errCode, PigException.BUG, e);
             }
 	        lr_c2.setKeyType((fields.length>1) ? DataType.TUPLE : keyType);
 	        lr_c2.setPlans(eps_c2);
@@ -1165,7 +1191,9 @@
         try {
             lr.setIndex(0);
         } catch (ExecException e) {
-            throw new PlanException("Unable to set index on newly created POLocalRearrange.", e);
+        	int errCode = 2058;
+        	String msg = "Unable to set index on newly created POLocalRearrange.";
+            throw new PlanException(msg, errCode, PigException.BUG, e);
         }
         lr.setKeyType(DataType.CHARARRAY);
         lr.setPlans(eps);
@@ -1426,10 +1454,9 @@
             {
                 if (succs.size()!=1)
                 {
-                    String msg = new String("forEach can only have one successor");
-                    LogFactory.
-                        getLog(LastInputStreamingOptimizer.class).error(msg);
-                    throw new VisitorException(msg);
+                    int errCode = 2028;
+                    String msg = "ForEach can only have one successor. Found " + succs.size() + " successors.";
+                    throw new MRCompilerException(msg, errCode, PigException.BUG);
                 }
             }
             plan.remove(pack);
@@ -1437,10 +1464,9 @@
             try {
                 plan.replace(forEach, joinPackage);
             } catch (PlanException e) {
-                String msg = new String("Error rewrite POJoinPackage");
-                LogFactory.
-                getLog(LastInputStreamingOptimizer.class).error(msg);
-                throw new VisitorException(msg, e);
+                int errCode = 2029;
+                String msg = "Error rewriting POJoinPackage.";
+                throw new MRCompilerException(msg, errCode, PigException.BUG, e);
             }
             
             LogFactory.
@@ -1465,24 +1491,24 @@
             if (mr.reducePlan.isEmpty()) return;
             List<PhysicalOperator> mpLeaves = mr.reducePlan.getLeaves();
             if (mpLeaves.size() != 1) {
-                String msg = new String("Expected reduce to have single leaf");
-                log.error(msg);
-                throw new VisitorException(msg);
+                int errCode = 2024; 
+                String msg = "Expected reduce to have single leaf. Found " + mpLeaves.size() + " leaves.";
+                throw new MRCompilerException(msg, errCode, PigException.BUG);
             }
             PhysicalOperator mpLeaf = mpLeaves.get(0);
             if (!(mpLeaf instanceof POStore)) {
-                String msg = new String("Expected leaf of reduce plan to " +
-                    "always be POStore!");
-                log.error(msg);
-                throw new VisitorException(msg);
+                int errCode = 2025;
+                String msg = "Expected leaf of reduce plan to " +
+                    "always be POStore. Found " + mpLeaf.getClass().getSimpleName();
+                throw new MRCompilerException(msg, errCode, PigException.BUG);
             }
             List<PhysicalOperator> preds =
                 mr.reducePlan.getPredecessors(mpLeaf);
             if (preds == null) return;
             if (preds.size() > 1) {
-                String msg = new String("Expected mr to have single predecessor");
-                log.error(msg);
-                throw new VisitorException(msg);
+                int errCode = 2030;
+                String msg ="Expected reduce plan leaf to have a single predecessor. Found " + preds.size() + " predecessors.";
+                throw new MRCompilerException(msg, errCode, PigException.BUG);
             }
             PhysicalOperator pred = preds.get(0);
             if (!(pred instanceof POLocalRearrange)) return;
@@ -1490,15 +1516,15 @@
             // Next question, does the next MROper have an empty map?
             List<MapReduceOper> succs = mPlan.getSuccessors(mr);
             if (succs == null) {
-                String msg = new String("Found mro with POLocalRearrange as"
-                    + " last oper but with no succesor!");
-                log.error(msg);
-                throw new VisitorException(msg);
+                int errCode = 2031;
+                String msg = "Found map reduce operator with POLocalRearrange as"
+                    + " last oper but with no succesor.";
+                throw new MRCompilerException(msg, errCode, PigException.BUG);
             }
             if (succs.size() > 1) {
-                String msg = new String("Expected mr to have single successor");
-                log.error(msg);
-                throw new VisitorException(msg);
+                int errCode = 2032;
+                String msg = "Expected map reduce operator to have a single successor. Found " + succs.size() + " successors.";
+                throw new MRCompilerException(msg, errCode, PigException.BUG);
             }
             MapReduceOper succ = succs.get(0);
             List<PhysicalOperator> succMpLeaves = succ.mapPlan.getLeaves();
@@ -1517,7 +1543,9 @@
                 succ.mapPlan.add(pred);
                 succ.mapPlan.connect(load, pred);
             } catch (PlanException pe) {
-                throw new VisitorException(pe);
+                int errCode = 2033;
+                String msg = "Problems in rearranging map reduce operators in plan.";
+                throw new MRCompilerException(msg, errCode, PigException.BUG, pe);
             }
         }
     }
@@ -1547,16 +1575,16 @@
                 if (mr.reducePlan.isEmpty()) return;
                 List<PhysicalOperator> mpLeaves = mr.reducePlan.getLeaves();
                 if (mpLeaves.size() != 1) {
-                    String msg = new String("Expected reduce to have single leaf");
-                    log.error(msg);
-                    throw new IOException(msg);
+                    int errCode = 2024; 
+                    String msg = "Expected reduce to have single leaf. Found " + mpLeaves.size() + " leaves.";
+                    throw new PigException(msg, errCode, PigException.BUG);
                 }
                 PhysicalOperator mpLeaf = mpLeaves.get(0);
                 if (!(mpLeaf instanceof POStore)) {
-                    String msg = new String("Expected leaf of reduce plan to " +
-                        "always be POStore!");
-                    log.error(msg);
-                    throw new IOException(msg);
+                    int errCode = 2025;
+                    String msg = "Expected leaf of reduce plan to " +
+                        "always be POStore. Found " + mpLeaf.getClass().getSimpleName();
+                    throw new PigException(msg, errCode, PigException.BUG);
                 }
                 FileSpec oldSpec = ((POStore)mpLeaf).getSFile();
                 

Added: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompilerException.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompilerException.java?rev=737239&view=auto
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompilerException.java (added)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MRCompilerException.java Fri Jan 23 23:07:30 2009
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.backend.hadoop.executionengine.mapReduceLayer;
+
+import org.apache.pig.impl.plan.VisitorException;
+
+public class MRCompilerException extends VisitorException {
+
+    private static final long serialVersionUID = 2L;
+
+    /**
+     * Create a new MRCompilerException with null as the error message.
+     */
+    public MRCompilerException() {
+        super();
+    }
+
+    /**
+     * Create a new MRCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     */
+    public MRCompilerException(String message) {
+        super(message);
+    }
+
+    /**
+     * Create a new MRCompilerException with the specified cause.
+     *
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown.
+     */
+    public MRCompilerException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Create a new MRCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown.
+     */
+    public MRCompilerException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    /**
+     * Create a new MRCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     */
+    public MRCompilerException(String message, int errCode) {
+        super(message, errCode);
+    }
+
+    /**
+     * Create a new MRCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown. 
+     */
+    public MRCompilerException(String message, int errCode, Throwable cause) {
+        super(message, errCode, cause);
+    }
+
+    /**
+     * Create a new MRCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     */
+    public MRCompilerException(String message, int errCode, byte errSrc) {
+        super(message, errCode, errSrc);
+    }
+
+    /**
+     * Create a new MRCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown. 
+     */
+    public MRCompilerException(String message, int errCode, byte errSrc,
+            Throwable cause) {
+        super(message, errCode, errSrc, cause);
+    }
+
+    /**
+     * Create a new MRCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param retry - If the exception is retriable or not
+     */ 
+    public MRCompilerException(String message, int errCode, boolean retry) {
+        super(message, errCode, retry);
+    }
+
+    /**
+     * Create a new MRCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     * @param retry - If the exception is retriable or not
+     */
+    public MRCompilerException(String message, int errCode, byte errSrc,
+            boolean retry) {
+        super(message, errCode, errSrc, retry);
+    }
+
+    /**
+     * Create a new MRCompilerException with the specified message, error code, error source, retriable or not, detalied message for the developer and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     * @param retry - If the exception is retriable or not
+     * @param detailedMsg - The detailed message shown to the developer 
+     */
+    public MRCompilerException(String message, int errCode, byte errSrc,
+            boolean retry, String detailedMsg) {
+        super(message, errCode, errSrc, retry, detailedMsg);
+    }
+
+    /**
+     * Create a new MRCompilerException with the specified message, error code, error source, retriable or not, detalied message for the developer and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     * @param retry - If the exception is retriable or not
+     * @param detailedMsg - The detailed message shown to the developer 
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown.
+     */
+    public MRCompilerException(String message, int errCode, byte errSrc,
+            boolean retry, String detailedMsg, Throwable cause) {
+        super(message, errCode, errSrc, retry, detailedMsg, cause);
+    }
+    
+}

Modified: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java?rev=737239&r1=737238&r2=737239&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/MapReduceLauncher.java Fri Jan 23 23:07:30 2009
@@ -30,7 +30,6 @@
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.executionengine.ExecutionEngine;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
-import org.apache.pig.backend.hadoop.datastorage.HConfiguration;
 import org.apache.pig.backend.hadoop.executionengine.HExecutionEngine;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompiler.LastInputStreamingOptimizer;
@@ -38,7 +37,6 @@
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MRPrinter;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.MRStreamHandler;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.POPackageAnnotator;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POJoinPackage;
 import org.apache.pig.impl.plan.PlanException;

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=737239&r1=737238&r2=737239&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 Jan 23 23:07:30 2009
@@ -32,18 +32,15 @@
 import org.apache.pig.EvalFunc;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.LoadFunc;
+import org.apache.pig.PigException;
 import org.apache.pig.data.DataType;
-import org.apache.pig.data.Tuple;
-import org.apache.pig.data.TupleFactory;
 import org.apache.pig.impl.PigContext;
 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.plans.PhysicalPlan;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.*;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.*;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.ExpressionOperator;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.BinaryExpressionOperator;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.UnaryExpressionOperator;
 import org.apache.pig.builtin.BinStorage;
 import org.apache.pig.impl.builtin.GFCross;
 import org.apache.pig.impl.io.FileLocalizer;
@@ -117,9 +114,9 @@
                 // currentExprPlan.connect(from, exprOp);
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -147,9 +144,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -176,9 +173,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -205,9 +202,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -234,9 +231,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -263,9 +260,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -291,9 +288,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -319,9 +316,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -347,9 +344,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -375,9 +372,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -403,9 +400,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -431,9 +428,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -456,8 +453,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan" + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -480,8 +478,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan" + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -502,8 +501,9 @@
         try {
             currentPlan.connect(from, exprOp);
         } catch (PlanException e) {
-            log.error("Invalid physical operators in the physical plan" + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
     }
 
@@ -591,11 +591,13 @@
                 currentPlan.connect(physOp, poGlobal);
             }
         } catch (PlanException e1) {
-            log.error("Invalid physical operators in the physical plan"
-                    + e1.getMessage());
-            throw new VisitorException(e1);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e1);
         } catch (ExecException e) {
-            throw new VisitorException(e);
+        	int errCode = 2058;
+        	String msg = "Unable to set index on newly create POLocalRearrange.";
+            throw new VisitorException(msg, errCode, PigException.BUG, e);
         }
         
         poPackage.setKeyType(DataType.TUPLE);
@@ -624,9 +626,9 @@
         try{
             currentPlan.connect(poPackage, fe);
         }catch (PlanException e1) {
-            log.error("Invalid physical operators in the physical plan"
-                    + e1.getMessage());
-            throw new VisitorException(e1);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e1);
         }
         LogToPhyMap.put(cs, fe);
     }
@@ -649,9 +651,9 @@
         try {
             currentPlan.connect(poGlobal, poPackage);
         } catch (PlanException e1) {
-            log.error("Invalid physical operators in the physical plan"
-                    + e1.getMessage());
-            throw new VisitorException(e1);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e1);
         }
 
         int count = 0;
@@ -679,7 +681,9 @@
             try {
                 physOp.setIndex(count++);
             } catch (ExecException e1) {
-                throw new VisitorException(e1);
+            	int errCode = 2058;
+            	String msg = "Unable to set index on newly create POLocalRearrange.";
+                throw new VisitorException(msg, errCode, PigException.BUG, e1);
             }
             if (plans.size() > 1) {
                 type = DataType.TUPLE;
@@ -696,9 +700,9 @@
                 currentPlan.connect(LogToPhyMap.get(op), physOp);
                 currentPlan.connect(physOp, poGlobal);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
 
         }
@@ -754,7 +758,9 @@
             pfrj = new POFRJoin(new OperatorKey(scope,nodeGen.getNextNodeId(scope)),frj.getRequestedParallelism(),
                                         inp, ppLists, keyTypes, null, fragment);
         } catch (ExecException e1) {
-            throw new VisitorException(e1);
+        	int errCode = 2058;
+        	String msg = "Unable to set index on newly create POLocalRearrange.";
+            throw new VisitorException(msg, errCode, PigException.BUG, e1);
         }
         pfrj.setResultType(DataType.TUPLE);
         currentPlan.add(pfrj);
@@ -762,9 +768,9 @@
             try {
                 currentPlan.connect(LogToPhyMap.get(op), pfrj);
             } catch (PlanException e) {
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
         LogToPhyMap.put(frj, pfrj);
@@ -802,13 +808,21 @@
 
         List<LogicalOperator> op = filter.getPlan().getPredecessors(filter);
 
-        PhysicalOperator from = LogToPhyMap.get(op.get(0));
+        PhysicalOperator from;
+        if(op != null) {
+            from = LogToPhyMap.get(op.get(0));
+        } else {
+            int errCode = 2051;
+            String msg = "Did not find a predecessor for Filter." ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG);
+        }
+        
         try {
             currentPlan.connect(from, poFilter);
         } catch (PlanException e) {
-            log.error("Invalid physical operators in the physical plan"
-                    + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
     }
 
@@ -823,13 +837,21 @@
         
         List<LogicalOperator> op = stream.getPlan().getPredecessors(stream);
 
-        PhysicalOperator from = LogToPhyMap.get(op.get(0));
+        PhysicalOperator from;
+        if(op != null) {
+            from = LogToPhyMap.get(op.get(0));
+        } else {                
+            int errCode = 2051;
+            String msg = "Did not find a predecessor for Stream." ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG);
+        }
+        
         try {
             currentPlan.connect(from, poStream);
         } catch (PlanException e) {
-            log.error("Invalid physical operators in the physical plan"
-                    + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
     }
 
@@ -857,10 +879,9 @@
             try {
                 currentPlan.connect(from, exprOp);
             } catch (PlanException e) {
-
-                log.error("Invalid physical operators in the physical plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }
@@ -904,9 +925,9 @@
         try {
             currentPlan.connect(from, poFE);
         } catch (PlanException e) {
-            log.error("Invalid physical operators in the physical plan"
-                    + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
 
     }
@@ -948,13 +969,23 @@
         // sort.setRequestedParallelism(s.getType());
         LogToPhyMap.put(s, sort);
         currentPlan.add(sort);
-        PhysicalOperator from =
-            LogToPhyMap.get(s.getPlan().getPredecessors(s).get(0));
+        List<LogicalOperator> op = s.getPlan().getPredecessors(s); 
+        PhysicalOperator from;
+        
+        if(op != null) {
+            from = LogToPhyMap.get(op.get(0));
+        } else {
+            int errCode = 2051;
+            String msg = "Did not find a predecessor for Sort." ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG);            
+        }
+        
         try {
             currentPlan.connect(from, sort);
         } catch (PlanException e) {
-            log.error("Invalid physical operator in the plan" + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
 
         sort.setResultType(s.getType());
@@ -973,13 +1004,23 @@
         LogToPhyMap.put(op, physOp);
         currentPlan.add(physOp);
         // Distinct will only have a single input
-        PhysicalOperator from = LogToPhyMap.get(op.getPlan().getPredecessors(op)
-                .get(0));
+        List<LogicalOperator> inputs = op.getPlan().getPredecessors(op);
+        PhysicalOperator from; 
+        
+        if(inputs != null) {
+            from = LogToPhyMap.get(inputs.get(0));
+        } else {
+            int errCode = 2051;
+            String msg = "Did not find a predecessor for Distinct." ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG);            
+        }
+
         try {
             currentPlan.connect(from, physOp);
         } catch (PlanException e) {
-            log.error("Invalid physical operator in the plan" + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
     }
 
@@ -992,20 +1033,45 @@
         try {
             splStrFile = new FileSpec(FileLocalizer.getTemporaryPath(null, pc).toString(),new FuncSpec(BinStorage.class.getName()));
         } catch (IOException e1) {
-            log.error("Unable to obtain a temporary path because " + e1.getMessage());
-            throw new VisitorException(e1);
+            byte errSrc = pc.getErrorSource();
+            int errCode = 0;
+            switch(errSrc) {
+            case PigException.BUG:
+                errCode = 2016;
+                break;
+            case PigException.REMOTE_ENVIRONMENT:
+                errCode = 6002;
+                break;
+            case PigException.USER_ENVIRONMENT:
+                errCode = 4003;
+                break;
+            }
+            String msg = "Unable to obtain a temporary path." ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, errSrc, e1);
+
         }
         ((POSplit)physOp).setSplitStore(splStrFile);
         LogToPhyMap.put(split, physOp);
 
         currentPlan.add(physOp);
-        PhysicalOperator from = LogToPhyMap.get(split.getPlan()
-                .getPredecessors(split).get(0));
+
+        List<LogicalOperator> op = split.getPlan().getPredecessors(split); 
+        PhysicalOperator from;
+        
+        if(op != null) {
+            from = LogToPhyMap.get(op.get(0));
+        } else {
+            int errCode = 2051;
+            String msg = "Did not find a predecessor for Split." ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG);            
+        }        
+
         try {
             currentPlan.connect(from, physOp);
         } catch (PlanException e) {
-            log.error("Invalid physical operator in the plan" + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
     }
 
@@ -1028,13 +1094,24 @@
         ((POFilter) physOp).setPlan((PhysicalPlan) currentPlan);
         currentPlan = currentPlans.pop();
         currentPlan.add(physOp);
-        PhysicalOperator from = LogToPhyMap.get(split.getPlan()
-                .getPredecessors(split).get(0));
+
+        List<LogicalOperator> op = split.getPlan().getPredecessors(split); 
+        PhysicalOperator from;
+        
+        if(op != null) {
+            from = LogToPhyMap.get(op.get(0));
+        } else {
+            int errCode = 2051;
+            String msg = "Did not find a predecessor for Split Output." ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG);            
+        }        
+        
         try {
             currentPlan.connect(from, physOp);
         } catch (PlanException e) {
-            log.error("Invalid physical operator in the plan" + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
     }
 
@@ -1061,9 +1138,9 @@
                 try {
                     currentPlan.connect(from, p);
                 } catch (PlanException e) {
-                    log.error("Invalid physical operator in the plan"
-                            + e.getMessage());
-                    throw new VisitorException(e);
+                    int errCode = 2015;
+                    String msg = "Invalid physical operators in the physical plan" ;
+                    throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
                 }
             }
         }
@@ -1094,13 +1171,24 @@
         store.setSFile(loStore.getOutputFile());
         store.setPc(pc);
         currentPlan.add(store);
-        PhysicalOperator from = LogToPhyMap.get(loStore
-                .getPlan().getPredecessors(loStore).get(0));
+        
+        List<LogicalOperator> op = loStore.getPlan().getPredecessors(loStore); 
+        PhysicalOperator from;
+        
+        if(op != null) {
+            from = LogToPhyMap.get(op.get(0));
+        } else {
+            int errCode = 2051;
+            String msg = "Did not find a predecessor for Store." ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG);            
+        }        
+
         try {
             currentPlan.connect(from, store);
         } catch (PlanException e) {
-            log.error("Invalid physical operator in the plan" + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
         LogToPhyMap.put(loStore, store);
     }
@@ -1143,9 +1231,9 @@
             try {
                 currentPlan.connect(from, physOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operator in the plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
 
@@ -1160,15 +1248,26 @@
         currentPlan.add(physOp);
 
         LogToPhyMap.put(op, physOp);
-        ExpressionOperator from = (ExpressionOperator) LogToPhyMap.get(op
-                .getPlan().getPredecessors(op).get(0));
+
+        List<LogicalOperator> inputs = op.getPlan().getPredecessors(op); 
+        ExpressionOperator from;
+        
+        if(inputs != null) {
+            from = (ExpressionOperator)LogToPhyMap.get(inputs.get(0));
+        } else {
+            int errCode = 2051;
+            String msg = "Did not find a predecessor for Negative." ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG);            
+        }
+   
         ((PONegative) physOp).setExpr(from);
         ((PONegative) physOp).setResultType(op.getType());
         try {
             currentPlan.connect(from, physOp);
         } catch (PlanException e) {
-            log.error("Invalid physical operator in the plan" + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
 
     }
@@ -1182,14 +1281,25 @@
         currentPlan.add(physOp);
 
         LogToPhyMap.put(op, physOp);
-        ExpressionOperator from = (ExpressionOperator) LogToPhyMap.get(op
-                .getPlan().getPredecessors(op).get(0));
+
+        List<LogicalOperator> inputs = op.getPlan().getPredecessors(op); 
+        ExpressionOperator from;
+        
+        if(inputs != null) {
+            from = (ExpressionOperator)LogToPhyMap.get(inputs.get(0));
+        } else {
+            int errCode = 2051;
+            String msg = "Did not find a predecessor for Null." ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG);            
+        }
+
         ((POIsNull) physOp).setExpr(from);
         try {
             currentPlan.connect(from, physOp);
         } catch (PlanException e) {
-            log.error("Invalid physical operator in the plan" + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
 
     }
@@ -1210,8 +1320,9 @@
         try {
             currentPlan.connect(from, physOp);
         } catch (PlanException e) {
-            log.error("Invalid physical operator in the plan" + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
 
     }
@@ -1234,8 +1345,9 @@
         try {
             currentPlan.connect(from, physOp);
         } catch (PlanException e) {
-            log.error("Invalid physical operator in the plan" + e.getMessage());
-            throw new VisitorException(e);
+            int errCode = 2015;
+            String msg = "Invalid physical operators in the physical plan" ;
+            throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
         }
 
     }
@@ -1251,11 +1363,20 @@
 
             List<LogicalOperator> op = limit.getPlan().getPredecessors(limit);
 
-            PhysicalOperator from = LogToPhyMap.get(op.get(0));
+            PhysicalOperator from;
+            if(op != null) {
+                from = LogToPhyMap.get(op.get(0));
+            } else {
+                int errCode = 2051;
+                String msg = "Did not find a predecessor for Limit." ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG);                
+            }
             try {
                     currentPlan.connect(from, poLimit);
             } catch (PlanException e) {
-                    log.error("Invalid physical operators in the physical plan" + e.getMessage());
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
     }
 
@@ -1274,9 +1395,9 @@
             try {
                 currentPlan.connect(from, physOp);
             } catch (PlanException e) {
-                log.error("Invalid physical operator in the plan"
-                        + e.getMessage());
-                throw new VisitorException(e);
+                int errCode = 2015;
+                String msg = "Invalid physical operators in the physical plan" ;
+                throw new LogicalToPhysicalTranslatorException(msg, errCode, PigException.BUG, e);
             }
         }
     }

Added: hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogicalToPhysicalTranslatorException.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogicalToPhysicalTranslatorException.java?rev=737239&view=auto
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogicalToPhysicalTranslatorException.java (added)
+++ hadoop/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/LogicalToPhysicalTranslatorException.java Fri Jan 23 23:07:30 2009
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.backend.hadoop.executionengine.physicalLayer;
+
+import org.apache.pig.impl.plan.VisitorException;
+
+public class LogicalToPhysicalTranslatorException extends VisitorException {
+
+    /**
+     * Create a new LogicalToPhysicalTranslatorException with null as the error message.
+     */
+    public LogicalToPhysicalTranslatorException() {
+        super();
+    }
+
+    /**
+     * Create a new LogicalToPhysicalTranslatorException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     */
+    public LogicalToPhysicalTranslatorException(String message) {
+        super(message);
+    }
+
+    /**
+     * Create a new LogicalToPhysicalTranslatorException with the specified cause.
+     *
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown.
+     */
+    public LogicalToPhysicalTranslatorException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Create a new LogicalToPhysicalTranslatorException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown.
+     */
+    public LogicalToPhysicalTranslatorException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    /**
+     * Create a new LogicalToPhysicalTranslatorException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     */
+    public LogicalToPhysicalTranslatorException(String message, int errCode) {
+        super(message, errCode);
+    }
+
+    /**
+     * Create a new LogicalToPhysicalTranslatorException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown. 
+     */
+    public LogicalToPhysicalTranslatorException(String message, int errCode, Throwable cause) {
+        super(message, errCode, cause);
+    }
+
+    /**
+     * Create a new LogicalToPhysicalTranslatorException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     */
+    public LogicalToPhysicalTranslatorException(String message, int errCode, byte errSrc) {
+        super(message, errCode, errSrc);
+    }
+
+    /**
+     * Create a new LogicalToPhysicalTranslatorException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown. 
+     */
+    public LogicalToPhysicalTranslatorException(String message, int errCode, byte errSrc,
+            Throwable cause) {
+        super(message, errCode, errSrc, cause);
+    }
+
+    /**
+     * Create a new LogicalToPhysicalTranslatorException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param retry - If the exception is retriable or not
+     */ 
+    public LogicalToPhysicalTranslatorException(String message, int errCode, boolean retry) {
+        super(message, errCode, retry);
+    }
+
+    /**
+     * Create a new LogicalToPhysicalTranslatorException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     * @param retry - If the exception is retriable or not
+     */
+    public LogicalToPhysicalTranslatorException(String message, int errCode, byte errSrc,
+            boolean retry) {
+        super(message, errCode, errSrc, retry);
+    }
+
+    /**
+     * Create a new LogicalToPhysicalTranslatorException with the specified message, error code, error source, retriable or not, detalied message for the developer and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     * @param retry - If the exception is retriable or not
+     * @param detailedMsg - The detailed message shown to the developer 
+     */
+    public LogicalToPhysicalTranslatorException(String message, int errCode, byte errSrc,
+            boolean retry, String detailedMsg) {
+        super(message, errCode, errSrc, retry, detailedMsg);
+    }
+
+    /**
+     * Create a new LogicalToPhysicalTranslatorException with the specified message, error code, error source, retriable or not, detalied message for the developer and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     * @param retry - If the exception is retriable or not
+     * @param detailedMsg - The detailed message shown to the developer 
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown.
+     */
+    public LogicalToPhysicalTranslatorException(String message, int errCode, byte errSrc,
+            boolean retry, String detailedMsg, Throwable cause) {
+        super(message, errCode, errSrc, retry, detailedMsg, cause);
+    }
+    
+}