You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ga...@apache.org on 2008/04/05 00:08:50 UTC

svn commit: r644955 [1/2] - in /incubator/pig/branches/types: ./ src/org/apache/pig/data/ src/org/apache/pig/impl/physicalLayer/ src/org/apache/pig/impl/physicalLayer/plans/ src/org/apache/pig/impl/physicalLayer/topLevelOperators/ src/org/apache/pig/im...

Author: gates
Date: Fri Apr  4 15:08:46 2008
New Revision: 644955

URL: http://svn.apache.org/viewvc?rev=644955&view=rev
Log:
Added Comparison operators and small changes to the Filter and Project operators contributed by Shravan.


Added:
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/EqualToExpr.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GTOrEqualToExpr.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LTOrEqualToExpr.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LessThanExpr.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/NotEqualToExpr.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestEqualTo.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestGTOrEqual.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestGreaterThan.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestLTOrEqual.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestLessThan.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestNotEqualTo.java
Modified:
    incubator/pig/branches/types/build.xml
    incubator/pig/branches/types/src/org/apache/pig/data/DataType.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POStatus.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/Result.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/PhyPlanVisitor.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POFilter.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/PhysicalOperator.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ExpressionOperator.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POProject.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GreaterThanExpr.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestFilter.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestPhyOp.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestProject.java
    incubator/pig/branches/types/test/org/apache/pig/test/utils/GenPhyOp.java
    incubator/pig/branches/types/test/org/apache/pig/test/utils/GenRandomData.java

Modified: incubator/pig/branches/types/build.xml
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/build.xml?rev=644955&r1=644954&r2=644955&view=diff
==============================================================================
--- incubator/pig/branches/types/build.xml (original)
+++ incubator/pig/branches/types/build.xml Fri Apr  4 15:08:46 2008
@@ -137,12 +137,13 @@
                 **/test/TestOperatorPlan.java, **/test/TestBuiltin.java,
                 **/test/TestConstExpr.java, **/test/TestFilter.java, **/test/TestPhyOp.java,
                 **/test/TestAdd.java, **/test/TestSubtract.java, **/test/TestMultiply.java,
-                **/test/TestDivide.java, **/test/TestMod.java,
+                **/test/TestDivide.java, **/test/TestMod.java, **/test/TestGreaterThan.java,
+        	    **/test/TestGTOrEqual.java,**/test/TestLessThan.java,**/test/TestLTOrEqual.java,
+        	    **/test/TestEqualTo.java,**/test/TestNotEqualTo.java
                 **/test/TestProject.java, **/test/utils/*.java, **/logicalLayer/*.java,
                 **/logicalLayer/schema/*.java, **/physicalLayer/topLevelOperators/*.java,
                 **/physicalLayer/topLevelOperator/**/*.java, **/physicalLayer/plans/*.java,
-                **/physicalLayer/util/*.java, **/physicalLayer/Result.java,
-                **/physicalLayer/POStatus.java"
+				**/physicalLayer/Result.java, **/physicalLayer/POStatus.java"
             destdir="${dist}" debug="${javac.debug}" optimize="${javac.optimize}" target="${javac.version}" source="${javac.version}" deprecation="${javac.deprecation}">
             <compilerarg line="${javac.args} ${javac.args.warnings}" />
             <classpath refid="${cp}" />
@@ -236,6 +237,12 @@
                 	<include name="**/TestMultiply.java" />
                 	<include name="**/TestDivide.java" />
                 	<include name="**/TestMod.java" />
+                	<include name="**/TestGreaterThan.java" />
+                	<include name="**/TestGTOrEqual.java" />
+                	<include name="**/TestLessThan.java" />
+                	<include name="**/TestLTOrEqual.java" />
+                	<include name="**/TestEqualTo.java" />
+                	<include name="**/TestNotEqualTo.java" />
                     <!--
                     <include name="**/*Test*.java" />
                     <exclude name="**/TestLargeFile.java" />

Modified: incubator/pig/branches/types/src/org/apache/pig/data/DataType.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/data/DataType.java?rev=644955&r1=644954&r2=644955&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/data/DataType.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/data/DataType.java Fri Apr  4 15:08:46 2008
@@ -20,6 +20,7 @@
 import java.io.IOException;
 import java.lang.Class;
 import java.lang.reflect.Type;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.TreeMap;
@@ -107,6 +108,32 @@
             }
             return ERROR;
         }
+    }
+    
+    public static int numTypes(){
+        byte[] types = genAllTypes();
+        return types.length;
+    }
+    public static byte[] genAllTypes(){
+        byte[] types = { DataType.BAG, DataType.BOOLEAN, DataType.BYTEARRAY, DataType.CHARARRAY, 
+                DataType.DOUBLE, DataType.FLOAT, DataType.INTEGER, DataType.LONG, DataType.MAP, DataType.TUPLE};
+        return types;
+    }
+    
+    private static String[] genAllTypeNames(){
+        String[] names = { "BAG", "BOOLEAN", "BYTEARRAY", "CHARARRAY", "DOUBLE", "FLOAT", "INTEGER", "LONG", 
+                "MAP", "TUPLE" };
+        return names;
+    }
+    
+    public static Map<Byte, String> genTypeToNameMap(){
+        byte[] types = genAllTypes();
+        String[] names = genAllTypeNames();
+        Map<Byte,String> ret = new HashMap<Byte, String>();
+        for(int i=0;i<types.length;i++){
+            ret.put(types[i], names[i]);
+        }
+        return ret;
     }
 
     /**

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POStatus.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POStatus.java?rev=644955&r1=644954&r2=644955&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POStatus.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/POStatus.java Fri Apr  4 15:08:46 2008
@@ -19,9 +19,12 @@
 
 public class POStatus {
     public static final byte STATUS_OK = 0;
+
     public static final byte STATUS_NULL = 1;
+
     public static final byte STATUS_ERR = 2;
-    public static final byte STATUS_EOP = 3; // end of processing 
-    
+
+    public static final byte STATUS_EOP = 3; // end of processing
+
     public static Object result;
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/Result.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/Result.java?rev=644955&r1=644954&r2=644955&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/Result.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/Result.java Fri Apr  4 15:08:46 2008
@@ -19,15 +19,17 @@
 
 import java.io.Serializable;
 
-public class Result implements Serializable{
+public class Result implements Serializable {
     /**
      * 
      */
     private static final long serialVersionUID = 1L;
+
     public byte returnStatus;
+
     public Object result;
-    
-    public Result(){
+
+    public Result() {
         returnStatus = POStatus.STATUS_ERR;
         result = null;
     }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java?rev=644955&r1=644954&r2=644955&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/ExprPlanVisitor.java Fri Apr  4 15:08:46 2008
@@ -29,11 +29,12 @@
 import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.arithmeticOperators.Mod;
 import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.arithmeticOperators.Multiply;
 import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.arithmeticOperators.Subtract;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.EqualToExpr;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.GTOrEqualToExpr;
 import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.GreaterThanExpr;
-//import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.LTOrEqualToExpr;
-//import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.LessThanExpr;
-//import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.NotEqualToExpr;
-//import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.NotEqualToExpr;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.LTOrEqualToExpr;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.LessThanExpr;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.NotEqualToExpr;
 
 /**
  * The visitor to be used for visiting expression plans.
@@ -41,7 +42,7 @@
  * Call the visit() method for a depth first traversal.
  *
  */
-public abstract class ExprPlanVisitor extends PhyPlanVisitor<ExpressionOperator, ExprPlan> {
+public class ExprPlanVisitor extends PhyPlanVisitor<ExpressionOperator, ExprPlan> {
 
     private final Log log = LogFactory.getLog(getClass());
     
@@ -62,7 +63,7 @@
         //do nothing
     }
     
-    /*public void visitLessThan(LessThanExpr lt){
+    public void visitLessThan(LessThanExpr lt){
         //do nothing
     }
     
@@ -80,7 +81,7 @@
     
     public void visitNotEqualTo(NotEqualToExpr eq){
         //do nothing
-    }*/
+    }
     
     public void visitAdd(Add add) {
         //do nothing

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/PhyPlanVisitor.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/PhyPlanVisitor.java?rev=644955&r1=644954&r2=644955&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/PhyPlanVisitor.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/PhyPlanVisitor.java Fri Apr  4 15:08:46 2008
@@ -35,10 +35,10 @@
  * the visit() method to traverse the plan in a depth first
  * fashion.
  * 
- * This class can be used to visit only the top level 
- * relational operators. Classes extending this should
- * use the specific plans to visit the attribute plans
- * of each of the top level operators.
+ * This class also visits the nested plans inside the operators.
+ * One has to extend this class to modify the nature of each visit
+ * and to maintain any relevant state information between the visits
+ * to two different operators.
  *
  * @param <O>
  * @param <P>
@@ -62,8 +62,9 @@
 //        //do nothing
 //    }
 //    
-    public void visitFilter(POFilter fl){
-        //do nothing
+    public void visitFilter(POFilter fl) throws ParseException{
+        ExprPlanVisitor epv = new ExprPlanVisitor(fl.getPlan());
+        epv.visit();
     }
 //    
 //    public void visitLocalRearrange(POLocalRearrange lr){

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POFilter.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POFilter.java?rev=644955&r1=644954&r2=644955&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POFilter.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POFilter.java Fri Apr  4 15:08:46 2008
@@ -22,7 +22,6 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.pig.backend.executionengine.ExecException;
-import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.OperatorKey;
@@ -33,43 +32,31 @@
 import org.apache.pig.impl.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators.ComparisonOperator;
 
-
 /**
- * This is an implementation of the Filter operator. It has
- * an Expression Plan that decides whether the input tuple
- * should be filtered or passed through. To avoid many function
- * calls, the filter operator, stores the Comparison Operator
- * that is the root of the Expression Plan and uses its getNext
+ * This is an implementation of the Filter operator. It has an Expression Plan
+ * that decides whether the input tuple should be filtered or passed through. To
+ * avoid many function calls, the filter operator, stores the Comparison
+ * Operator that is the root of the Expression Plan and uses its getNext
  * directly.
  * 
- * Since the filter is supposed to return tuples only, getNext
- * is not supported on any other data type.
- *
+ * Since the filter is supposed to return tuples only, getNext is not supported
+ * on any other data type.
+ * 
  */
 public class POFilter extends PhysicalOperator<PhyPlanVisitor> {
 
     private Log log = LogFactory.getLog(getClass());
-    
-    //The expression plan
+
+    // The expression plan
     ExprPlan plan;
-    
-    //The root comparison operator of the expression plan
+
+    // The root comparison operator of the expression plan
     ComparisonOperator comOp;
-    
-    //The operand type for the comparison operator needed
-    //to call the comparison operators getNext with the 
-    //appropriate type
+
+    // The operand type for the comparison operator needed
+    // to call the comparison operators getNext with the
+    // appropriate type
     byte compOperandType;
-    
-    //Dummy types used to access the getNext of appropriate
-    //type
-    DataByteArray ba;
-    String s;
-    Double d;
-    Float f;
-    Integer i;
-    Long l;
-    
 
     public POFilter(OperatorKey k) {
         this(k, -1, null);
@@ -78,74 +65,79 @@
     public POFilter(OperatorKey k, int rp) {
         this(k, rp, null);
     }
-    
-    public POFilter(OperatorKey k, List<PhysicalOperator<PhyPlanVisitor>> inputs){
-        this(k,-1,inputs);
+
+    public POFilter(OperatorKey k, List<PhysicalOperator> inputs) {
+        this(k, -1, inputs);
     }
 
-    public POFilter(OperatorKey k, int rp, List<PhysicalOperator<PhyPlanVisitor>> inputs) {
+    public POFilter(OperatorKey k, int rp, List<PhysicalOperator> inputs) {
         super(k, rp, inputs);
     }
-    
+
     /**
-     * Attaches the proccesed input tuple to the expression plan
-     * and checks if comparison operator returns a true. If so the
-     * tuple is not filtered and let to pass through. Else, further
-     * input is processed till a tuple that can be passed through is
-     * found or EOP is reached.
+     * Attaches the proccesed input tuple to the expression plan and checks if
+     * comparison operator returns a true. If so the tuple is not filtered and
+     * let to pass through. Else, further input is processed till a tuple that
+     * can be passed through is found or EOP is reached.
      */
     @Override
     public Result getNext(Tuple t) throws ExecException {
         Result res = null;
         Result inp = null;
-        while(true){
+        while (true) {
             inp = processInput();
-            if(inp.returnStatus==POStatus.STATUS_EOP)
+            if (inp.returnStatus == POStatus.STATUS_EOP
+                    || inp.returnStatus == POStatus.STATUS_ERR)
                 break;
-            if(inp.returnStatus==POStatus.STATUS_NULL || inp.returnStatus==POStatus.STATUS_ERR){
-                log.warn("An erroneous/Null return status was returned by processInput(). Continuing with further input.");
+            if (inp.returnStatus == POStatus.STATUS_NULL) {
                 continue;
             }
-            
-            plan.attachInput((Tuple)inp.result);
-            
-            switch(compOperandType){
+
+            plan.attachInput((Tuple) inp.result);
+
+            switch (compOperandType) {
             case DataType.BYTEARRAY:
-                res = comOp.getNext(ba);
-                if(res.returnStatus!=POStatus.STATUS_OK) continue;
+                res = comOp.getNext(dummyDBA);
+                if (res.returnStatus != POStatus.STATUS_OK)
+                    continue;
                 break;
             case DataType.CHARARRAY:
-                res = comOp.getNext(s);
-                if(res.returnStatus!=POStatus.STATUS_OK) continue;
+                res = comOp.getNext(dummyString);
+                if (res.returnStatus != POStatus.STATUS_OK)
+                    continue;
                 break;
             case DataType.DOUBLE:
-                res = comOp.getNext(d);
-                if(res.returnStatus!=POStatus.STATUS_OK) continue;
+                res = comOp.getNext(dummyDouble);
+                if (res.returnStatus != POStatus.STATUS_OK)
+                    continue;
                 break;
             case DataType.FLOAT:
-                res = comOp.getNext(f);
-                if(res.returnStatus!=POStatus.STATUS_OK) continue;
+                res = comOp.getNext(dummyFloat);
+                if (res.returnStatus != POStatus.STATUS_OK)
+                    continue;
                 break;
             case DataType.INTEGER:
-                res = comOp.getNext(i);
-                if(res.returnStatus!=POStatus.STATUS_OK) continue;
+                res = comOp.getNext(dummyInt);
+                if (res.returnStatus != POStatus.STATUS_OK)
+                    continue;
                 break;
             case DataType.LONG:
-                res = comOp.getNext(l);
-                if(res.returnStatus!=POStatus.STATUS_OK) continue;
+                res = comOp.getNext(dummyLong);
+                if (res.returnStatus != POStatus.STATUS_OK)
+                    continue;
                 break;
             }
-            
-            if(res==null){
+
+            if (res == null) {
                 return new Result();
             }
-            if((Boolean)res.result==true){
+            if ((Boolean) res.result == true) {
                 return inp;
             }
         }
         return inp;
     }
-    
+
     @Override
     public String name() {
         return "Filter - " + mKey.toString();
@@ -168,7 +160,11 @@
 
     public void setPlan(ExprPlan plan) {
         this.plan = plan;
-        comOp = (ComparisonOperator)(plan.getLeaves()).get(0);
+        comOp = (ComparisonOperator) (plan.getLeaves()).get(0);
         compOperandType = comOp.getOperandType();
+    }
+
+    public ExprPlan getPlan() {
+        return plan;
     }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/PhysicalOperator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/PhysicalOperator.java?rev=644955&r1=644954&r2=644955&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/PhysicalOperator.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/PhysicalOperator.java Fri Apr  4 15:08:46 2008
@@ -36,69 +36,91 @@
 
 /**
  * 
- * This is the base class for all operators. This supports a 
- * generic way of processing inputs which can be overridden by 
- * operators extending this class. The input model assumes that
- * it can either be taken from an operator or can be attached 
- * directly to this operator. Also it is assumed that inputs to an
+ * This is the base class for all operators. This supports a generic way of
+ * processing inputs which can be overridden by operators extending this class.
+ * The input model assumes that it can either be taken from an operator or can
+ * be attached directly to this operator. Also it is assumed that inputs to an
  * operator are always in the form of a tuple.
  * 
- * For this pipeline rework, we assume a pull based model, i.e, 
- * the root operator is going to call getNext with the appropriate
- * type which initiates a cascade of getNext calls that unroll to
- * create input for the root operator to work on. 
+ * For this pipeline rework, we assume a pull based model, i.e, the root
+ * operator is going to call getNext with the appropriate type which initiates a
+ * cascade of getNext calls that unroll to create input for the root operator to
+ * work on.
+ * 
+ * Any operator that extends the PhysicalOperator, supports a getNext with all
+ * the different types of parameter types. The concrete implementation should
+ * use the result type of its input operator to decide the type of getNext's
+ * parameter. This is done to avoid switch/case based on the type as much as
+ * possible. The default is assumed to return an erroneus Result corresponding
+ * to an unsupported operation on that type. So the operators need to implement
+ * only those types that are supported.
  * 
- * Any operator that extends the PhysicalOperator, supports a getNext
- * with all the different types of parameter types. The concrete implementation
- * should use the result type of its input operator to decide the type of getNext's
- * parameter. This is done to avoid switch/case based on the type
- * as much as possible. The default is assumed to return an erroneus Result
- * corresponding to an unsupported operation on that type. So the operators need
- * to implement only those types that are supported.
- *
  * @param <V>
  */
-public abstract class PhysicalOperator<V extends PhyPlanVisitor> extends Operator<V> {
-    
+public abstract class PhysicalOperator<V extends PhyPlanVisitor> extends
+        Operator<V> {
+
     private Log log = LogFactory.getLog(getClass());
-    
+
     static final long serialVersionUID = 1L;
-    
-    //The degree of parallelism requested
+
+    // The degree of parallelism requested
     protected int requestedParallelism;
-    
-    //The inputs that this operator will read data from
-    protected List<PhysicalOperator<V>> inputs;
-    
-    //The outputs that this operator will write data to
-    //Will be used to create Targeted tuples
-    protected List<PhysicalOperator<V>> outputs;
-    
-    //The data type for the results of this operator
+
+    // The inputs that this operator will read data from
+    protected List<PhysicalOperator> inputs;
+
+    // The outputs that this operator will write data to
+    // Will be used to create Targeted tuples
+    protected List<PhysicalOperator> outputs;
+
+    // The data type for the results of this operator
     protected byte resultType = DataType.TUPLE;
-    
-    //Specifies if the input has been directly attached
+
+    // Specifies if the input has been directly attached
     protected boolean inputAttached = false;
-    
-    //If inputAttached is true, input is set to the input tuple
+
+    // If inputAttached is true, input is set to the input tuple
     protected Tuple input = null;
-    
-    //The result of performing the operation along with the output
+
+    // The result of performing the operation along with the output
     protected Result res = null;
 
+    // Dummy types used to access the getNext of appropriate
+    // type. These will be null
+    static DataByteArray dummyDBA;
+
+    static String dummyString;
+
+    static Double dummyDouble;
+
+    static Float dummyFloat;
+
+    static Integer dummyInt;
+
+    static Long dummyLong;
+
+    static Boolean dummyBool;
+
+    static Tuple dummyTuple;
+
+    static DataBag dummyBag;
+
+    static Map dummyMap;
+
     public PhysicalOperator(OperatorKey k) {
-        this(k,-1,null);
+        this(k, -1, null);
     }
-    
-    public PhysicalOperator(OperatorKey k, int rp){
-        this(k,rp,null);
-    }
-    
-    public PhysicalOperator(OperatorKey k, List<PhysicalOperator<V>> inp){
-        this(k,-1,inp);
+
+    public PhysicalOperator(OperatorKey k, int rp) {
+        this(k, rp, null);
+    }
+
+    public PhysicalOperator(OperatorKey k, List<PhysicalOperator> inp) {
+        this(k, -1, inp);
     }
-    
-    public PhysicalOperator(OperatorKey k, int rp, List<PhysicalOperator<V>> inp){
+
+    public PhysicalOperator(OperatorKey k, int rp, List<PhysicalOperator> inp) {
         super(k);
         requestedParallelism = rp;
         inputs = inp;
@@ -112,7 +134,7 @@
     public void setRequestedParallelism(int requestedParallelism) {
         this.requestedParallelism = requestedParallelism;
     }
-    
+
     public byte getResultType() {
         return resultType;
     }
@@ -120,126 +142,116 @@
     public void setResultType(byte resultType) {
         this.resultType = resultType;
     }
-    
-    public List<PhysicalOperator<V>> getInputs() {
+
+    public List<PhysicalOperator> getInputs() {
         return inputs;
     }
-    
-    public void setInputs(List<PhysicalOperator<V>> inputs) {
+
+    public void setInputs(List<PhysicalOperator> inputs) {
         this.inputs = inputs;
     }
-    
+
     public boolean isInputAttached() {
         return inputAttached;
     }
 
-    public void setInputAttached(boolean inputAttached) {
-        this.inputAttached = inputAttached;
-    }
-    
     /**
-     * Shorts the input path of this operator by providing
-     * the input tuple directly
-     * @param t - The tuple that should be used as input
+     * Shorts the input path of this operator by providing the input tuple
+     * directly
+     * 
+     * @param t -
+     *            The tuple that should be used as input
      */
-    public void attachInput(Tuple t){
+    public void attachInput(Tuple t) {
         input = t;
         this.inputAttached = true;
     }
-    
+
     /**
      * Detaches any tuples that are attached
-     *
+     * 
      */
-    public void detachInput(){
+    public void detachInput() {
         input = null;
         this.inputAttached = false;
     }
-    
+
     /**
-     * A blocking operator should override this to return true.
-     * Blocking operators are those that need the full bag before
-     * operate on the tuples inside the bag. Example is the Global Rearrange.
-     * Non-blocking or pipeline operators are those that work on
-     * a tuple by tuple basis.
+     * A blocking operator should override this to return true. Blocking
+     * operators are those that need the full bag before operate on the tuples
+     * inside the bag. Example is the Global Rearrange. Non-blocking or pipeline
+     * operators are those that work on a tuple by tuple basis.
+     * 
      * @return true if blocking and false otherwise
      */
-    public boolean isBlocking(){
+    public boolean isBlocking() {
         return false;
     }
-    
+
     /**
-     * A generic method for parsing input that either returns 
-     * the attached input if it exists or fetches it from its
-     * predecessor. If special processing is required, this 
-     * method should be overridden.
-     * @return The Result object that results from processing the
-     *             input
+     * A generic method for parsing input that either returns the attached input
+     * if it exists or fetches it from its predecessor. If special processing is
+     * required, this method should be overridden.
+     * 
+     * @return The Result object that results from processing the input
      * @throws ExecException
      */
-    public Result processInput() throws ExecException{
+    public Result processInput() throws ExecException {
         Result res = new Result();
         Tuple inpValue = null;
-        if(input==null && inputs==null) {
+        if (input == null && inputs == null) {
             log.warn("No inputs found. Signaling End of Processing.");
             res.returnStatus = POStatus.STATUS_EOP;
             return res;
         }
-        if(!isInputAttached())
+        if (!isInputAttached())
             return inputs.get(0).getNext(inpValue);
-        else{
+        else {
             res.result = input;
             res.returnStatus = POStatus.STATUS_OK;
             detachInput();
             return res;
         }
     }
-    
-    public abstract void visit(V v) throws ParseException ;
+
+    public abstract void visit(V v) throws ParseException;
 
     public Result getNext(Integer i) throws ExecException {
         return res;
     }
-    
-    
+
     public Result getNext(Long l) throws ExecException {
         return res;
     }
 
-    
     public Result getNext(Double d) throws ExecException {
         return res;
     }
 
-    
     public Result getNext(Float f) throws ExecException {
         return res;
     }
 
-    
     public Result getNext(String s) throws ExecException {
         return res;
     }
 
-    
     public Result getNext(DataByteArray ba) throws ExecException {
         return res;
     }
-    
-    public Result getNext(Map m) throws ExecException{
+
+    public Result getNext(Map m) throws ExecException {
         return res;
     }
-    
-    public Result getNext(Boolean b) throws ExecException{
+
+    public Result getNext(Boolean b) throws ExecException {
         return res;
     }
 
-    
     public Result getNext(Tuple t) throws ExecException {
         return res;
     }
 
-    
     public Result getNext(DataBag db) throws ExecException {
         return res;
     }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ExpressionOperator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ExpressionOperator.java?rev=644955&r1=644954&r2=644955&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ExpressionOperator.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ExpressionOperator.java Fri Apr  4 15:08:46 2008
@@ -45,6 +45,4 @@
     }
     
     public abstract void visit(ExprPlanVisitor v) throws ParseException;
-    
-    
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POProject.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POProject.java?rev=644955&r1=644954&r2=644955&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POProject.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POProject.java Fri Apr  4 15:08:46 2008
@@ -18,24 +18,20 @@
 package org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators;
 
 import java.io.IOException;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
-import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
-import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.OperatorKey;
 import org.apache.pig.impl.logicalLayer.parser.ParseException;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
-import org.apache.pig.backend.executionengine.ExecException;
-import org.apache.pig.impl.physicalLayer.util.operatorHelper;
 
 /**
  * Implements the overloaded form of the project operator.
@@ -58,8 +54,11 @@
     //The bag iterator used while straeming tuple
     Iterator<Tuple> bagIterator = null;
     
-    //Temporary tuple
-    Tuple temp = null;
+    //Represents the fact that this instance of POProject
+    //is overloaded to stream tuples in the bag rather
+    //than passing the entire bag. It is the responsibility
+    //of the translator to set this.
+    boolean overloaded = false;
     
     public POProject(OperatorKey k) {
         this(k,-1,0);
@@ -115,19 +114,13 @@
         Result res = processInput();
 
         if(res.returnStatus != POStatus.STATUS_OK){
-            if((res.returnStatus == POStatus.STATUS_ERR)){
-                res.returnStatus = POStatus.STATUS_NULL;
-            }
             return res;
         }
         try {
             res.result = ((Tuple)res.result).get(column);
         } catch (IOException e) {
-            //Instead of propagating ERR through the pipeline
-            //considering this a nullified operation and returning NULL 
-            //res.returnStatus = POStatus.STATUS_ERR;
-            res.returnStatus = POStatus.STATUS_NULL;
-            log.fatal(e.getMessage());
+            res.returnStatus = POStatus.STATUS_ERR;
+            log.warn(e.getMessage());
         }
         return res;
     }
@@ -197,7 +190,7 @@
             
             try {
                 Object ret = inpValue.get(column);
-                if(ret instanceof DataBag){
+                if(overloaded){
                     DataBag retBag = (DataBag)ret;
                     bagIterator = retBag.iterator();
                     if(bagIterator.hasNext()){
@@ -232,6 +225,14 @@
 
     public void setColumn(int column) {
         this.column = column;
+    }
+
+    public boolean isOverloaded() {
+        return overloaded;
+    }
+
+    public void setOverloaded(boolean overloaded) {
+        this.overloaded = overloaded;
     }
 
 }

Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/EqualToExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/EqualToExpr.java?rev=644955&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/EqualToExpr.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/EqualToExpr.java Fri Apr  4 15:08:46 2008
@@ -0,0 +1,421 @@
+/*
+ * 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.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators;
+
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.OperatorKey;
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.impl.physicalLayer.POStatus;
+import org.apache.pig.impl.physicalLayer.Result;
+import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
+
+public class EqualToExpr extends ComparisonOperator {
+
+    private final Log log = LogFactory.getLog(getClass());
+
+    public EqualToExpr(OperatorKey k) {
+        this(k, -1);
+    }
+
+    public EqualToExpr(OperatorKey k, int rp) {
+        super(k, rp);
+    }
+
+    @Override
+    public void visit(ExprPlanVisitor v) throws ParseException {
+        v.visitEqualTo(this);
+    }
+
+    @Override
+    public String name() {
+        return "Equal To - " + mKey.toString();
+    }
+
+    @Override
+    public Result getNext(DataByteArray inp) throws ExecException {
+        byte status;
+        Result res;
+
+        DataByteArray left = null, right = null;
+
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (DataByteArray) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (DataByteArray) res.result;
+
+        int ret = left.compareTo(right);
+        if (ret == 0) {
+
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Double inp) throws ExecException {
+        byte status;
+        Result res;
+
+        Double left = null, right = null;
+
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Double) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Double) res.result;
+
+        if (left == right) {
+
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Float inp) throws ExecException {
+        byte status;
+        Result res;
+
+        Float left = null, right = null;
+
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Float) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Float) res.result;
+
+        if (left == right) {
+
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Integer inp) throws ExecException {
+        byte status;
+        Result res;
+
+        Integer left = null, right = null;
+
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Integer) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Integer) res.result;
+
+        if (left == right) {
+
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Long inp) throws ExecException {
+        byte status;
+        Result res;
+
+        Long left = null, right = null;
+
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Long) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Long) res.result;
+
+        if (left == right) {
+
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(String inp) throws ExecException {
+        byte status;
+        Result res;
+
+        String left = null, right = null;
+
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (String) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (String) res.result;
+
+        int ret = left.compareTo(right);
+        if (ret == 0) {
+
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Boolean inp) throws ExecException {
+        byte status;
+        Result res;
+
+        Boolean left = null, right = null;
+
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Boolean) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Boolean) res.result;
+
+        if (left.equals(right)) {
+
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(DataBag inp) throws ExecException {
+        byte status;
+        Result res;
+
+        DataBag left = null, right = null;
+
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (DataBag) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (DataBag) res.result;
+
+        int ret = left.compareTo(right);
+        if (ret == 0) {
+
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Map inp) throws ExecException {
+        byte status;
+        Result res;
+
+        Map left = null, right = null;
+
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Map) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Map) res.result;
+
+        int ret = DataType.compare(left, right);
+        if (ret == 0) {
+
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Tuple inp) throws ExecException {
+        byte status;
+        Result res;
+
+        Tuple left = null, right = null;
+
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Tuple) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Tuple) res.result;
+
+        int ret = left.compareTo(right);
+        if (ret == 0) {
+
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+}

Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GTOrEqualToExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GTOrEqualToExpr.java?rev=644955&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GTOrEqualToExpr.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GTOrEqualToExpr.java Fri Apr  4 15:08:46 2008
@@ -0,0 +1,263 @@
+/*
+ * 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.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.impl.logicalLayer.OperatorKey;
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
+import org.apache.pig.impl.physicalLayer.POStatus;
+import org.apache.pig.impl.physicalLayer.Result;
+import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
+import org.apache.pig.backend.executionengine.ExecException;
+
+
+public class GTOrEqualToExpr extends ComparisonOperator {
+
+    private final Log log = LogFactory.getLog(getClass());
+    
+    public GTOrEqualToExpr(OperatorKey k) {
+        this(k,-1);
+    }
+
+    public GTOrEqualToExpr(OperatorKey k, int rp) {
+        super(k, rp);
+    }
+
+    @Override
+    public void visit(ExprPlanVisitor v) throws ParseException {
+        v.visitGTOrEqual(this);
+    }
+
+    @Override
+    public String name() {
+        return "Greater Than or Equal - " + mKey.toString();
+    }
+    
+    @Override
+    public Result getNext(DataByteArray ba) throws ExecException {
+        byte status;
+        Result res;
+        DataByteArray left=null, right=null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (DataByteArray)res.result;
+        
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if(status!=POStatus.STATUS_OK) { 
+             
+                
+            return res;
+        }
+        right = (DataByteArray)res.result;
+        
+        int ret = left.compareTo(right);
+        if(ret==-1 || ret==0){
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        }
+        else{
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Double d) throws ExecException {
+        byte status;
+        Result res;
+        Double left=null, right=null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if(status!=POStatus.STATUS_OK) { 
+             
+                
+            return res;
+        }
+        left = (Double)res.result;
+        
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if(status!=POStatus.STATUS_OK) { 
+             
+                
+            return res;
+        }
+        right = (Double)res.result;
+        
+        if(left>=right){
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        }
+        else{
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Float f) throws ExecException {
+        byte status;
+        Result res;
+        Float left=null, right=null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if(status!=POStatus.STATUS_OK) { 
+             
+                
+            return res;
+        }
+        left = (Float)res.result;
+        
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if(status!=POStatus.STATUS_OK) { 
+             
+                
+            return res;
+        }
+        right = (Float)res.result;
+        
+        if(left>=right){
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        }
+        else{
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Integer i) throws ExecException {
+        byte status;
+        Result res;
+        Integer left=null, right=null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if(status!=POStatus.STATUS_OK) { 
+             
+                
+            return res;
+        }
+        left = (Integer)res.result;
+        
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if(status!=POStatus.STATUS_OK) { 
+             
+                
+            return res;
+        }
+        right = (Integer)res.result;
+        
+        if(left>=right){
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        }
+        else{
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Long l) throws ExecException {
+        byte status;
+        Result res;
+        Long left=null, right=null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if(status!=POStatus.STATUS_OK) { 
+             
+                
+            return res;
+        }
+        left = (Long)res.result;
+        
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if(status!=POStatus.STATUS_OK) { 
+             
+                
+            return res;
+        }
+        right = (Long)res.result;
+        
+        if(left>=right){
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        }
+        else{
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(String s) throws ExecException {
+        byte status;
+        Result res;
+        String left=null, right=null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if(status!=POStatus.STATUS_OK) { 
+             
+                
+            return res;
+        }
+        left = (String)res.result;
+        
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if(status!=POStatus.STATUS_OK) { 
+             
+                
+            return res;
+        }
+        right = (String)res.result;
+        
+        int ret = left.compareTo(right);
+        if(ret>=0){
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        }
+        else{
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+}

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GreaterThanExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GreaterThanExpr.java?rev=644955&r1=644954&r2=644955&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GreaterThanExpr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GreaterThanExpr.java Fri Apr  4 15:08:46 2008
@@ -27,13 +27,12 @@
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.backend.executionengine.ExecException;
 
-
 public class GreaterThanExpr extends ComparisonOperator {
 
     private final Log log = LogFactory.getLog(getClass());
-    
+
     public GreaterThanExpr(OperatorKey k) {
-        this(k,-1);
+        this(k, -1);
     }
 
     public GreaterThanExpr(OperatorKey k, int rp) {
@@ -49,41 +48,36 @@
     public void visit(ExprPlanVisitor v) throws ParseException {
         v.visitGreaterThan(this);
     }
-    
+
     @Override
     public Result getNext(DataByteArray ba) throws ExecException {
         byte status;
         Result res;
-        DataByteArray left=null, right=null;
+        DataByteArray left = null, right = null;
         res = lhs.getNext(left);
         status = res.returnStatus;
-        if(status!=POStatus.STATUS_OK) { 
-            log.warn("\t\t Recieved an error or a null or an unexpected end of processing. Hence reporting a NULL back.");
-            res.returnStatus=POStatus.STATUS_NULL;
+        if (status != POStatus.STATUS_OK) {
+
             return res;
         }
-        left = (DataByteArray)res.result;
-        
-        
-        
+        left = (DataByteArray) res.result;
+
         res = rhs.getNext(right);
         status = res.returnStatus;
-        if(status!=POStatus.STATUS_OK) { 
-            log.warn("\t\t Recieved an error or a null or an unexpected end of processing. Hence reporting a NULL back.");
-            res.returnStatus=POStatus.STATUS_NULL;
+        if (status != POStatus.STATUS_OK) {
+
             return res;
         }
-        right = (DataByteArray)res.result;
-        
+        right = (DataByteArray) res.result;
+
         int ret = left.compareTo(right);
-        if(ret==-1){
+        if (ret == -1) {
             res.result = new Boolean(true);
-            //left = right = null;
+            // left = right = null;
             return res;
-        }
-        else{
+        } else {
             res.result = new Boolean(false);
-            //left = right = null;
+            // left = right = null;
             return res;
         }
     }
@@ -92,35 +86,30 @@
     public Result getNext(Double d) throws ExecException {
         byte status;
         Result res;
-        Double left=null, right=null;
+        Double left = null, right = null;
         res = lhs.getNext(left);
         status = res.returnStatus;
-        if(status!=POStatus.STATUS_OK) { 
-            log.warn("\t\t Recieved an error or a null or an unexpected end of processing. Hence reporting a NULL back.");
-            res.returnStatus=POStatus.STATUS_NULL;
+        if (status != POStatus.STATUS_OK) {
+
             return res;
         }
-        left = (Double)res.result;
-        
-        
-        
+        left = (Double) res.result;
+
         res = rhs.getNext(right);
         status = res.returnStatus;
-        if(status!=POStatus.STATUS_OK) { 
-            log.warn("\t\t Recieved an error or a null or an unexpected end of processing. Hence reporting a NULL back.");
-            res.returnStatus=POStatus.STATUS_NULL;
+        if (status != POStatus.STATUS_OK) {
+
             return res;
         }
-        right = (Double)res.result;
-        
-        if(left>right){
+        right = (Double) res.result;
+
+        if (left > right) {
             res.result = new Boolean(true);
-            //left = right = null;
+            // left = right = null;
             return res;
-        }
-        else{
+        } else {
             res.result = new Boolean(false);
-            //left = right = null;
+            // left = right = null;
             return res;
         }
     }
@@ -129,35 +118,30 @@
     public Result getNext(Float f) throws ExecException {
         byte status;
         Result res;
-        Float left=null, right=null;
+        Float left = null, right = null;
         res = lhs.getNext(left);
         status = res.returnStatus;
-        if(status!=POStatus.STATUS_OK) { 
-            log.warn("\t\t Recieved an error or a null or an unexpected end of processing. Hence reporting a NULL back.");
-            res.returnStatus=POStatus.STATUS_NULL;
+        if (status != POStatus.STATUS_OK) {
+
             return res;
         }
-        left = (Float)res.result;
-        
-        
-        
+        left = (Float) res.result;
+
         res = rhs.getNext(right);
         status = res.returnStatus;
-        if(status!=POStatus.STATUS_OK) { 
-            log.warn("\t\t Recieved an error or a null or an unexpected end of processing. Hence reporting a NULL back.");
-            res.returnStatus=POStatus.STATUS_NULL;
+        if (status != POStatus.STATUS_OK) {
+
             return res;
         }
-        right = (Float)res.result;
-        
-        if(left>right){
+        right = (Float) res.result;
+
+        if (left > right) {
             res.result = new Boolean(true);
-            //left = right = null;
+            // left = right = null;
             return res;
-        }
-        else{
+        } else {
             res.result = new Boolean(false);
-            //left = right = null;
+            // left = right = null;
             return res;
         }
     }
@@ -166,35 +150,30 @@
     public Result getNext(Integer i) throws ExecException {
         byte status;
         Result res;
-        Integer left=null, right=null;
+        Integer left = null, right = null;
         res = lhs.getNext(left);
         status = res.returnStatus;
-        if(status!=POStatus.STATUS_OK) { 
-            log.warn("\t\t Recieved an error or a null or an unexpected end of processing. Hence reporting a NULL back.");
-            res.returnStatus=POStatus.STATUS_NULL;
+        if (status != POStatus.STATUS_OK) {
+
             return res;
         }
-        left = (Integer)res.result;
-        
-        
-        
+        left = (Integer) res.result;
+
         res = rhs.getNext(right);
         status = res.returnStatus;
-        if(status!=POStatus.STATUS_OK) { 
-            log.warn("\t\t Recieved an error or a null or an unexpected end of processing. Hence reporting a NULL back.");
-            res.returnStatus=POStatus.STATUS_NULL;
+        if (status != POStatus.STATUS_OK) {
+
             return res;
         }
-        right = (Integer)res.result;
-        
-        if(left>right){
+        right = (Integer) res.result;
+
+        if (left > right) {
             res.result = new Boolean(true);
-            //left = right = null;
+            // left = right = null;
             return res;
-        }
-        else{
+        } else {
             res.result = new Boolean(false);
-            //left = right = null;
+            // left = right = null;
             return res;
         }
     }
@@ -203,35 +182,30 @@
     public Result getNext(Long l) throws ExecException {
         byte status;
         Result res;
-        Long left=null, right=null;
+        Long left = null, right = null;
         res = lhs.getNext(left);
         status = res.returnStatus;
-        if(status!=POStatus.STATUS_OK) { 
-            log.warn("\t\t Recieved an error or a null or an unexpected end of processing. Hence reporting a NULL back.");
-            res.returnStatus=POStatus.STATUS_NULL;
+        if (status != POStatus.STATUS_OK) {
+
             return res;
         }
-        left = (Long)res.result;
-        
-        
-        
+        left = (Long) res.result;
+
         res = rhs.getNext(right);
         status = res.returnStatus;
-        if(status!=POStatus.STATUS_OK) { 
-            log.warn("\t\t Recieved an error or a null or an unexpected end of processing. Hence reporting a NULL back.");
-            res.returnStatus=POStatus.STATUS_NULL;
+        if (status != POStatus.STATUS_OK) {
+
             return res;
         }
-        right = (Long)res.result;
-        
-        if(left>right){
+        right = (Long) res.result;
+
+        if (left > right) {
             res.result = new Boolean(true);
-            //left = right = null;
+            // left = right = null;
             return res;
-        }
-        else{
+        } else {
             res.result = new Boolean(false);
-            //left = right = null;
+            // left = right = null;
             return res;
         }
     }
@@ -240,35 +214,30 @@
     public Result getNext(String s) throws ExecException {
         byte status;
         Result res;
-        String left=null, right=null;
-        
+        String left = null, right = null;
+
         res = lhs.getNext(left);
         status = res.returnStatus;
-        if(status!=POStatus.STATUS_OK) { 
-            log.warn("\t\t Recieved an error or a null or an unexpected end of processing. Hence reporting a NULL back.");
-            res.returnStatus=POStatus.STATUS_NULL;
+        if (status != POStatus.STATUS_OK) {
+
             return res;
         }
-        left = (String)res.result;
-        
-        
-        
+        left = (String) res.result;
+
         res = rhs.getNext(right);
         status = res.returnStatus;
-        if(status!=POStatus.STATUS_OK) { 
-            log.warn("\t\t Recieved an error or a null or an unexpected end of processing. Hence reporting a NULL back.");
-            res.returnStatus=POStatus.STATUS_NULL;
+        if (status != POStatus.STATUS_OK) {
+
             return res;
         }
-        right = (String)res.result;
-        
+        right = (String) res.result;
+
         int ret = left.compareTo(right);
-        if(ret>0){
+        if (ret > 0) {
             res.result = new Boolean(true);
-            //left = right = null;
+            // left = right = null;
             return res;
-        }
-        else{
+        } else {
             res.result = new Boolean(false);
             //left = right = null;
             return res;

Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LTOrEqualToExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LTOrEqualToExpr.java?rev=644955&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LTOrEqualToExpr.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LTOrEqualToExpr.java Fri Apr  4 15:08:46 2008
@@ -0,0 +1,244 @@
+/*
+ * 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.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.impl.logicalLayer.OperatorKey;
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
+import org.apache.pig.impl.physicalLayer.POStatus;
+import org.apache.pig.impl.physicalLayer.Result;
+import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
+import org.apache.pig.backend.executionengine.ExecException;
+
+public class LTOrEqualToExpr extends ComparisonOperator {
+    private final Log log = LogFactory.getLog(getClass());
+
+    public LTOrEqualToExpr(OperatorKey k) {
+        this(k, -1);
+    }
+
+    public LTOrEqualToExpr(OperatorKey k, int rp) {
+        super(k, rp);
+    }
+
+    @Override
+    public void visit(ExprPlanVisitor v) throws ParseException {
+        v.visiLTOrEqual(this);
+    }
+
+    @Override
+    public String name() {
+        return "Less Than or Equal - " + mKey.toString();
+    }
+
+    @Override
+    public Result getNext(DataByteArray ba) throws ExecException {
+        byte status;
+        Result res;
+        DataByteArray left = null, right = null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (DataByteArray) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (DataByteArray) res.result;
+
+        int ret = left.compareTo(right);
+        if (ret == 1 || ret == 0) {
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Double d) throws ExecException {
+        byte status;
+        Result res;
+        Double left = null, right = null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Double) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Double) res.result;
+
+        if (left <= right) {
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Float f) throws ExecException {
+        byte status;
+        Result res;
+        Float left = null, right = null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Float) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Float) res.result;
+
+        if (left <= right) {
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Integer i) throws ExecException {
+        byte status;
+        Result res;
+        Integer left = null, right = null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Integer) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Integer) res.result;
+
+        if (left <= right) {
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Long l) throws ExecException {
+        byte status;
+        Result res;
+        Long left = null, right = null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Long) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Long) res.result;
+
+        if (left <= right) {
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(String s) throws ExecException {
+        byte status;
+        Result res;
+        String left = null, right = null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (String) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (String) res.result;
+
+        int ret = left.compareTo(right);
+        if (ret <= 0) {
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+}

Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LessThanExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LessThanExpr.java?rev=644955&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LessThanExpr.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LessThanExpr.java Fri Apr  4 15:08:46 2008
@@ -0,0 +1,245 @@
+/*
+ * 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.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.impl.logicalLayer.OperatorKey;
+import org.apache.pig.impl.logicalLayer.parser.ParseException;
+import org.apache.pig.impl.physicalLayer.POStatus;
+import org.apache.pig.impl.physicalLayer.Result;
+import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
+import org.apache.pig.backend.executionengine.ExecException;
+
+public class LessThanExpr extends ComparisonOperator {
+
+    private final Log log = LogFactory.getLog(getClass());
+
+    public LessThanExpr(OperatorKey k) {
+        this(k, -1);
+    }
+
+    public LessThanExpr(OperatorKey k, int rp) {
+        super(k, rp);
+    }
+
+    @Override
+    public void visit(ExprPlanVisitor v) throws ParseException {
+        v.visitLessThan(this);
+    }
+
+    @Override
+    public String name() {
+        return "Less Than - " + mKey.toString();
+    }
+
+    @Override
+    public Result getNext(DataByteArray ba) throws ExecException {
+        byte status;
+        Result res;
+        DataByteArray left = null, right = null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (DataByteArray) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (DataByteArray) res.result;
+
+        int ret = left.compareTo(right);
+        if (ret == 1) {
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Double d) throws ExecException {
+        byte status;
+        Result res;
+        Double left = null, right = null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Double) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Double) res.result;
+
+        if (left < right) {
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Float f) throws ExecException {
+        byte status;
+        Result res;
+        Float left = null, right = null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Float) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Float) res.result;
+
+        if (left < right) {
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Integer i) throws ExecException {
+        byte status;
+        Result res;
+        Integer left = null, right = null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Integer) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Integer) res.result;
+
+        if (left < right) {
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(Long l) throws ExecException {
+        byte status;
+        Result res;
+        Long left = null, right = null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (Long) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (Long) res.result;
+
+        if (left < right) {
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+
+    @Override
+    public Result getNext(String s) throws ExecException {
+        byte status;
+        Result res;
+        String left = null, right = null;
+        res = lhs.getNext(left);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        left = (String) res.result;
+
+        res = rhs.getNext(right);
+        status = res.returnStatus;
+        if (status != POStatus.STATUS_OK) {
+
+            return res;
+        }
+        right = (String) res.result;
+
+        int ret = left.compareTo(right);
+        if (ret < 0) {
+            res.result = new Boolean(true);
+            //left = right = null;
+            return res;
+        } else {
+            res.result = new Boolean(false);
+            //left = right = null;
+            return res;
+        }
+    }
+}