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 [3/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/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=737239&r1=737238&r2=737239&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 Jan 23 23:07:30 2009
@@ -40,963 +40,1001 @@
 import org.apache.pig.impl.plan.VisitorException;
 
 /**
- * This is just a cast that converts DataByteArray into either
- * String or Integer. Just added it for testing the POUnion. 
- * Need the full operator implementation.
+ * This is just a cast that converts DataByteArray into either String or
+ * Integer. Just added it for testing the POUnion. Need the full operator
+ * implementation.
  */
 public class POCast extends ExpressionOperator {
-    private FuncSpec loadFSpec = null;
+	private FuncSpec loadFSpec = null;
 	transient private LoadFunc load;
 	private Log log = LogFactory.getLog(getClass());
-    private boolean castNotNeeded = false;
-	
-    private static final long serialVersionUID = 1L;
-
-    public POCast(OperatorKey k) {
-        super(k);
-        // TODO Auto-generated constructor stub
-    }
-
-    public POCast(OperatorKey k, int rp) {
-        super(k, rp);
-        // TODO Auto-generated constructor stub
-    }
-    
-    private void instantiateFunc() {
-        if(load!=null) return;
-        if(this.loadFSpec != null) {
-            this.load = (LoadFunc) PigContext.instantiateFuncFromSpec(this.loadFSpec);
-        }
-    }
-    
-    public void setLoadFSpec(FuncSpec lf) {
-    	this.loadFSpec = lf;
-        instantiateFunc();
-    }
-
-    @Override
-    public void visit(PhyPlanVisitor v) throws VisitorException {
-        v.visitCast(this);
-
-    }
-
-    @Override
-    public String name() {
-        return "Cast" + "[" + DataType.findTypeName(resultType) + "]" +" - " + mKey.toString();
-    }
-
-    @Override
-    public boolean supportsMultipleInputs() {
-        // TODO Auto-generated method stub
-        return false;
-    }
-
-    @Override
-    public Result getNext(Integer i) throws ExecException {
-    	PhysicalOperator in = inputs.get(0);
-        Byte castToType = DataType.INTEGER;
-    	Byte resultType = in.getResultType();
-        switch(resultType) {
-        case DataType.BAG : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;
-        }
-        
-        case DataType.TUPLE : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;
-        }
-
-        case DataType.BYTEARRAY : {
-        	DataByteArray dba = null;
-        	Result res = in.getNext(dba);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		//res.result = new Integer(Integer.valueOf((((DataByteArray)res.result).toString())));
-        	    if(castNotNeeded) {
-                    // we examined the data once before and
-                    // determined that the input is the same
-                    // type as the type we are casting to
-                    // so just send the input out as output
-                    return res;
-                }
-        		try {
-                    dba = (DataByteArray) res.result;
-                } catch (ClassCastException e) {
-                    // check if the type of res.result is
-                    // same as the type we are trying to cast to
-                    if(DataType.findType(res.result) == castToType) {
-                        // remember this for future calls
-                        castNotNeeded  = true;
-                        // just return the output
-                        return res;
-                    } else {
-                        // the input is a differen type
-                        // rethrow the exception
-                        throw e;
-                    }
-
-                }
-        		try {
-                    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);
-                    }
+	private boolean castNotNeeded = false;
+
+	private static final long serialVersionUID = 1L;
+
+	public POCast(OperatorKey k) {
+		super(k);
+		// TODO Auto-generated constructor stub
+	}
+
+	public POCast(OperatorKey k, int rp) {
+		super(k, rp);
+		// TODO Auto-generated constructor stub
+	}
+
+	private void instantiateFunc() {
+		if (load != null)
+			return;
+		if (this.loadFSpec != null) {
+			this.load = (LoadFunc) PigContext
+					.instantiateFuncFromSpec(this.loadFSpec);
+		}
+	}
+
+	public void setLoadFSpec(FuncSpec lf) {
+		this.loadFSpec = lf;
+		instantiateFunc();
+	}
+
+	@Override
+	public void visit(PhyPlanVisitor v) throws VisitorException {
+		v.visitCast(this);
+
+	}
+
+	@Override
+	public String name() {
+		return "Cast" + "[" + DataType.findTypeName(resultType) + "]" + " - "
+				+ mKey.toString();
+	}
+
+	@Override
+	public boolean supportsMultipleInputs() {
+		// TODO Auto-generated method stub
+		return false;
+	}
+
+	@Override
+	public Result getNext(Integer i) throws ExecException {
+		PhysicalOperator in = inputs.get(0);
+		Byte castToType = DataType.INTEGER;
+		Byte resultType = in.getResultType();
+		switch (resultType) {
+		case DataType.BAG: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.TUPLE: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.BYTEARRAY: {
+			DataByteArray dba = null;
+			Result res = in.getNext(dba);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				// res.result = new
+				// Integer(Integer.valueOf((((DataByteArray)res.result).toString())));
+				if (castNotNeeded) {
+					// we examined the data once before and
+					// determined that the input is the same
+					// type as the type we are casting to
+					// so just send the input out as output
+					return res;
+				}
+				try {
+					dba = (DataByteArray) res.result;
+				} catch (ClassCastException e) {
+					// check if the type of res.result is
+					// same as the type we are trying to cast to
+					if (DataType.findType(res.result) == castToType) {
+						// remember this for future calls
+						castNotNeeded = true;
+						// just return the output
+						return res;
+					} else {
+						// the input is a differen type
+						// rethrow the exception
+						throw e;
+					}
+
+				}
+				try {
+					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);
+					}
+				} catch (ExecException ee) {
+					throw ee;
 				} catch (IOException e) {
 					log.error("Error while casting from ByteArray to Integer");
 				}
-        	}
-        	return res;
-        }
-        
-        case DataType.MAP : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;        	
-        }
-        
-        case DataType.BOOLEAN : {
-        	Boolean b = null;
-        	Result res = in.getNext(b);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		if (((Boolean)res.result) == true) res.result = new Integer(1);
-                else res.result = new Integer(0);
-        	}
-        	return res;
-        }
-        case DataType.INTEGER : {
-        	
-        	Result res = in.getNext(i);
-        	return res;
-        }
-
-        case DataType.DOUBLE : {
-        	Double d = null;
-        	Result res = in.getNext(d);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		//res.result = DataType.toInteger(res.result);
-        		res.result = new Integer(((Double)res.result).intValue());
-        	}
-        	return res;
-        }
-
-        case DataType.LONG : {
-        	Long l = null;
-        	Result res = in.getNext(l);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Integer(((Long)res.result).intValue());
-        	}
-        	return res;
-        }
-        
-        case DataType.FLOAT : {
-        	Float f = null;
-        	Result res = in.getNext(f);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Integer(((Float)res.result).intValue());
-        	}
-        	return res;
-        }
-        
-        case DataType.CHARARRAY : {
-        	String str = null;
-        	Result res = in.getNext(str);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Integer(Integer.valueOf((String)res.result));
-        	}
-        	return res;
-        }
-        
-        }
-        
-        Result res = new Result();
-        res.returnStatus = POStatus.STATUS_ERR;
-        return res;
-    }
-    
-    @Override
-    public Result getNext(Long l) throws ExecException {
-    	PhysicalOperator in = inputs.get(0);
-        Byte castToType = DataType.LONG;
-    	Byte resultType = in.getResultType();
-        switch(resultType) {
-        case DataType.BAG : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;
-        }
-        
-        case DataType.TUPLE : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;
-        }
-
-        case DataType.MAP : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;        	
-        }
-                
-        case DataType.BYTEARRAY : {
-        	DataByteArray dba = null;
-        	Result res = in.getNext(dba);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        	    if(castNotNeeded) {
-                    // we examined the data once before and
-                    // determined that the input is the same
-                    // type as the type we are casting to
-                    // so just send the input out as output
-                    return res;
-                }
-        		//res.result = new Long(Long.valueOf((((DataByteArray)res.result).toString())));
-        		try {
-                    dba = (DataByteArray) res.result;
-                } catch (ClassCastException e) {
-                    // check if the type of res.result is
-                    // same as the type we are trying to cast to
-                    if(DataType.findType(res.result) == castToType) {
-                        // remember this for future calls
-                        castNotNeeded  = true;
-                        // just return the output
-                        return res;
-                    } else {
-                        // the input is a differen type
-                        // rethrow the exception
-                        throw e;
-                    }
-
-                }
-        		try {
-                    if(null != load) {
-					    res.result = load.bytesToLong(dba.get());
-                    } else {
-					    String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to long.";
-					    log.error(msg);
-                        throw new ExecException(msg);
-                    }
+			}
+			return res;
+		}
+
+		case DataType.MAP: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.BOOLEAN: {
+			Boolean b = null;
+			Result res = in.getNext(b);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				if (((Boolean) res.result) == true)
+					res.result = new Integer(1);
+				else
+					res.result = new Integer(0);
+			}
+			return res;
+		}
+		case DataType.INTEGER: {
+
+			Result res = in.getNext(i);
+			return res;
+		}
+
+		case DataType.DOUBLE: {
+			Double d = null;
+			Result res = in.getNext(d);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				// res.result = DataType.toInteger(res.result);
+				res.result = new Integer(((Double) res.result).intValue());
+			}
+			return res;
+		}
+
+		case DataType.LONG: {
+			Long l = null;
+			Result res = in.getNext(l);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Integer(((Long) res.result).intValue());
+			}
+			return res;
+		}
+
+		case DataType.FLOAT: {
+			Float f = null;
+			Result res = in.getNext(f);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Integer(((Float) res.result).intValue());
+			}
+			return res;
+		}
+
+		case DataType.CHARARRAY: {
+			String str = null;
+			Result res = in.getNext(str);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Integer(Integer.valueOf((String) res.result));
+			}
+			return res;
+		}
+
+		}
+
+		Result res = new Result();
+		res.returnStatus = POStatus.STATUS_ERR;
+		return res;
+	}
+
+	@Override
+	public Result getNext(Long l) throws ExecException {
+		PhysicalOperator in = inputs.get(0);
+		Byte castToType = DataType.LONG;
+		Byte resultType = in.getResultType();
+		switch (resultType) {
+		case DataType.BAG: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.TUPLE: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.MAP: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.BYTEARRAY: {
+			DataByteArray dba = null;
+			Result res = in.getNext(dba);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				if (castNotNeeded) {
+					// we examined the data once before and
+					// determined that the input is the same
+					// type as the type we are casting to
+					// so just send the input out as output
+					return res;
+				}
+				// res.result = new
+				// Long(Long.valueOf((((DataByteArray)res.result).toString())));
+				try {
+					dba = (DataByteArray) res.result;
+				} catch (ClassCastException e) {
+					// check if the type of res.result is
+					// same as the type we are trying to cast to
+					if (DataType.findType(res.result) == castToType) {
+						// remember this for future calls
+						castNotNeeded = true;
+						// just return the output
+						return res;
+					} else {
+						// the input is a differen type
+						// rethrow the exception
+						throw e;
+					}
+
+				}
+				try {
+					if (null != load) {
+						res.result = load.bytesToLong(dba.get());
+					} else {
+						String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to long.";
+						log.error(msg);
+						throw new ExecException(msg);
+					}
+				} catch (ExecException ee) {
+					throw ee;
 				} catch (IOException e) {
 					log.error("Error while casting from ByteArray to Long");
 				}
-        	}
-        	return res;
-        }
-        
-        case DataType.BOOLEAN : {
-        	Boolean b = null;
-        	Result res = in.getNext(b);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		if (((Boolean)res.result) == true) res.result = new Long(1);
-                else res.result = new Long(0);
-        	}
-        	return res;
-        }
-        case DataType.INTEGER : {
-        	Integer dummyI = null;
-        	Result res = in.getNext(dummyI);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Long(((Integer)res.result).longValue());
-        	}
-        	return res;
-        }
-
-        case DataType.DOUBLE : {
-        	Double d = null;
-        	Result res = in.getNext(d);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		//res.result = DataType.toInteger(res.result);
-        		res.result = new Long(((Double)res.result).longValue());
-        	}
-        	return res;
-        }
-
-        case DataType.LONG : {
-        	
-        	Result res = in.getNext(l);
-        	
-        	return res;
-        }
-        
-        case DataType.FLOAT : {
-        	Float f = null;
-        	Result res = in.getNext(f);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Long(((Float)res.result).longValue());
-        	}
-        	return res;
-        }
-        
-        case DataType.CHARARRAY : {
-        	String str = null;
-        	Result res = in.getNext(str);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Long(Long.valueOf((String)res.result));
-        	}
-        	return res;
-        }
-        
-        }
-        
-        Result res = new Result();
-        res.returnStatus = POStatus.STATUS_ERR;
-        return res;
-    }
-    
-    @Override
-    public Result getNext(Double d) throws ExecException {
-    	PhysicalOperator in = inputs.get(0);
-        Byte castToType = DataType.DOUBLE;
-    	Byte resultType = in.getResultType();
-        switch(resultType) {
-        case DataType.BAG : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;
-        }
-        
-        case DataType.TUPLE : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;
-        }
-
-        case DataType.MAP : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;        	
-        }
-                
-        case DataType.BYTEARRAY : {
-        	DataByteArray dba = null;
-        	Result res = in.getNext(dba);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		//res.result = new Double(Double.valueOf((((DataByteArray)res.result).toString())));
-        	    if(castNotNeeded) {
-                    // we examined the data once before and
-                    // determined that the input is the same
-                    // type as the type we are casting to
-                    // so just send the input out as output
-                    return res;
-                }
-        		try {
-                    dba = (DataByteArray) res.result;
-                } catch (ClassCastException e) {
-                    // check if the type of res.result is
-                    // same as the type we are trying to cast to
-                    if(DataType.findType(res.result) == castToType) {
-                        // remember this for future calls
-                        castNotNeeded  = true;
-                        // just return the output
-                        return res;
-                    } else {
-                        // the input is a differen type
-                        // rethrow the exception
-                        throw e;
-                    }
-
-                }
-        		try {
-                    if(null != load) {
-					    res.result = load.bytesToDouble(dba.get());
-                    } else {
-					    String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to double.";
-					    log.error(msg);
-                        throw new ExecException(msg);
-                    }
+			}
+			return res;
+		}
+
+		case DataType.BOOLEAN: {
+			Boolean b = null;
+			Result res = in.getNext(b);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				if (((Boolean) res.result) == true)
+					res.result = new Long(1);
+				else
+					res.result = new Long(0);
+			}
+			return res;
+		}
+		case DataType.INTEGER: {
+			Integer dummyI = null;
+			Result res = in.getNext(dummyI);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Long(((Integer) res.result).longValue());
+			}
+			return res;
+		}
+
+		case DataType.DOUBLE: {
+			Double d = null;
+			Result res = in.getNext(d);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				// res.result = DataType.toInteger(res.result);
+				res.result = new Long(((Double) res.result).longValue());
+			}
+			return res;
+		}
+
+		case DataType.LONG: {
+
+			Result res = in.getNext(l);
+
+			return res;
+		}
+
+		case DataType.FLOAT: {
+			Float f = null;
+			Result res = in.getNext(f);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Long(((Float) res.result).longValue());
+			}
+			return res;
+		}
+
+		case DataType.CHARARRAY: {
+			String str = null;
+			Result res = in.getNext(str);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Long(Long.valueOf((String) res.result));
+			}
+			return res;
+		}
+
+		}
+
+		Result res = new Result();
+		res.returnStatus = POStatus.STATUS_ERR;
+		return res;
+	}
+
+	@Override
+	public Result getNext(Double d) throws ExecException {
+		PhysicalOperator in = inputs.get(0);
+		Byte castToType = DataType.DOUBLE;
+		Byte resultType = in.getResultType();
+		switch (resultType) {
+		case DataType.BAG: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.TUPLE: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.MAP: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.BYTEARRAY: {
+			DataByteArray dba = null;
+			Result res = in.getNext(dba);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				// res.result = new
+				// Double(Double.valueOf((((DataByteArray)res.result).toString())));
+				if (castNotNeeded) {
+					// we examined the data once before and
+					// determined that the input is the same
+					// type as the type we are casting to
+					// so just send the input out as output
+					return res;
+				}
+				try {
+					dba = (DataByteArray) res.result;
+				} catch (ClassCastException e) {
+					// check if the type of res.result is
+					// same as the type we are trying to cast to
+					if (DataType.findType(res.result) == castToType) {
+						// remember this for future calls
+						castNotNeeded = true;
+						// just return the output
+						return res;
+					} else {
+						// the input is a differen type
+						// rethrow the exception
+						throw e;
+					}
+
+				}
+				try {
+					if (null != load) {
+						res.result = load.bytesToDouble(dba.get());
+					} else {
+						String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to double.";
+						log.error(msg);
+						throw new ExecException(msg);
+					}
+				} catch (ExecException ee) {
+					throw ee;
 				} catch (IOException e) {
 					log.error("Error while casting from ByteArray to Double");
 				}
-        	}
-        	return res;
-        }
-        
-        case DataType.BOOLEAN : {
-        	Boolean b = null;
-        	Result res = in.getNext(b);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		if (((Boolean)res.result) == true) res.result = new Double(1);
-                else res.result = new Double(0);
-        	}
-        	return res;
-        }
-        case DataType.INTEGER : {
-        	Integer dummyI = null;
-        	Result res = in.getNext(dummyI);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Double(((Integer)res.result).doubleValue());
-        	}
-        	return res;
-        }
-
-        case DataType.DOUBLE : {
-        	
-        	Result res = in.getNext(d);
-        	
-        	return res;
-        }
-
-        case DataType.LONG : {
-        	Long l = null;
-        	Result res = in.getNext(l);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Double(((Long)res.result).doubleValue());
-        	}
-        	return res;
-        }
-        
-        case DataType.FLOAT : {
-        	Float f = null;
-        	Result res = in.getNext(f);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Double(((Float)res.result).doubleValue());
-        	}
-        	return res;
-        }
-        
-        case DataType.CHARARRAY : {
-        	String str = null;
-        	Result res = in.getNext(str);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Double(Double.valueOf((String)res.result));
-        	}
-        	return res;
-        }
-        
-        }
-        
-        Result res = new Result();
-        res.returnStatus = POStatus.STATUS_ERR;
-        return res;
-    }
-    
-    @Override
-    public Result getNext(Float f) throws ExecException {
-    	PhysicalOperator in = inputs.get(0);
-        Byte castToType = DataType.FLOAT;
-    	Byte resultType = in.getResultType();
-        switch(resultType) {
-        case DataType.BAG : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;
-        }
-        
-        case DataType.TUPLE : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;
-        }
-
-        case DataType.MAP : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;        	
-        }
-                
-        case DataType.BYTEARRAY : {
-        	DataByteArray dba = null;
-        	Result res = in.getNext(dba);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		//res.result = new Float(Float.valueOf((((DataByteArray)res.result).toString())));
-        	    if(castNotNeeded) {
-                    // we examined the data once before and
-                    // determined that the input is the same
-                    // type as the type we are casting to
-                    // so just send the input out as output
-                    return res;
-                }
-        		try {
-                    dba = (DataByteArray) res.result;
-                } catch (ClassCastException e) {
-                    // check if the type of res.result is
-                    // same as the type we are trying to cast to
-                    if(DataType.findType(res.result) == castToType) {
-                        // remember this for future calls
-                        castNotNeeded  = true;
-                        // just return the output
-                        return res;
-                    } else {
-                        // the input is a differen type
-                        // rethrow the exception
-                        throw e;
-                    }
-
-                }
-        		try {
-                    if(null != load) {
-					    res.result = load.bytesToFloat(dba.get());
-                    } else {
-					    String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to float.";
-					    log.error(msg);
-                        throw new ExecException(msg);
-                    }
+			}
+			return res;
+		}
+
+		case DataType.BOOLEAN: {
+			Boolean b = null;
+			Result res = in.getNext(b);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				if (((Boolean) res.result) == true)
+					res.result = new Double(1);
+				else
+					res.result = new Double(0);
+			}
+			return res;
+		}
+		case DataType.INTEGER: {
+			Integer dummyI = null;
+			Result res = in.getNext(dummyI);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Double(((Integer) res.result).doubleValue());
+			}
+			return res;
+		}
+
+		case DataType.DOUBLE: {
+
+			Result res = in.getNext(d);
+
+			return res;
+		}
+
+		case DataType.LONG: {
+			Long l = null;
+			Result res = in.getNext(l);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Double(((Long) res.result).doubleValue());
+			}
+			return res;
+		}
+
+		case DataType.FLOAT: {
+			Float f = null;
+			Result res = in.getNext(f);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Double(((Float) res.result).doubleValue());
+			}
+			return res;
+		}
+
+		case DataType.CHARARRAY: {
+			String str = null;
+			Result res = in.getNext(str);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Double(Double.valueOf((String) res.result));
+			}
+			return res;
+		}
+
+		}
+
+		Result res = new Result();
+		res.returnStatus = POStatus.STATUS_ERR;
+		return res;
+	}
+
+	@Override
+	public Result getNext(Float f) throws ExecException {
+		PhysicalOperator in = inputs.get(0);
+		Byte castToType = DataType.FLOAT;
+		Byte resultType = in.getResultType();
+		switch (resultType) {
+		case DataType.BAG: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.TUPLE: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.MAP: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.BYTEARRAY: {
+			DataByteArray dba = null;
+			Result res = in.getNext(dba);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				// res.result = new
+				// Float(Float.valueOf((((DataByteArray)res.result).toString())));
+				if (castNotNeeded) {
+					// we examined the data once before and
+					// determined that the input is the same
+					// type as the type we are casting to
+					// so just send the input out as output
+					return res;
+				}
+				try {
+					dba = (DataByteArray) res.result;
+				} catch (ClassCastException e) {
+					// check if the type of res.result is
+					// same as the type we are trying to cast to
+					if (DataType.findType(res.result) == castToType) {
+						// remember this for future calls
+						castNotNeeded = true;
+						// just return the output
+						return res;
+					} else {
+						// the input is a differen type
+						// rethrow the exception
+						throw e;
+					}
+
+				}
+				try {
+					if (null != load) {
+						res.result = load.bytesToFloat(dba.get());
+					} else {
+						String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to float.";
+						log.error(msg);
+						throw new ExecException(msg);
+					}
+				} catch (ExecException ee) {
+					throw ee;
 				} catch (IOException e) {
 					log.error("Error while casting from ByteArray to Float");
 				}
-        	}
-        	return res;
-        }
-        
-        case DataType.BOOLEAN : {
-        	Boolean b = null;
-        	Result res = in.getNext(b);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		if (((Boolean)res.result) == true) res.result = new Float(1);
-                else res.result = new Float(0);
-        	}
-        	return res;
-        }
-        case DataType.INTEGER : {
-        	Integer dummyI = null;
-        	Result res = in.getNext(dummyI);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Float(((Integer)res.result).floatValue());
-        	}
-        	return res;
-        }
-
-        case DataType.DOUBLE : {
-        	Double d = null;
-        	Result res = in.getNext(d);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		//res.result = DataType.toInteger(res.result);
-        		res.result = new Float(((Double)res.result).floatValue());
-        	}
-        	return res;
-        }
-
-        case DataType.LONG : {
-        	
-        	Long l = null;
-        	Result res = in.getNext(l);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Float(((Long)res.result).floatValue());
-        	}
-        	return res;
-        }
-        
-        case DataType.FLOAT : {
-        
-        	Result res = in.getNext(f);
-        	
-        	return res;
-        }
-        
-        case DataType.CHARARRAY : {
-        	String str = null;
-        	Result res = in.getNext(str);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new Float(Float.valueOf((String)res.result));
-        	}
-        	return res;
-        }
-        
-        }
-        
-        Result res = new Result();
-        res.returnStatus = POStatus.STATUS_ERR;
-        return res;
-    }
-    
-    @Override
-    public Result getNext(String str) throws ExecException {
-    	PhysicalOperator in = inputs.get(0);
-        Byte castToType = DataType.CHARARRAY;
-    	Byte resultType = in.getResultType();
-        switch(resultType) {
-        case DataType.BAG : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;
-        }
-        
-        case DataType.TUPLE : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;
-        }
-
-        case DataType.MAP : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res;        	
-        }
-                
-        case DataType.BYTEARRAY : {
-        	DataByteArray dba = null;
-        	Result res = in.getNext(dba);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		//res.result = new String(((DataByteArray)res.result).toString());
-        	    if(castNotNeeded) {
-                    // we examined the data once before and
-                    // determined that the input is the same
-                    // type as the type we are casting to
-                    // so just send the input out as output
-                    return res;
-                }
-        		try {
-                    dba = (DataByteArray) res.result;
-                } catch (ClassCastException e) {
-                    // check if the type of res.result is
-                    // same as the type we are trying to cast to
-                    if(DataType.findType(res.result) == castToType) {
-                        // remember this for future calls
-                        castNotNeeded  = true;
-                        // just return the output
-                        return res;
-                    } else {
-                        // the input is a differen type
-                        // rethrow the exception
-                        throw e;
-                    }
-
-                }
-        		try {
-                    if(null != load) {
-					    res.result = load.bytesToCharArray(dba.get());
-                    } else {
-					    String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to string.";
-					    log.error(msg);
-                        throw new ExecException(msg);
-                    }
+			}
+			return res;
+		}
+
+		case DataType.BOOLEAN: {
+			Boolean b = null;
+			Result res = in.getNext(b);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				if (((Boolean) res.result) == true)
+					res.result = new Float(1);
+				else
+					res.result = new Float(0);
+			}
+			return res;
+		}
+		case DataType.INTEGER: {
+			Integer dummyI = null;
+			Result res = in.getNext(dummyI);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Float(((Integer) res.result).floatValue());
+			}
+			return res;
+		}
+
+		case DataType.DOUBLE: {
+			Double d = null;
+			Result res = in.getNext(d);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				// res.result = DataType.toInteger(res.result);
+				res.result = new Float(((Double) res.result).floatValue());
+			}
+			return res;
+		}
+
+		case DataType.LONG: {
+
+			Long l = null;
+			Result res = in.getNext(l);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Float(((Long) res.result).floatValue());
+			}
+			return res;
+		}
+
+		case DataType.FLOAT: {
+
+			Result res = in.getNext(f);
+
+			return res;
+		}
+
+		case DataType.CHARARRAY: {
+			String str = null;
+			Result res = in.getNext(str);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new Float(Float.valueOf((String) res.result));
+			}
+			return res;
+		}
+
+		}
+
+		Result res = new Result();
+		res.returnStatus = POStatus.STATUS_ERR;
+		return res;
+	}
+
+	@Override
+	public Result getNext(String str) throws ExecException {
+		PhysicalOperator in = inputs.get(0);
+		Byte castToType = DataType.CHARARRAY;
+		Byte resultType = in.getResultType();
+		switch (resultType) {
+		case DataType.BAG: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.TUPLE: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.MAP: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		case DataType.BYTEARRAY: {
+			DataByteArray dba = null;
+			Result res = in.getNext(dba);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				// res.result = new
+				// String(((DataByteArray)res.result).toString());
+				if (castNotNeeded) {
+					// we examined the data once before and
+					// determined that the input is the same
+					// type as the type we are casting to
+					// so just send the input out as output
+					return res;
+				}
+				try {
+					dba = (DataByteArray) res.result;
+				} catch (ClassCastException e) {
+					// check if the type of res.result is
+					// same as the type we are trying to cast to
+					if (DataType.findType(res.result) == castToType) {
+						// remember this for future calls
+						castNotNeeded = true;
+						// just return the output
+						return res;
+					} else {
+						// the input is a differen type
+						// rethrow the exception
+						throw e;
+					}
+
+				}
+				try {
+					if (null != load) {
+						res.result = load.bytesToCharArray(dba.get());
+					} else {
+						String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to string.";
+						log.error(msg);
+						throw new ExecException(msg);
+					}
+				} catch (ExecException ee) {
+					throw ee;
 				} catch (IOException e) {
 					log.error("Error while casting from ByteArray to CharArray");
 				}
-        	}
-        	return res;
-        }
-        
-        case DataType.BOOLEAN : {
-        	Boolean b = null;
-        	Result res = in.getNext(b);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		if (((Boolean)res.result) == true) res.result = new String("1");
-                else res.result = new String("1");
-        	}
-        	return res;
-        }
-        case DataType.INTEGER : {
-        	Integer dummyI = null;
-        	Result res = in.getNext(dummyI);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new String(((Integer)res.result).toString());
-        	}
-        	return res;
-        }
-
-        case DataType.DOUBLE : {
-        	Double d = null;
-        	Result res = in.getNext(d);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		//res.result = DataType.toInteger(res.result);
-        		res.result = new String(((Double)res.result).toString());
-        	}
-        	return res;
-        }
-
-        case DataType.LONG : {
-        	
-        	Long l = null;
-        	Result res = in.getNext(l);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new String(((Long)res.result).toString());
-        	}
-        	return res;
-        }
-        
-        case DataType.FLOAT : {
-        	Float f = null;
-        	Result res = in.getNext(f);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		res.result = new String(((Float)res.result).toString());
-        	}
-        	return res;
-        }
-        
-        case DataType.CHARARRAY : {
-        	
-        	Result res = in.getNext(str);
-        	
-        	return res;
-        }
-        
-        }
-        
-        Result res = new Result();
-        res.returnStatus = POStatus.STATUS_ERR;
-        return res;
-    }
-    
-    @Override
-    public Result getNext(Tuple t) throws ExecException {
-    	PhysicalOperator in = inputs.get(0);
-        Byte castToType = DataType.TUPLE;
-    	Byte resultType = in.getResultType();
-        switch(resultType) {
-        
-        case DataType.TUPLE : {
-        	Result res = in.getNext(t);
-        	return res;
-        }
-        
-        case DataType.BYTEARRAY : {
-        	DataByteArray dba = null;
-        	Result res = in.getNext(dba);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		//res.result = new String(((DataByteArray)res.result).toString());
-        	    if(castNotNeeded) {
-                    // we examined the data once before and
-                    // determined that the input is the same
-                    // type as the type we are casting to
-                    // so just send the input out as output
-                    return res;
-                }
-        		try {
-                    dba = (DataByteArray) res.result;
-                } catch (ClassCastException e) {
-                    // check if the type of res.result is
-                    // same as the type we are trying to cast to
-                    if(DataType.findType(res.result) == castToType) {
-                        // remember this for future calls
-                        castNotNeeded  = true;
-                        // just return the output
-                        return res;
-                    } else {
-                        // the input is a differen type
-                        // rethrow the exception
-                        throw e;
-                    }
-
-                }
-        		try {
-                    if(null != load) {
-					    res.result = load.bytesToTuple(dba.get());
-                    } else {
-					    String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to tuple.";
-					    log.error(msg);
-                        throw new ExecException(msg);
-                    }
+			}
+			return res;
+		}
+
+		case DataType.BOOLEAN: {
+			Boolean b = null;
+			Result res = in.getNext(b);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				if (((Boolean) res.result) == true)
+					res.result = new String("1");
+				else
+					res.result = new String("1");
+			}
+			return res;
+		}
+		case DataType.INTEGER: {
+			Integer dummyI = null;
+			Result res = in.getNext(dummyI);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new String(((Integer) res.result).toString());
+			}
+			return res;
+		}
+
+		case DataType.DOUBLE: {
+			Double d = null;
+			Result res = in.getNext(d);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				// res.result = DataType.toInteger(res.result);
+				res.result = new String(((Double) res.result).toString());
+			}
+			return res;
+		}
+
+		case DataType.LONG: {
+
+			Long l = null;
+			Result res = in.getNext(l);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new String(((Long) res.result).toString());
+			}
+			return res;
+		}
+
+		case DataType.FLOAT: {
+			Float f = null;
+			Result res = in.getNext(f);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				res.result = new String(((Float) res.result).toString());
+			}
+			return res;
+		}
+
+		case DataType.CHARARRAY: {
+
+			Result res = in.getNext(str);
+
+			return res;
+		}
+
+		}
+
+		Result res = new Result();
+		res.returnStatus = POStatus.STATUS_ERR;
+		return res;
+	}
+
+	@Override
+	public Result getNext(Tuple t) throws ExecException {
+		PhysicalOperator in = inputs.get(0);
+		Byte castToType = DataType.TUPLE;
+		Byte resultType = in.getResultType();
+		switch (resultType) {
+
+		case DataType.TUPLE: {
+			Result res = in.getNext(t);
+			return res;
+		}
+
+		case DataType.BYTEARRAY: {
+			DataByteArray dba = null;
+			Result res = in.getNext(dba);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				// res.result = new
+				// String(((DataByteArray)res.result).toString());
+				if (castNotNeeded) {
+					// we examined the data once before and
+					// determined that the input is the same
+					// type as the type we are casting to
+					// so just send the input out as output
+					return res;
+				}
+				try {
+					dba = (DataByteArray) res.result;
+				} catch (ClassCastException e) {
+					// check if the type of res.result is
+					// same as the type we are trying to cast to
+					if (DataType.findType(res.result) == castToType) {
+						// remember this for future calls
+						castNotNeeded = true;
+						// just return the output
+						return res;
+					} else {
+						// the input is a differen type
+						// rethrow the exception
+						throw e;
+					}
+
+				}
+				try {
+					if (null != load) {
+						res.result = load.bytesToTuple(dba.get());
+					} else {
+						String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to tuple.";
+						log.error(msg);
+						throw new ExecException(msg);
+					}
+				} catch (ExecException ee) {
+					throw ee;
 				} catch (IOException e) {
 					log.error("Error while casting from ByteArray to Tuple");
 				}
-        	}
-        	return res;
-        }
-
-        case DataType.BAG :
-        
-        case DataType.MAP : 
-        	
-        case DataType.INTEGER :
-        	
-        case DataType.DOUBLE :
-        	
-        case DataType.LONG :
-        	
-        case DataType.FLOAT :
-        	
-        case DataType.CHARARRAY :
-        
-        case DataType.BOOLEAN : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res; 
-        }
-        
-        
-        }
-        
-        Result res = new Result();
-        res.returnStatus = POStatus.STATUS_ERR;
-        return res;
-    }
-    
-    @Override
-    public Result getNext(DataBag bag) throws ExecException {
-    	PhysicalOperator in = inputs.get(0);
-        Byte castToType = DataType.BAG;
-    	Byte resultType = in.getResultType();
-        switch(resultType) {
-        
-        case DataType.BAG : {
-        	Result res = in.getNext(bag);
-        	return res;
-        }
-        
-        case DataType.BYTEARRAY : {
-        	DataByteArray dba = null;
-        	Result res = in.getNext(dba);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		//res.result = new String(((DataByteArray)res.result).toString());
-        	    if(castNotNeeded) {
-                    // we examined the data once before and
-                    // determined that the input is the same
-                    // type as the type we are casting to
-                    // so just send the input out as output
-                    return res;
-                }
-        		try {
-                    dba = (DataByteArray) res.result;
-                } catch (ClassCastException e) {
-                    // check if the type of res.result is
-                    // same as the type we are trying to cast to
-                    if(DataType.findType(res.result) == castToType) {
-                        // remember this for future calls
-                        castNotNeeded  = true;
-                        // just return the output
-                        return res;
-                    } else {
-                        // the input is a differen type
-                        // rethrow the exception
-                        throw e;
-                    }
-
-                }
-        		try {
-                    if(null != load) {
-					    res.result = load.bytesToBag(dba.get());
-                    } else {
-					    String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to bag.";
-					    log.error(msg);
-                        throw new ExecException(msg);
-                    }
+			}
+			return res;
+		}
+
+		case DataType.BAG:
+
+		case DataType.MAP:
+
+		case DataType.INTEGER:
+
+		case DataType.DOUBLE:
+
+		case DataType.LONG:
+
+		case DataType.FLOAT:
+
+		case DataType.CHARARRAY:
+
+		case DataType.BOOLEAN: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		}
+
+		Result res = new Result();
+		res.returnStatus = POStatus.STATUS_ERR;
+		return res;
+	}
+
+	@Override
+	public Result getNext(DataBag bag) throws ExecException {
+		PhysicalOperator in = inputs.get(0);
+		Byte castToType = DataType.BAG;
+		Byte resultType = in.getResultType();
+		switch (resultType) {
+
+		case DataType.BAG: {
+			Result res = in.getNext(bag);
+			return res;
+		}
+
+		case DataType.BYTEARRAY: {
+			DataByteArray dba = null;
+			Result res = in.getNext(dba);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				// res.result = new
+				// String(((DataByteArray)res.result).toString());
+				if (castNotNeeded) {
+					// we examined the data once before and
+					// determined that the input is the same
+					// type as the type we are casting to
+					// so just send the input out as output
+					return res;
+				}
+				try {
+					dba = (DataByteArray) res.result;
+				} catch (ClassCastException e) {
+					// check if the type of res.result is
+					// same as the type we are trying to cast to
+					if (DataType.findType(res.result) == castToType) {
+						// remember this for future calls
+						castNotNeeded = true;
+						// just return the output
+						return res;
+					} else {
+						// the input is a differen type
+						// rethrow the exception
+						throw e;
+					}
+
+				}
+				try {
+					if (null != load) {
+						res.result = load.bytesToBag(dba.get());
+					} else {
+						String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to bag.";
+						log.error(msg);
+						throw new ExecException(msg);
+					}
+				} catch (ExecException ee) {
+					throw ee;
 				} catch (IOException e) {
 					log.error("Error while casting from ByteArray to DataBag");
 				}
-        	}
-        	return res;
-        }
-
-        case DataType.TUPLE :
-        
-        case DataType.MAP : 
-        	
-        case DataType.INTEGER :
-        	
-        case DataType.DOUBLE :
-        	
-        case DataType.LONG :
-        	
-        case DataType.FLOAT :
-        	
-        case DataType.CHARARRAY :
-        
-        case DataType.BOOLEAN : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res; 
-        }
-        
-        
-        }
-        
-        Result res = new Result();
-        res.returnStatus = POStatus.STATUS_ERR;
-        return res;
-    }
-    
-    @Override
-    public Result getNext(Map m) throws ExecException {
-    	PhysicalOperator in = inputs.get(0);
-        Byte castToType = DataType.MAP;
-    	Byte resultType = in.getResultType();
-        switch(resultType) {
-        
-        case DataType.MAP : {
-        	Result res = in.getNext(m);
-        	return res;
-        }
-        
-        case DataType.BYTEARRAY : {
-        	DataByteArray dba = null;
-        	Result res = in.getNext(dba);
-        	if(res.returnStatus == POStatus.STATUS_OK && res.result != null) {
-        		//res.result = new String(((DataByteArray)res.result).toString());
-        	    if(castNotNeeded) {
-                    // we examined the data once before and
-                    // determined that the input is the same
-                    // type as the type we are casting to
-                    // so just send the input out as output
-                    return res;
-                }
-        		try {
-                    dba = (DataByteArray) res.result;
-                } catch (ClassCastException e) {
-                    // check if the type of res.result is
-                    // same as the type we are trying to cast to
-                    if(DataType.findType(res.result) == castToType) {
-                        // remember this for future calls
-                        castNotNeeded  = true;
-                        // just return the output
-                        return res;
-                    } else {
-                        // the input is a differen type
-                        // rethrow the exception
-                        throw e;
-                    }
-
-                }
-        		try {
-                    if(null != load) {
-					    res.result = load.bytesToMap(dba.get());
-                    } else {
-					    String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to map.";
-					    log.error(msg);
-                        throw new ExecException(msg);
-                    }
+			}
+			return res;
+		}
+
+		case DataType.TUPLE:
+
+		case DataType.MAP:
+
+		case DataType.INTEGER:
+
+		case DataType.DOUBLE:
+
+		case DataType.LONG:
+
+		case DataType.FLOAT:
+
+		case DataType.CHARARRAY:
+
+		case DataType.BOOLEAN: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		}
+
+		Result res = new Result();
+		res.returnStatus = POStatus.STATUS_ERR;
+		return res;
+	}
+
+	@Override
+	public Result getNext(Map m) throws ExecException {
+		PhysicalOperator in = inputs.get(0);
+		Byte castToType = DataType.MAP;
+		Byte resultType = in.getResultType();
+		switch (resultType) {
+
+		case DataType.MAP: {
+			Result res = in.getNext(m);
+			return res;
+		}
+
+		case DataType.BYTEARRAY: {
+			DataByteArray dba = null;
+			Result res = in.getNext(dba);
+			if (res.returnStatus == POStatus.STATUS_OK && res.result != null) {
+				// res.result = new
+				// String(((DataByteArray)res.result).toString());
+				if (castNotNeeded) {
+					// we examined the data once before and
+					// determined that the input is the same
+					// type as the type we are casting to
+					// so just send the input out as output
+					return res;
+				}
+				try {
+					dba = (DataByteArray) res.result;
+				} catch (ClassCastException e) {
+					// check if the type of res.result is
+					// same as the type we are trying to cast to
+					if (DataType.findType(res.result) == castToType) {
+						// remember this for future calls
+						castNotNeeded = true;
+						// just return the output
+						return res;
+					} else {
+						// the input is a differen type
+						// rethrow the exception
+						throw e;
+					}
+
+				}
+				try {
+					if (null != load) {
+						res.result = load.bytesToMap(dba.get());
+					} else {
+						String msg = "Received a bytearray from the UDF. Cannot determine how to convert the bytearray to map.";
+						log.error(msg);
+						throw new ExecException(msg);
+					}
+        		} catch (ExecException ee) {
+        			throw ee;
 				} catch (IOException e) {
 					log.error("Error while casting from ByteArray to Map");
 				}
-        	}
-        	return res;
-        }
-
-        case DataType.TUPLE :
-        
-        case DataType.BAG : 
-        	
-        case DataType.INTEGER :
-        	
-        case DataType.DOUBLE :
-        	
-        case DataType.LONG :
-        	
-        case DataType.FLOAT :
-        	
-        case DataType.CHARARRAY :
-        
-        case DataType.BOOLEAN : {
-        	Result res = new Result();
-        	res.returnStatus = POStatus.STATUS_ERR;
-        	return res; 
-        }
-        
-        
-        }
-        
-        Result res = new Result();
-        res.returnStatus = POStatus.STATUS_ERR;
-        return res;
-    }
-    
-    private void readObject(ObjectInputStream is) throws IOException, ClassNotFoundException{
-        is.defaultReadObject();
-        instantiateFunc();
-    }
-
-    @Override
-    public POCast clone() throws CloneNotSupportedException {
-        POCast clone = new POCast(new OperatorKey(mKey.scope, 
-            NodeIdGenerator.getGenerator().getNextNodeId(mKey.scope)));
-        clone.cloneHelper(this);
-        clone.loadFSpec = loadFSpec;
-        clone.instantiateFunc();
-        return clone;
-    }
-    
+			}
+			return res;
+		}
+
+		case DataType.TUPLE:
+
+		case DataType.BAG:
+
+		case DataType.INTEGER:
+
+		case DataType.DOUBLE:
+
+		case DataType.LONG:
+
+		case DataType.FLOAT:
+
+		case DataType.CHARARRAY:
+
+		case DataType.BOOLEAN: {
+			Result res = new Result();
+			res.returnStatus = POStatus.STATUS_ERR;
+			return res;
+		}
+
+		}
+
+		Result res = new Result();
+		res.returnStatus = POStatus.STATUS_ERR;
+		return res;
+	}
+
+	private void readObject(ObjectInputStream is) throws IOException,
+			ClassNotFoundException {
+		is.defaultReadObject();
+		instantiateFunc();
+	}
+
+	@Override
+	public POCast clone() throws CloneNotSupportedException {
+		POCast clone = new POCast(new OperatorKey(mKey.scope, NodeIdGenerator
+				.getGenerator().getNextNodeId(mKey.scope)));
+		clone.cloneHelper(this);
+		clone.loadFSpec = loadFSpec;
+		clone.instantiateFunc();
+		return clone;
+	}
 
 }

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=737239&r1=737238&r2=737239&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 Jan 23 23:07:30 2009
@@ -24,6 +24,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.DataType;
 import org.apache.pig.data.Tuple;
@@ -38,6 +39,7 @@
 import org.apache.pig.impl.plan.OperatorKey;
 import org.apache.pig.impl.plan.NodeIdGenerator;
 import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.impl.plan.PlanException;
 
 /**
  * The local rearrange operator is a part of the co-group
@@ -463,7 +465,7 @@
      * @param plans2
      * @throws ExecException 
      */
-    public void setPlansFromCombiner(List<PhysicalPlan> plans) throws ExecException {
+    public void setPlansFromCombiner(List<PhysicalPlan> plans) throws PlanException {
         this.plans = plans;
         leafOps.clear();
         mProjectedColsMap.clear();
@@ -485,9 +487,9 @@
                 if(leaf instanceof POProject) {
                     POProject project = (POProject) leaf;
                     if(project.isStar()) {
-                        log.error("Unexpected data during optimization");
-                        throw new ExecException("Unexpected data during optimization (Local rearrange" +
-                                " in combiner has a project *" );
+                        int errCode = 2021;
+                        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);
                     }

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=737239&r1=737238&r2=737239&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 Jan 23 23:07:30 2009
@@ -29,6 +29,7 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStream;
@@ -80,7 +81,9 @@
             try {
                 FileUtil.chmod(executable.toString(), "a+x");
             } catch (InterruptedException ie) {
-                throw new ExecException(ie);
+                int errCode = 6013;
+                String msg = "Unable to chmod " + executable + " . Thread interrupted.";
+                throw new ExecException(msg, errCode, PigException.REMOTE_ENVIRONMENT, ie);
             }
         }
         

Modified: hadoop/pig/trunk/src/org/apache/pig/data/DataType.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/DataType.java?rev=737239&r1=737238&r2=737239&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/data/DataType.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/data/DataType.java Fri Jan 23 23:07:30 2009
@@ -32,10 +32,11 @@
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparable;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.logicalLayer.schema.SchemaMergeException;
-import org.apache.pig.impl.logicalLayer.FrontendException;
 
 /**
  * A class of static final values used to encode data type and a number of
@@ -378,43 +379,57 @@
      * @throws ExecException if the type can't be forced to an Integer.
      */
     public static Integer toInteger(Object o) throws ExecException {
-        switch (findType(o)) {
-        case BOOLEAN:
-            if (((Boolean)o) == true) return new Integer(1);
-            else return new Integer(0);
-
-        case BYTE:
-            return new Integer(((Byte)o).intValue());
-
-        case INTEGER:
-            return (Integer)o;
-
-        case LONG:
-            return new Integer(((Long)o).intValue());
-
-        case FLOAT:
-            return new Integer(((Float)o).intValue());
-
-        case DOUBLE:
-            return new Integer(((Double)o).intValue());
-
-        case BYTEARRAY:
-            return Integer.valueOf(((DataByteArray)o).toString());
-
-        case CHARARRAY:
-            return Integer.valueOf((String)o);
-
-        case NULL:
-            return null;
-
-        case MAP:
-        case TUPLE:
-        case BAG:
-        case UNKNOWN:
-        default:
-            throw new ExecException("Cannot convert a " + findTypeName(o) +
-                " to an Integer");
-        }
+        try {
+			switch (findType(o)) {
+			case BOOLEAN:
+			    if (((Boolean)o) == true) return new Integer(1);
+			    else return new Integer(0);
+
+			case BYTE:
+			    return new Integer(((Byte)o).intValue());
+
+			case INTEGER:
+			    return (Integer)o;
+
+			case LONG:
+			    return new Integer(((Long)o).intValue());
+
+			case FLOAT:
+			    return new Integer(((Float)o).intValue());
+
+			case DOUBLE:
+			    return new Integer(((Double)o).intValue());
+
+			case BYTEARRAY:
+			    return Integer.valueOf(((DataByteArray)o).toString());
+
+			case CHARARRAY:
+			    return Integer.valueOf((String)o);
+
+			case NULL:
+			    return null;
+
+			case MAP:
+			case TUPLE:
+			case BAG:
+			case UNKNOWN:
+			default:
+			    int errCode = 1071;
+			    String msg = "Cannot convert a " + findTypeName(o) +
+			    " to an Integer";
+			    throw new ExecException(msg, errCode, PigException.INPUT);
+			}
+		} catch (ExecException ee) {
+			throw ee;
+		} catch (NumberFormatException nfe) {
+			int errCode = 1074;
+			String msg = "Problem with formatting. Could not convert " + o + " to Integer.";
+			throw new ExecException(msg, errCode, PigException.INPUT, nfe);
+		} catch (Exception e) {
+			int errCode = 2054;
+			String msg = "Internal error. Could not convert " + o + " to Integer.";
+			throw new ExecException(msg, errCode, PigException.BUG);
+		}
     }
 
     /**
@@ -428,43 +443,58 @@
      * @throws ExecException if the type can't be forced to a Long.
      */
     public static Long toLong(Object o) throws ExecException {
-        switch (findType(o)) {
-        case BOOLEAN:
-            if (((Boolean)o) == true) return new Long(1);
-            else return new Long(0);
+        try {
+			switch (findType(o)) {
+			case BOOLEAN:
+			    if (((Boolean)o) == true) return new Long(1);
+			    else return new Long(0);
+
+			case BYTE:
+			    return new Long(((Byte)o).longValue());
+
+			case INTEGER:
+			    return new Long(((Integer)o).longValue());
+
+			case LONG:
+			    return (Long)o;
+
+			case FLOAT:
+			    return new Long(((Float)o).longValue());
+
+			case DOUBLE:
+			    return new Long(((Double)o).longValue());
+
+			case BYTEARRAY:
+			    return Long.valueOf(((DataByteArray)o).toString());
+
+			case CHARARRAY:
+			    return Long.valueOf((String)o);
+
+			case NULL:
+			    return null;
+
+			case MAP:
+			case TUPLE:
+			case BAG:
+			case UNKNOWN:
+			default:
+			    int errCode = 1071;
+			    String msg = "Cannot convert a " + findTypeName(o) +
+			    " to a Long";
+			    throw new ExecException(msg, errCode, PigException.INPUT);
+			}
+		} catch (ExecException ee) {
+			throw ee;
+		} catch (NumberFormatException nfe) {
+			int errCode = 1074;
+			String msg = "Problem with formatting. Could not convert " + o + " to Long.";
+			throw new ExecException(msg, errCode, PigException.INPUT, nfe);
+		} catch (Exception e) {
+			int errCode = 2054;
+			String msg = "Internal error. Could not convert " + o + " to Long.";
+			throw new ExecException(msg, errCode, PigException.BUG);
+		}
 
-        case BYTE:
-            return new Long(((Byte)o).longValue());
-
-        case INTEGER:
-            return new Long(((Integer)o).longValue());
-
-        case LONG:
-            return (Long)o;
-
-        case FLOAT:
-            return new Long(((Float)o).longValue());
-
-        case DOUBLE:
-            return new Long(((Double)o).longValue());
-
-        case BYTEARRAY:
-            return Long.valueOf(((DataByteArray)o).toString());
-
-        case CHARARRAY:
-            return Long.valueOf((String)o);
-
-        case NULL:
-            return null;
-
-        case MAP:
-        case TUPLE:
-        case BAG:
-        case UNKNOWN:
-        default:
-            throw new ExecException("Cannot convert a " + findTypeName(o) +
-                " to a Long");
-        }
     }
 
     /**
@@ -478,38 +508,52 @@
      * @throws ExecException if the type can't be forced to a Float.
      */
     public static Float toFloat(Object o) throws ExecException {
-        switch (findType(o)) {
-        case INTEGER:
-            return new Float(((Integer)o).floatValue());
-
-        case LONG:
-            return new Float(((Long)o).floatValue());
-
-        case FLOAT:
-            return (Float)o;
-
-        case DOUBLE:
-            return new Float(((Double)o).floatValue());
-
-        case BYTEARRAY:
-            return Float.valueOf(((DataByteArray)o).toString());
-
-        case CHARARRAY:
-            return Float.valueOf((String)o);
-
-        case NULL:
-            return null;
-
-        case BOOLEAN:
-        case BYTE:
-        case MAP:
-        case TUPLE:
-        case BAG:
-        case UNKNOWN:
-        default:
-            throw new ExecException("Cannot convert a " + findTypeName(o) +
-                " to a Float");
-        }
+        try {
+			switch (findType(o)) {
+			case INTEGER:
+			    return new Float(((Integer)o).floatValue());
+
+			case LONG:
+			    return new Float(((Long)o).floatValue());
+
+			case FLOAT:
+			    return (Float)o;
+
+			case DOUBLE:
+			    return new Float(((Double)o).floatValue());
+
+			case BYTEARRAY:
+			    return Float.valueOf(((DataByteArray)o).toString());
+
+			case CHARARRAY:
+			    return Float.valueOf((String)o);
+
+			case NULL:
+			    return null;
+
+			case BOOLEAN:
+			case BYTE:
+			case MAP:
+			case TUPLE:
+			case BAG:
+			case UNKNOWN:
+			default:
+			    int errCode = 1071;
+			    String msg = "Cannot convert a " + findTypeName(o) +
+			    " to a Float";
+			    throw new ExecException(msg, errCode, PigException.INPUT);
+			}
+		} catch (ExecException ee) {
+			throw ee;
+		} catch (NumberFormatException nfe) {
+			int errCode = 1074;
+			String msg = "Problem with formatting. Could not convert " + o + " to Float.";
+			throw new ExecException(msg, errCode, PigException.INPUT, nfe);
+		} catch (Exception e) {
+			int errCode = 2054;
+			String msg = "Internal error. Could not convert " + o + " to Float.";
+			throw new ExecException(msg, errCode, PigException.BUG);
+		}
     }
 
     /**
@@ -523,38 +567,52 @@
      * @throws ExecException if the type can't be forced to a Double.
      */
     public static Double toDouble(Object o) throws ExecException {
-        switch (findType(o)) {
-        case INTEGER:
-            return new Double(((Integer)o).doubleValue());
-
-        case LONG:
-            return new Double(((Long)o).doubleValue());
-
-        case FLOAT:
-            return new Double(((Float)o).doubleValue());
-
-        case DOUBLE:
-            return (Double)o;
-
-        case BYTEARRAY:
-            return Double.valueOf(((DataByteArray)o).toString());
-
-        case CHARARRAY:
-            return Double.valueOf((String)o);
-
-        case NULL:
-            return null;
-
-        case BOOLEAN:
-        case BYTE:
-        case MAP:
-        case TUPLE:
-        case BAG:
-        case UNKNOWN:
-        default:
-            throw new ExecException("Cannot convert a " + findTypeName(o) +
-                " to a Double");
-        }
+        try {
+			switch (findType(o)) {
+			case INTEGER:
+			    return new Double(((Integer)o).doubleValue());
+
+			case LONG:
+			    return new Double(((Long)o).doubleValue());
+
+			case FLOAT:
+			    return new Double(((Float)o).doubleValue());
+
+			case DOUBLE:
+			    return (Double)o;
+
+			case BYTEARRAY:
+			    return Double.valueOf(((DataByteArray)o).toString());
+
+			case CHARARRAY:
+			    return Double.valueOf((String)o);
+
+			case NULL:
+			    return null;
+
+			case BOOLEAN:
+			case BYTE:
+			case MAP:
+			case TUPLE:
+			case BAG:
+			case UNKNOWN:
+			default:
+			    int errCode = 1071;
+			    String msg = "Cannot convert a " + findTypeName(o) +
+			    " to a Double";
+			    throw new ExecException(msg, errCode, PigException.INPUT);
+			}
+		} catch (ExecException ee) {
+			throw ee;
+		} catch (NumberFormatException nfe) {
+			int errCode = 1074;
+			String msg = "Problem with formatting. Could not convert " + o + " to Double.";
+			throw new ExecException(msg, errCode, PigException.INPUT, nfe);
+		} catch (Exception e) {
+			int errCode = 2054;
+			String msg = "Internal error. Could not convert " + o + " to Double.";
+			throw new ExecException(msg, errCode, PigException.BUG);
+		}
     }
 
     /**
@@ -569,10 +627,18 @@
         if (o == null) return null;
 
         if (o instanceof Map) {
-            return (Map<Object, Object>)o;
+            try {
+				return (Map<Object, Object>)o;
+    		} catch (Exception e) {
+    			int errCode = 2054;
+    			String msg = "Internal error. Could not convert " + o + " to Map.";
+    			throw new ExecException(msg, errCode, PigException.BUG);
+    		}
         } else {
-            throw new ExecException("Cannot convert a " + findTypeName(o) +
-                " to a Map");
+            int errCode = 1071;
+            String msg = "Cannot convert a " + findTypeName(o) +
+            " to a Map";
+            throw new ExecException(msg, errCode, PigException.INPUT);
         }
     }
 
@@ -588,10 +654,18 @@
         if (o == null) return null;
 
         if (o instanceof Tuple) {
-            return (Tuple)o;
+            try {
+				return (Tuple)o;
+    		} catch (Exception e) {
+    			int errCode = 2054;
+    			String msg = "Internal error. Could not convert " + o + " to Tuple.";
+    			throw new ExecException(msg, errCode, PigException.BUG);
+    		}
         } else {
-            throw new ExecException("Cannot convert a " + findTypeName(o) +
-                " to a Tuple");
+            int errCode = 1071;
+            String msg = "Cannot convert a " + findTypeName(o) +
+            " to a Tuple";
+            throw new ExecException(msg, errCode, PigException.INPUT);
         }
     }
 
@@ -607,10 +681,18 @@
         if (o == null) return null;
 
         if (o instanceof DataBag) {
-            return (DataBag)o;
+            try {
+				return (DataBag)o;
+    		} catch (Exception e) {
+    			int errCode = 2054;
+    			String msg = "Internal error. Could not convert " + o + " to Bag.";
+    			throw new ExecException(msg, errCode, PigException.BUG);
+    		}
         } else {
-            throw new ExecException("Cannot convert a " + findTypeName(o) +
-                " to a DataBag");
+            int errCode = 1071;
+            String msg = "Cannot convert a " + findTypeName(o) +
+            " to a DataBag";
+            throw new ExecException(msg, errCode, PigException.INPUT);
         }
     }
 
@@ -802,7 +884,9 @@
                 return new Schema.FieldSchema(null, bagSchema, BAG);
             }
         default: {
-                throw new ExecException("Cannot determine field schema for " + o);
+                int errCode = 1073;
+                String msg = "Cannot determine field schema for " + o;
+                throw new ExecException(msg, errCode, PigException.INPUT);
             }
         }
     }

Modified: hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java?rev=737239&r1=737238&r2=737239&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/data/DefaultTuple.java Fri Jan 23 23:07:30 2009
@@ -31,6 +31,7 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.WritableComparable;
 
+import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 
 /**
@@ -372,8 +373,10 @@
 
     private void checkBounds(int fieldNum) throws ExecException {
         if (fieldNum >= mFields.size()) {
-            throw new ExecException("Out of bounds access: Request for field number " + fieldNum +
-                " exceeds tuple size of " + mFields.size());
+            int errCode = 1072;
+            String msg = "Out of bounds access: Request for field number " + fieldNum +
+            " exceeds tuple size of " + mFields.size();
+            throw new ExecException(msg, errCode, PigException.INPUT);
         }
     }
     

Modified: hadoop/pig/trunk/src/org/apache/pig/impl/PigContext.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/PigContext.java?rev=737239&r1=737238&r2=737239&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/impl/PigContext.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/impl/PigContext.java Fri Jan 23 23:07:30 2009
@@ -43,6 +43,7 @@
 import org.apache.pig.FuncSpec;
 import org.apache.pig.Main;
 import org.apache.pig.ExecType;
+import org.apache.pig.PigException;
 import org.apache.pig.backend.datastorage.DataStorage;
 import org.apache.pig.backend.datastorage.DataStorageException;
 import org.apache.pig.backend.datastorage.ElementDescriptor;
@@ -180,7 +181,9 @@
             
             default:
             {
-                throw new ExecException("Unkown execType: " + execType);
+                int errCode = 2040;
+                String msg = "Unkown exec type: " + execType;
+                throw new ExecException(msg, errCode, PigException.BUG);
             }
         }
 
@@ -227,7 +230,21 @@
             src = dfs.asElement(oldName);            
         }
         catch (DataStorageException e) {
-            throw WrappedIOException.wrap("Unable to rename " + oldName + " to " + newName, e);
+            byte errSrc = getErrorSource();            
+            int errCode = 0;
+            switch(errSrc) {
+            case PigException.REMOTE_ENVIRONMENT:
+                errCode = 6005;
+                break;
+            case PigException.USER_ENVIRONMENT:
+                errCode = 4005;
+                break;
+            default:
+                errCode = 2038;
+                    break;
+            }
+            String msg = "Unable to rename " + oldName + " to " + newName;            
+            throw new PigException(msg, errCode, errSrc, e);
         }
 
         if (dst.exists()) {
@@ -253,7 +270,21 @@
             dstElement = dstStorage.asElement(dst);
         }
         catch (DataStorageException e) {
-            throw WrappedIOException.wrap("Unable to copy " + src + " to " + dst + (localDst ? "locally" : ""), e);
+            byte errSrc = getErrorSource();            
+            int errCode = 0;
+            switch(errSrc) {
+            case PigException.REMOTE_ENVIRONMENT:
+                errCode = 6006;
+                break;
+            case PigException.USER_ENVIRONMENT:
+                errCode = 4006;
+                break;
+            default:
+                errCode = 2039;
+                    break;
+            }
+            String msg = "Unable to copy " + src + " to " + dst;            
+            throw new PigException(msg, errCode, errSrc, e);
         }
         
         srcElement.copy(dstElement, this.properties, false);
@@ -379,15 +410,18 @@
                 // do nothing
             } 
             catch (UnsupportedClassVersionError e) {
-                throw WrappedIOException.wrap(e) ;
+                int errCode = 1069;
+                String msg = "Problem resolving class version numbers for class " + name;
+                throw new PigException(msg, errCode, PigException.INPUT, e) ;
             }
             
         }
 
         // create ClassNotFoundException exception and attach to IOException
         // so that we don't need to buble interface changes throughout the code
-        ClassNotFoundException e = new ClassNotFoundException("Could not resolve " + name + " using imports: " + packageImportList);
-        throw WrappedIOException.wrap(e.getMessage(), e);
+        int errCode = 1070;
+        String msg = "Could not resolve " + name + " using imports: " + packageImportList;
+        throw new PigException(msg, errCode, PigException.INPUT);
     }
     
     
@@ -419,7 +453,7 @@
             }
         }
         catch(NoSuchMethodException nme) {
-            // Second channce. Try with var arg constructor
+            // Second chance. Try with var arg constructor
             try {
                 Constructor c = objClass.getConstructor(String[].class);
                 Object[] wrappedArgs = new Object[1] ;
@@ -513,7 +547,9 @@
             break;
             default:
             {
-                throw new ExecException("Unkown execType: " + execType);
+                int errCode = 2040;
+                String msg = "Unkown exec type: " + execType;
+                throw new ExecException(msg, errCode, PigException.BUG);
             }
         }
         
@@ -548,4 +584,19 @@
     public List<String> getPathsToSkip() {
         return skippedShipPaths;
     }
+    
+    /**
+     * Check the execution mode and return the appropriate error source
+     * 
+     * @return error source
+     */
+    public byte getErrorSource() {
+        if(execType == ExecType.LOCAL) {
+            return PigException.USER_ENVIRONMENT;
+        } else if (execType == ExecType.MAPREDUCE) {
+            return PigException.REMOTE_ENVIRONMENT;
+        } else {
+            return PigException.BUG;
+        }        
+    }
 }

Modified: hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/ExpressionOperator.java
URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/ExpressionOperator.java?rev=737239&r1=737238&r2=737239&view=diff
==============================================================================
--- hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/ExpressionOperator.java (original)
+++ hadoop/pig/trunk/src/org/apache/pig/impl/logicalLayer/ExpressionOperator.java Fri Jan 23 23:07:30 2009
@@ -19,9 +19,9 @@
 
 import java.util.List;
 import java.util.ArrayList;
+
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.data.DataType;
-import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.logicalLayer.parser.ParseException;
 import org.apache.pig.impl.plan.OperatorKey;
 import org.apache.commons.logging.Log;