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/07/03 23:12:10 UTC

svn commit: r673806 [1/3] - in /incubator/pig/branches/types: src/org/apache/pig/ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/impl/logicalLayer/parser/ src/org/apache/pig/impl/mapReduceLayer/ src/org/apache/pig/impl/physicalLayer/expressio...

Author: gates
Date: Thu Jul  3 14:12:08 2008
New Revision: 673806

URL: http://svn.apache.org/viewvc?rev=673806&view=rev
Log:
PIG-161  Fixed filters.  Reworked BinaryComparisonOperators so they use getNext correctly.


Modified:
    incubator/pig/branches/types/src/org/apache/pig/PigServer.java
    incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LORegexp.java
    incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java
    incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt
    incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/BinaryComparisonOperator.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/EqualToExpr.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/GTOrEqualToExpr.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/GreaterThanExpr.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/LTOrEqualToExpr.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/LessThanExpr.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/NotEqualToExpr.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POAnd.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POBinCond.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/PONot.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POOr.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/PORegexp.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/plans/PhyPlanVisitor.java
    incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/relationalOperators/POFilter.java
    incubator/pig/branches/types/src/org/apache/pig/impl/plan/OperatorPlan.java
    incubator/pig/branches/types/src/org/apache/pig/tools/grunt/Grunt.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestEqualTo.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestFilter.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
    incubator/pig/branches/types/test/org/apache/pig/test/TestPOBinCond.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestRegexp.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestTypeCheckingValidator.java
    incubator/pig/branches/types/test/org/apache/pig/test/utils/GenPhyOp.java

Modified: incubator/pig/branches/types/src/org/apache/pig/PigServer.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/PigServer.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/PigServer.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/PigServer.java Thu Jul  3 14:12:08 2008
@@ -56,7 +56,6 @@
 import org.apache.pig.impl.logicalLayer.parser.QueryParser;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.logicalLayer.validators.LogicalPlanValidationExecutor;
-import org.apache.pig.impl.physicalLayer.POPrinter;
 import org.apache.pig.impl.physicalLayer.plans.PhysicalPlan;
 import org.apache.pig.impl.plan.CompilationMessageCollector;
 import org.apache.pig.impl.plan.OperatorKey;
@@ -338,8 +337,6 @@
             PhysicalPlan pp = compilePp(lp);
             stream.println("-----------------------------------------------");
             stream.println("Physical Plan:");
-            POPrinter pv = new POPrinter(stream, pp);
-            pv.visit();
 
             stream.println("-----------------------------------------------");
             pigContext.getExecutionEngine().explain(pp, stream);

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LORegexp.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LORegexp.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LORegexp.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LORegexp.java Thu Jul  3 14:12:08 2008
@@ -49,8 +49,8 @@
      *            regular expression to match
      */
     public LORegexp(LogicalPlan plan, OperatorKey key,
-            ExpressionOperator operand, String regexp) {
-        super(plan, key, operand, new LOConst(plan, key, regexp));
+            ExpressionOperator operand, ExpressionOperator regexp) {
+        super(plan, key, operand, regexp);
     }
 
     public ExpressionOperator getOperand() {
@@ -84,7 +84,7 @@
 
     @Override
     public boolean supportsMultipleInputs() {
-        return false;
+        return true;
     }
 
     @Override

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/LogToPhyTranslationVisitor.java Thu Jul  3 14:12:08 2008
@@ -256,9 +256,11 @@
     @Override
     public void visit(LORegexp op) throws VisitorException {
         String scope = op.getOperatorKey().scope;
-        ExpressionOperator exprOp = new PORegexp(new OperatorKey(scope, nodeGen
-                .getNextNodeId(scope)), op.getRequestedParallelism());
-        exprOp.setResultType(op.getLhsOperand().getType());
+        BinaryComparisonOperator exprOp =
+            new PORegexp(new OperatorKey(scope, nodeGen.getNextNodeId(scope)),
+            op.getRequestedParallelism());
+        exprOp.setLhs((ExpressionOperator)LogToPhyMap.get(op.getLhsOperand()));
+        exprOp.setRhs((ExpressionOperator)LogToPhyMap.get(op.getRhsOperand()));
         LogicalPlan lp = op.mPlan;
 
         currentPlan.add(exprOp);
@@ -416,8 +418,9 @@
     @Override
     public void visit(LOAnd op) throws VisitorException {
         String scope = op.getOperatorKey().scope;
-        ExpressionOperator exprOp = new POAnd(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
-        exprOp.setResultType(DataType.BOOLEAN);
+        BinaryComparisonOperator exprOp = new POAnd(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
+        exprOp.setLhs((ExpressionOperator)LogToPhyMap.get(op.getLhsOperand()));
+        exprOp.setRhs((ExpressionOperator)LogToPhyMap.get(op.getRhsOperand()));
         LogicalPlan lp = op.mPlan;
         
         currentPlan.add(exprOp);
@@ -438,8 +441,9 @@
     @Override
     public void visit(LOOr op) throws VisitorException {
         String scope = op.getOperatorKey().scope;
-        ExpressionOperator exprOp = new POOr(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
-        exprOp.setResultType(DataType.BOOLEAN);
+        BinaryComparisonOperator exprOp = new POOr(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
+        exprOp.setLhs((ExpressionOperator)LogToPhyMap.get(op.getLhsOperand()));
+        exprOp.setRhs((ExpressionOperator)LogToPhyMap.get(op.getRhsOperand()));
         LogicalPlan lp = op.mPlan;
         
         currentPlan.add(exprOp);
@@ -460,8 +464,8 @@
     @Override
     public void visit(LONot op) throws VisitorException {
         String scope = op.getOperatorKey().scope;
-        ExpressionOperator exprOp = new PONot(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
-        exprOp.setResultType(DataType.BOOLEAN);
+        UnaryComparisonOperator exprOp = new PONot(new OperatorKey(scope, nodeGen.getNextNodeId(scope)), op.getRequestedParallelism());
+        exprOp.setExpr((ExpressionOperator)LogToPhyMap.get(op.getOperand()));
         LogicalPlan lp = op.mPlan;
         
         currentPlan.add(exprOp);
@@ -656,53 +660,6 @@
 
     }
 
-    /*
-    @Override
-    public void visit(LOGenerate g) throws VisitorException {
-        boolean currentPhysicalPlan = false;
-        String scope = g.getOperatorKey().scope;
-        List<PhysicalPlan> exprPlans = new ArrayList<PhysicalPlan>();
-        List<LogicalPlan> plans = g.getGeneratePlans();
-
-        currentPlans.push(currentPlan);
-        for (LogicalPlan plan : plans) {
-            currentPlan = new PhysicalPlan();
-            PlanWalker<LogicalOperator, LogicalPlan> childWalker = mCurrentWalker
-                    .spawnChildWalker(plan);
-            pushWalker(childWalker);
-            childWalker.walk(this);
-            exprPlans.add((PhysicalPlan) currentPlan);
-            popWalker();
-        }
-        currentPlan = currentPlans.pop();
-
-        // PhysicalOperator poGen = new POGenerate(new OperatorKey("",
-        // r.nextLong()), inputs, toBeFlattened);
-        PhysicalOperator poGen = new POGenerate(new OperatorKey(scope, nodeGen
-                .getNextNodeId(scope)), g.getRequestedParallelism(), exprPlans,
-                g.getFlatten());
-        poGen.setResultType(DataType.TUPLE);
-        LogToPhyMap.put(g, poGen);
-        currentPlan.add(poGen);
-
-        // generate cannot have multiple inputs
-        List<LogicalOperator> op = g.getPlan().getPredecessors(g);
-
-        // generate may not have any predecessors
-        if (op == null)
-            return;
-
-        PhysicalOperator from = LogToPhyMap.get(op.get(0));
-        try {
-            currentPlan.connect(from, poGen);
-        } catch (PlanException e) {
-            log.error("Invalid physical operators in the physical plan"
-                    + e.getMessage());
-        }
-
-    }
-    */
-
     @Override
     public void visit(LOSort s) throws VisitorException {
         String scope = s.getOperatorKey().scope;

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/logicalLayer/parser/QueryParser.jjt Thu Jul  3 14:12:08 2008
@@ -924,10 +924,13 @@
 |	LOOKAHEAD(InfixExpr(over,specs,lp,input) <MATCHES>) 
 		(lhs=InfixExpr(over,specs,lp,input) <MATCHES> t1=<QUOTEDSTRING> 
 			{
-				cond = new LORegexp(lp, new OperatorKey(scope, getNextId()), lhs, unquote(t1.image)); 
+                LOConst rconst = new LOConst(lp, new OperatorKey(scope, getNextId()), unquote(t1.image));
+				cond = new LORegexp(lp, new OperatorKey(scope, getNextId()), lhs, rconst); 
+				lp.add(rconst); 
 				lp.add(cond); 
 				log.debug("PUnaryCond: Added operator " + cond.getClass().getName() + " " + cond + " to logical plan " + lp);
 				lp.connect(lhs, cond);
+				lp.connect(rconst, cond);
 				log.debug("PUnaryCond: Connected operator " + cond.getClass().getName() + " " + cond + " to " + lhs + " logical plan " + lp);
 			}
 		)

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java Thu Jul  3 14:12:08 2008
@@ -400,7 +400,10 @@
     
     
     private MapReduceOper endSingleInputPlanWithStr(FileSpec fSpec) throws PlanException{
-        if(compiledInputs.length>1) throw new PlanException("Received a multi input plan when expecting only a single input one.");
+        if(compiledInputs.length>1) {
+            log.error("Received a multi input plan when expecting only a single input one.");
+            throw new PlanException("Received a multi input plan when expecting only a single input one.");
+        }
         MapReduceOper mro = compiledInputs[0];
         POStore str = getStore();
         str.setSFile(fSpec);
@@ -751,8 +754,10 @@
         
         List<PhysicalPlan> eps1 = new ArrayList<PhysicalPlan>();
         
-        if(fields==null)
+        if(fields==null) {
+            log.error("No Expression Plan found in POSort");
             throw new PlanException("No Expression Plan found in POSort");
+        }
         for (int i : fields) {
             PhysicalPlan ep = new PhysicalPlan();
             POProject prj = new POProject(new OperatorKey(scope,nig.getNextNodeId(scope)));
@@ -809,8 +814,10 @@
         List<PhysicalPlan> eps1 = new ArrayList<PhysicalPlan>();
         List<Boolean> flat1 = new ArrayList<Boolean>();
         
-        if(fields==null)
+        if(fields==null) {
+            log.error("No Expression Plan found in POSort");
             throw new PlanException("No Expression Plan found in POSort");
+        }
         for (int i : fields) {
             PhysicalPlan ep = new PhysicalPlan();
             POProject prj = new POProject(new OperatorKey(scope,nig.getNextNodeId(scope)));

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/BinaryComparisonOperator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/BinaryComparisonOperator.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/BinaryComparisonOperator.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/BinaryComparisonOperator.java Thu Jul  3 14:12:08 2008
@@ -37,6 +37,12 @@
     //The result will be comunicated using the Status object.
     //This is a slight abuse of the status object.
     protected byte operandType;
+
+    // Default instances of true and false, used so that all the equality
+    // operators don't have to instantiate a true or false object on each
+    // test.
+    protected Boolean trueRef;
+    protected Boolean falseRef;
     
     public BinaryComparisonOperator(OperatorKey k) {
         this(k,-1);
@@ -53,4 +59,11 @@
     public void setOperandType(byte operandType) {
         this.operandType = operandType;
     }
+
+    // Necessary because the objects are serialized, not constructed on the
+    // other side.
+    protected void initializeRefs() {
+        trueRef = new Boolean(true);
+        falseRef = new Boolean(false);
+    }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/EqualToExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/EqualToExpr.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/EqualToExpr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/EqualToExpr.java Thu Jul  3 14:12:08 2008
@@ -17,20 +17,16 @@
  */
 package org.apache.pig.impl.physicalLayer.expressionOperators;
 
-import java.util.Map;
-
 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.plan.OperatorKey;
+import org.apache.pig.impl.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
-import org.apache.pig.impl.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.backend.executionengine.ExecException;
 
 public class EqualToExpr extends BinaryComparisonOperator {
 
@@ -46,11 +42,7 @@
 
     public EqualToExpr(OperatorKey k, int rp) {
         super(k, rp);
-    }
-
-    @Override
-    public void visit(PhyPlanVisitor v) throws VisitorException {
-        v.visitEqualTo(this);
+        resultType = DataType.BOOLEAN;
     }
 
     @Override
@@ -59,367 +51,70 @@
     }
 
     @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.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(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.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(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.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(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.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(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;
-        }
+    public void visit(PhyPlanVisitor v) throws VisitorException {
+        v.visitEqualTo(this);
     }
 
     @Override
-    public Result getNext(Map inp) throws ExecException {
+    public Result getNext(Boolean bool) throws ExecException {
         byte status;
-        Result res;
-
-        Map left = null, right = null;
+        Result left, right;
 
-        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;
+        switch (operandType) {
+        case DataType.BYTEARRAY: {
+            left = lhs.getNext(dummyDBA);
+            right = rhs.getNext(dummyDBA);
+            return doComparison(left, right);
+                            }
+
+        case DataType.DOUBLE: {
+            left = lhs.getNext(dummyDouble);
+            right = rhs.getNext(dummyDouble);
+            return doComparison(left, right);
+                            }
+
+        case DataType.FLOAT: {
+            left = lhs.getNext(dummyFloat);
+            right = rhs.getNext(dummyFloat);
+            return doComparison(left, right);
+                            }
+
+        case DataType.INTEGER: {
+            left = lhs.getNext(dummyInt);
+            right = rhs.getNext(dummyInt);
+            return doComparison(left, right);
+                            }
+
+        case DataType.LONG: {
+            left = lhs.getNext(dummyLong);
+            right = rhs.getNext(dummyLong);
+            return doComparison(left, right);
+                            }
+
+        case DataType.CHARARRAY: {
+            left = lhs.getNext(dummyString);
+            right = rhs.getNext(dummyString);
+            return doComparison(left, right);
+                            }
+
+
+        default:
+            throw new RuntimeException("Equal to doesn't know how to " +
+                "handle type " + DataType.findTypeName(operandType));
         }
     }
 
-    @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;
+    private Result doComparison(Result left, Result right) {
+        if (trueRef == null) initializeRefs();
+        if (left.returnStatus != POStatus.STATUS_OK) return left;
+        if (right.returnStatus != POStatus.STATUS_OK) return right;
+        assert(left instanceof Comparable);
+        assert(right instanceof Comparable);
+        if (((Comparable)left.result).compareTo((Comparable)right.result) == 0) {
+            left.result = trueRef;
         } else {
-
-            res.result = new Boolean(false);
-            //left = right = null;
-            return res;
+            left.result = falseRef;
         }
+        return left;
     }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/GTOrEqualToExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/GTOrEqualToExpr.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/GTOrEqualToExpr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/GTOrEqualToExpr.java Thu Jul  3 14:12:08 2008
@@ -22,13 +22,12 @@
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.DataType;
 import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
-import org.apache.pig.impl.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.backend.executionengine.ExecException;
 
-
 public class GTOrEqualToExpr extends BinaryComparisonOperator {
 
     /**
@@ -36,233 +35,86 @@
      */
     private static final long serialVersionUID = 1L;
     private final Log log = LogFactory.getLog(getClass());
-    
+
     public GTOrEqualToExpr(OperatorKey k) {
-        this(k,-1);
+        this(k, -1);
     }
 
     public GTOrEqualToExpr(OperatorKey k, int rp) {
         super(k, rp);
-    }
-
-    @Override
-    public void visit(PhyPlanVisitor v) throws VisitorException {
-        v.visitGTOrEqual(this);
+        resultType = DataType.BOOLEAN;
     }
 
     @Override
     public String name() {
         return "Greater Than or Equal" + "[" + DataType.findTypeName(resultType) + "]" +" - " + 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;
-        }
+    public void visit(PhyPlanVisitor v) throws VisitorException {
+        v.visitGTOrEqual(this);
     }
 
     @Override
-    public Result getNext(Integer i) throws ExecException {
+    public Result getNext(Boolean bool) 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;
-        }
-    }
+        Result left, right;
 
-    @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;
+        switch (operandType) {
+        case DataType.BYTEARRAY: {
+            left = lhs.getNext(dummyDBA);
+            right = rhs.getNext(dummyDBA);
+            return doComparison(left, right);
+                            }
+
+        case DataType.DOUBLE: {
+            left = lhs.getNext(dummyDouble);
+            right = rhs.getNext(dummyDouble);
+            return doComparison(left, right);
+                            }
+
+        case DataType.FLOAT: {
+            left = lhs.getNext(dummyFloat);
+            right = rhs.getNext(dummyFloat);
+            return doComparison(left, right);
+                            }
+
+        case DataType.INTEGER: {
+            left = lhs.getNext(dummyInt);
+            right = rhs.getNext(dummyInt);
+            return doComparison(left, right);
+                            }
+
+        case DataType.LONG: {
+            left = lhs.getNext(dummyLong);
+            right = rhs.getNext(dummyLong);
+            return doComparison(left, right);
+                            }
+
+        case DataType.CHARARRAY: {
+            left = lhs.getNext(dummyString);
+            right = rhs.getNext(dummyString);
+            return doComparison(left, right);
+                            }
+
+
+        default:
+            throw new RuntimeException("Greater than or equal to doesn't know how to " +
+                "handle type " + DataType.findTypeName(operandType));
         }
     }
 
-    @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;
+    private Result doComparison(Result left, Result right) {
+        if (trueRef == null) initializeRefs();
+        if (left.returnStatus != POStatus.STATUS_OK) return left;
+        if (right.returnStatus != POStatus.STATUS_OK) return right;
+        assert(left instanceof Comparable);
+        assert(right instanceof Comparable);
+        if (((Comparable)left.result).compareTo((Comparable)right.result) >= 0) {
+            left.result = trueRef;
+        } else {
+            left.result = falseRef;
         }
+        return left;
     }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/GreaterThanExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/GreaterThanExpr.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/GreaterThanExpr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/GreaterThanExpr.java Thu Jul  3 14:12:08 2008
@@ -42,6 +42,7 @@
 
     public GreaterThanExpr(OperatorKey k, int rp) {
         super(k, rp);
+        resultType = DataType.BOOLEAN;
     }
 
     @Override
@@ -55,197 +56,65 @@
     }
 
     @Override
-    public Result getNext(DataByteArray ba) throws ExecException {
+    public Result getNext(Boolean bool) throws ExecException {
         byte status;
-        Result res;
-        DataByteArray left = null, right = null;
-        res = lhs.getNext(left);
-        status = res.returnStatus;
-        if (status != POStatus.STATUS_OK) {
+        Result left, right;
 
-            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;
+        switch (operandType) {
+        case DataType.BYTEARRAY: {
+            left = lhs.getNext(dummyDBA);
+            right = rhs.getNext(dummyDBA);
+            return doComparison(left, right);
+                            }
+
+        case DataType.DOUBLE: {
+            left = lhs.getNext(dummyDouble);
+            right = rhs.getNext(dummyDouble);
+            return doComparison(left, right);
+                            }
+
+        case DataType.FLOAT: {
+            left = lhs.getNext(dummyFloat);
+            right = rhs.getNext(dummyFloat);
+            return doComparison(left, right);
+                            }
+
+        case DataType.INTEGER: {
+            left = lhs.getNext(dummyInt);
+            right = rhs.getNext(dummyInt);
+            return doComparison(left, right);
+                            }
+
+        case DataType.LONG: {
+            left = lhs.getNext(dummyLong);
+            right = rhs.getNext(dummyLong);
+            return doComparison(left, right);
+                            }
+
+        case DataType.CHARARRAY: {
+            left = lhs.getNext(dummyString);
+            right = rhs.getNext(dummyString);
+            return doComparison(left, right);
+                            }
+
+
+        default:
+            throw new RuntimeException("Greater than doesn't know how to " +
+                "handle type " + DataType.findTypeName(operandType));
         }
     }
 
-    @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;
+    private Result doComparison(Result left, Result right) {
+        if (trueRef == null) initializeRefs();
+        if (left.returnStatus != POStatus.STATUS_OK) return left;
+        if (right.returnStatus != POStatus.STATUS_OK) return right;
+        assert(left instanceof Comparable);
+        assert(right instanceof Comparable);
+        if (((Comparable)left.result).compareTo((Comparable)right.result) > 0) {
+            left.result = trueRef;
         } else {
-            res.result = new Boolean(false);
-            //left = right = null;
-            return res;
+            left.result = falseRef;
         }
+        return left;
     }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/LTOrEqualToExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/LTOrEqualToExpr.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/LTOrEqualToExpr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/LTOrEqualToExpr.java Thu Jul  3 14:12:08 2008
@@ -22,13 +22,14 @@
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.DataType;
 import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
-import org.apache.pig.impl.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.backend.executionengine.ExecException;
 
 public class LTOrEqualToExpr extends BinaryComparisonOperator {
+
     /**
      * 
      */
@@ -41,11 +42,7 @@
 
     public LTOrEqualToExpr(OperatorKey k, int rp) {
         super(k, rp);
-    }
-
-    @Override
-    public void visit(PhyPlanVisitor v) throws VisitorException {
-        v.visiLTOrEqual(this);
+        resultType = DataType.BOOLEAN;
     }
 
     @Override
@@ -54,196 +51,70 @@
     }
 
     @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;
-        }
+    public void visit(PhyPlanVisitor v) throws VisitorException {
+        v.visitLTOrEqual(this);
     }
 
     @Override
-    public Result getNext(Integer i) throws ExecException {
+    public Result getNext(Boolean bool) throws ExecException {
         byte status;
-        Result res;
-        Integer left = null, right = null;
-        res = lhs.getNext(left);
-        status = res.returnStatus;
-        if (status != POStatus.STATUS_OK) {
+        Result left, right;
 
-            return res;
-        }
-        left = (Integer) res.result;
+        switch (operandType) {
+        case DataType.BYTEARRAY: {
+            left = lhs.getNext(dummyDBA);
+            right = rhs.getNext(dummyDBA);
+            return doComparison(left, right);
+                            }
 
-        res = rhs.getNext(right);
-        status = res.returnStatus;
-        if (status != POStatus.STATUS_OK) {
+        case DataType.DOUBLE: {
+            left = lhs.getNext(dummyDouble);
+            right = rhs.getNext(dummyDouble);
+            return doComparison(left, right);
+                            }
 
-            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;
-        }
-    }
+        case DataType.FLOAT: {
+            left = lhs.getNext(dummyFloat);
+            right = rhs.getNext(dummyFloat);
+            return doComparison(left, right);
+                            }
 
-    @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) {
+        case DataType.INTEGER: {
+            left = lhs.getNext(dummyInt);
+            right = rhs.getNext(dummyInt);
+            return doComparison(left, right);
+                            }
 
-            return res;
-        }
-        left = (Long) res.result;
+        case DataType.LONG: {
+            left = lhs.getNext(dummyLong);
+            right = rhs.getNext(dummyLong);
+            return doComparison(left, right);
+                            }
 
-        res = rhs.getNext(right);
-        status = res.returnStatus;
-        if (status != POStatus.STATUS_OK) {
+        case DataType.CHARARRAY: {
+            left = lhs.getNext(dummyString);
+            right = rhs.getNext(dummyString);
+            return doComparison(left, right);
+                            }
 
-            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;
+        default:
+            throw new RuntimeException("Less than doesn't know how to " +
+                "handle type " + DataType.findTypeName(operandType));
         }
     }
 
-    @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;
+    private Result doComparison(Result left, Result right) {
+        if (trueRef == null) initializeRefs();
+        if (left.returnStatus != POStatus.STATUS_OK) return left;
+        if (right.returnStatus != POStatus.STATUS_OK) return right;
+        assert(left instanceof Comparable);
+        assert(right instanceof Comparable);
+        if (((Comparable)left.result).compareTo((Comparable)right.result) <= 0) {
+            left.result = trueRef;
         } else {
-            res.result = new Boolean(false);
-            //left = right = null;
-            return res;
+            left.result = falseRef;
         }
+        return left;
     }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/LessThanExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/LessThanExpr.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/LessThanExpr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/LessThanExpr.java Thu Jul  3 14:12:08 2008
@@ -22,9 +22,9 @@
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.DataType;
 import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
-import org.apache.pig.impl.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.backend.executionengine.ExecException;
 
@@ -42,11 +42,7 @@
 
     public LessThanExpr(OperatorKey k, int rp) {
         super(k, rp);
-    }
-
-    @Override
-    public void visit(PhyPlanVisitor v) throws VisitorException {
-        v.visitLessThan(this);
+        resultType = DataType.BOOLEAN;
     }
 
     @Override
@@ -55,196 +51,70 @@
     }
 
     @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;
-        }
+    public void visit(PhyPlanVisitor v) throws VisitorException {
+        v.visitLessThan(this);
     }
 
     @Override
-    public Result getNext(Long l) throws ExecException {
+    public Result getNext(Boolean bool) 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;
+        Result left, right;
 
-        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;
+        switch (operandType) {
+        case DataType.BYTEARRAY: {
+            left = lhs.getNext(dummyDBA);
+            right = rhs.getNext(dummyDBA);
+            return doComparison(left, right);
+                            }
+
+        case DataType.DOUBLE: {
+            left = lhs.getNext(dummyDouble);
+            right = rhs.getNext(dummyDouble);
+            return doComparison(left, right);
+                            }
+
+        case DataType.FLOAT: {
+            left = lhs.getNext(dummyFloat);
+            right = rhs.getNext(dummyFloat);
+            return doComparison(left, right);
+                            }
+
+        case DataType.INTEGER: {
+            left = lhs.getNext(dummyInt);
+            right = rhs.getNext(dummyInt);
+            return doComparison(left, right);
+                            }
+
+        case DataType.LONG: {
+            left = lhs.getNext(dummyLong);
+            right = rhs.getNext(dummyLong);
+            return doComparison(left, right);
+                            }
+
+        case DataType.CHARARRAY: {
+            left = lhs.getNext(dummyString);
+            right = rhs.getNext(dummyString);
+            return doComparison(left, right);
+                            }
+
+
+        default:
+            throw new RuntimeException("Less than doesn't know how to " +
+                "handle type " + DataType.findTypeName(operandType));
         }
     }
 
-    @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;
+    private Result doComparison(Result left, Result right) {
+        if (trueRef == null) initializeRefs();
+        if (left.returnStatus != POStatus.STATUS_OK) return left;
+        if (right.returnStatus != POStatus.STATUS_OK) return right;
+        assert(left instanceof Comparable);
+        assert(right instanceof Comparable);
+        if (((Comparable)left.result).compareTo((Comparable)right.result) < 0) {
+            left.result = trueRef;
         } else {
-            res.result = new Boolean(false);
-            //left = right = null;
-            return res;
+            left.result = falseRef;
         }
+        return left;
     }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/NotEqualToExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/NotEqualToExpr.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/NotEqualToExpr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/NotEqualToExpr.java Thu Jul  3 14:12:08 2008
@@ -17,20 +17,16 @@
  */
 package org.apache.pig.impl.physicalLayer.expressionOperators;
 
-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.plan.OperatorKey;
-import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.impl.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
-import org.apache.pig.impl.physicalLayer.plans.PhyPlanVisitor;
 import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.backend.executionengine.ExecException;
 
 public class NotEqualToExpr extends BinaryComparisonOperator {
 
@@ -46,11 +42,7 @@
 
     public NotEqualToExpr(OperatorKey k, int rp) {
         super(k, rp);
-    }
-
-    @Override
-    public void visit(PhyPlanVisitor v) throws VisitorException {
-        v.visitNotEqualTo(this);
+        resultType = DataType.BOOLEAN;
     }
 
     @Override
@@ -59,367 +51,70 @@
     }
 
     @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.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(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.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(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.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(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.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(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;
-        }
+    public void visit(PhyPlanVisitor v) throws VisitorException {
+        v.visitNotEqualTo(this);
     }
 
     @Override
-    public Result getNext(Map inp) throws ExecException {
+    public Result getNext(Boolean bool) throws ExecException {
         byte status;
-        Result res;
-
-        Map left = null, right = null;
+        Result left, right;
 
-        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;
+        switch (operandType) {
+        case DataType.BYTEARRAY: {
+            left = lhs.getNext(dummyDBA);
+            right = rhs.getNext(dummyDBA);
+            return doComparison(left, right);
+                            }
+
+        case DataType.DOUBLE: {
+            left = lhs.getNext(dummyDouble);
+            right = rhs.getNext(dummyDouble);
+            return doComparison(left, right);
+                            }
+
+        case DataType.FLOAT: {
+            left = lhs.getNext(dummyFloat);
+            right = rhs.getNext(dummyFloat);
+            return doComparison(left, right);
+                            }
+
+        case DataType.INTEGER: {
+            left = lhs.getNext(dummyInt);
+            right = rhs.getNext(dummyInt);
+            return doComparison(left, right);
+                            }
+
+        case DataType.LONG: {
+            left = lhs.getNext(dummyLong);
+            right = rhs.getNext(dummyLong);
+            return doComparison(left, right);
+                            }
+
+        case DataType.CHARARRAY: {
+            left = lhs.getNext(dummyString);
+            right = rhs.getNext(dummyString);
+            return doComparison(left, right);
+                            }
+
+
+        default:
+            throw new RuntimeException("Not equal to doesn't know how to " +
+                "handle type " + DataType.findTypeName(operandType));
         }
     }
 
-    @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;
+    private Result doComparison(Result left, Result right) {
+        if (trueRef == null) initializeRefs();
+        if (left.returnStatus != POStatus.STATUS_OK) return left;
+        if (right.returnStatus != POStatus.STATUS_OK) return right;
+        assert(left instanceof Comparable);
+        assert(right instanceof Comparable);
+        if (((Comparable)left.result).compareTo((Comparable)right.result) != 0) {
+            left.result = trueRef;
         } else {
-
-            res.result = new Boolean(false);
-            //left = right = null;
-            return res;
+            left.result = falseRef;
         }
+        return left;
     }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POAnd.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POAnd.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POAnd.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POAnd.java Thu Jul  3 14:12:08 2008
@@ -37,11 +37,12 @@
     private static final long serialVersionUID = 1L;
 
     public POAnd(OperatorKey k) {
-        super(k);
+        this(k, -1);
     }
 
     public POAnd(OperatorKey k, int rp) {
         super(k, rp);
+        resultType = DataType.BOOLEAN;
     }
 
     @Override
@@ -56,26 +57,16 @@
 
     @Override
     public Result getNext(Boolean b) 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;
+        Result left;
+        left = lhs.getNext(dummyBool);
+        if(left.returnStatus != POStatus.STATUS_OK) {
+            return left;
         }
-        left = (Boolean)res.result;
         // Short circuit.
-        if (!left) return res;
+        if (!(((Boolean)left.result).booleanValue())) return left;
         
-        res = rhs.getNext(right);
-        status = res.returnStatus;
-        if(status != POStatus.STATUS_OK) {
-            return res;
-        }
-        right = (Boolean)res.result;
-        
-        res.result = new Boolean(left && right);
-        return res;
+        // No matter what, what we get from the right side is what we'll
+        // return, error, null, true, or false.
+        return rhs.getNext(dummyBool);
     }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POBinCond.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POBinCond.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POBinCond.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POBinCond.java Thu Jul  3 14:12:08 2008
@@ -65,55 +65,55 @@
 
     @Override
     public Result getNext(DataBag db) throws ExecException {
-        Result res = cond.getNext(db);
+        Result res = cond.getNext(dummyBool);
         return ((Boolean)res.result) == true ? lhs.getNext(db) : rhs.getNext(db);
     }
 
     @Override
     public Result getNext(DataByteArray ba) throws ExecException {
-        Result res = cond.getNext(ba);
+        Result res = cond.getNext(dummyBool);
         return ((Boolean)res.result) == true ? lhs.getNext(ba) : rhs.getNext(ba);
     }
 
     @Override
     public Result getNext(Double d) throws ExecException {
-        Result res = cond.getNext(d);
+        Result res = cond.getNext(dummyBool);
         return ((Boolean)res.result) == true ? lhs.getNext(d) : rhs.getNext(d);
     }
 
     @Override
     public Result getNext(Float f) throws ExecException {
-        Result res = cond.getNext(f);
+        Result res = cond.getNext(dummyBool);
         return ((Boolean)res.result) == true ? lhs.getNext(f) : rhs.getNext(f);
     }
 
     @Override
     public Result getNext(Integer i) throws ExecException {
-        Result res = cond.getNext(i);
+        Result res = cond.getNext(dummyBool);
         return ((Boolean)res.result) == true ? lhs.getNext(i) : rhs.getNext(i);
     }
 
     @Override
     public Result getNext(Long l) throws ExecException {
-        Result res = cond.getNext(l);
+        Result res = cond.getNext(dummyBool);
         return ((Boolean)res.result) == true ? lhs.getNext(l) : rhs.getNext(l);
     }
 
     @Override
     public Result getNext(Map m) throws ExecException {
-        Result res = cond.getNext(m);
+        Result res = cond.getNext(dummyBool);
         return ((Boolean)res.result) == true ? lhs.getNext(m) : rhs.getNext(m);
     }
 
     @Override
     public Result getNext(String s) throws ExecException {
-        Result res = cond.getNext(s);
+        Result res = cond.getNext(dummyBool);
         return ((Boolean)res.result) == true ? lhs.getNext(s) : rhs.getNext(s);
     }
 
     @Override
     public Result getNext(Tuple t) throws ExecException {
-        Result res = cond.getNext(t);
+        Result res = cond.getNext(dummyBool);
         return ((Boolean)res.result) == true ? lhs.getNext(t) : rhs.getNext(t);
     }
 

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/PONot.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/PONot.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/PONot.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/PONot.java Thu Jul  3 14:12:08 2008
@@ -36,12 +36,15 @@
      */
     private static final long serialVersionUID = 1L;
 
+    private Result trueRes, falseRes;
+
     public PONot(OperatorKey k) {
-        super(k);
+        this(k, -1);
     }
 
     public PONot(OperatorKey k, int rp) {
         super(k, rp);
+        resultType = DataType.BOOLEAN;
     }
 
     @Override
@@ -56,15 +59,19 @@
 
     @Override
     public Result getNext(Boolean b) throws ExecException {
-        byte status;
-        Result res;
-        Boolean dummy = null;
-        res = expr.getNext(dummy);
-        status = res.returnStatus;
-        if(status != POStatus.STATUS_OK) {
+        if (trueRes == null) {
+            trueRes = new Result();
+            trueRes.returnStatus = POStatus.STATUS_OK;
+            trueRes.result = new Boolean(true);
+            falseRes = new Result();
+            falseRes.returnStatus = POStatus.STATUS_OK;
+            falseRes.result = new Boolean(false);
+        }
+        res = expr.getNext(dummyBool);
+        if(res.returnStatus != POStatus.STATUS_OK) {
             return res;
         }
-        res.result = !(Boolean)res.result;
-        return res;
+        if (((Boolean)res.result).booleanValue()) return falseRes;
+        else return trueRes;
     }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POOr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POOr.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POOr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/POOr.java Thu Jul  3 14:12:08 2008
@@ -37,11 +37,12 @@
     private static final long serialVersionUID = 1L;
 
     public POOr(OperatorKey k) {
-        super(k);
+        this(k, -1);
     }
 
     public POOr(OperatorKey k, int rp) {
         super(k, rp);
+        resultType = DataType.BOOLEAN;
     }
 
     @Override
@@ -56,26 +57,16 @@
 
     @Override
     public Result getNext(Boolean b) 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;
+        Result left;
+        left = lhs.getNext(dummyBool);
+        if(left.returnStatus != POStatus.STATUS_OK) {
+            return left;
         }
-        left = (Boolean)res.result;
         // Short circuit.
-        if (left) return res;
+        if (((Boolean)left.result).booleanValue()) return left;
         
-        res = rhs.getNext(right);
-        status = res.returnStatus;
-        if(status != POStatus.STATUS_OK) {
-            return res;
-        }
-        right = (Boolean)res.result;
-        
-        res.result = new Boolean(left || right);
-        return res;
+        // No matter what, what we get from the right side is what we'll
+        // return, error, null, true, or false.
+        return rhs.getNext(dummyBool);
     }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/PORegexp.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/PORegexp.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/PORegexp.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/expressionOperators/PORegexp.java Thu Jul  3 14:12:08 2008
@@ -20,7 +20,7 @@
 import java.util.regex.PatternSyntaxException;
 
 import org.apache.pig.backend.executionengine.ExecException;
-import org.apache.pig.data.DataByteArray;
+import org.apache.pig.data.DataType;
 import org.apache.pig.impl.plan.OperatorKey;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
@@ -40,6 +40,7 @@
 
     public PORegexp(OperatorKey k, int rp) {
         super(k, rp);
+        resultType = DataType.BOOLEAN;
     }
 
     @Override
@@ -53,42 +54,21 @@
     }
 
     @Override
-    public Result getNext(DataByteArray inp) throws ExecException {
-        // TODO, no idea how to take this on.
-        return new Result();
-    }
-
-    @Override
-    public Result getNext(String inp) throws ExecException {
+    public Result getNext(Boolean bool) throws ExecException {
         byte status;
-        Result res;
-
-        String left = null, right = null;
-
-        res = lhs.getNext(left);
-        status = res.returnStatus;
-        if (status != POStatus.STATUS_OK) {
+        Result left, right;
 
-            return res;
-        }
-        left = (String) res.result;
-
-        res = rhs.getNext(right);
-        status = res.returnStatus;
-        if (status != POStatus.STATUS_OK) {
+        left = lhs.getNext(dummyString);
+        right = rhs.getNext(dummyString);
 
-            return res;
-        }
-        right = (String) res.result;
-
-        // left is expression to match against, right is regular expression
-        try {
-            res.result = new Boolean(left.matches(right));
-            return res;
-        } catch (PatternSyntaxException pse) {
-            throw new ExecException("Unable to parse regular expression " +
-                right, pse);
+        if (trueRef == null) initializeRefs();
+        if (left.returnStatus != POStatus.STATUS_OK) return left;
+        if (right.returnStatus != POStatus.STATUS_OK) return right;
+        if (((String)left.result).matches((String)right.result)) {
+            left.result = trueRef;
+        } else {
+            left.result = falseRef;
         }
+        return left;
     }
-
 }

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=673806&r1=673805&r2=673806&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 Thu Jul  3 14:12:08 2008
@@ -159,7 +159,7 @@
         //do nothing
     }
     
-    public void visiLTOrEqual(LTOrEqualToExpr lte) throws VisitorException{
+    public void visitLTOrEqual(LTOrEqualToExpr lte) throws VisitorException{
         //do nothing
     }
     

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/relationalOperators/POFilter.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/relationalOperators/POFilter.java?rev=673806&r1=673805&r2=673806&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/relationalOperators/POFilter.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/relationalOperators/POFilter.java Thu Jul  3 14:12:08 2008
@@ -99,7 +99,13 @@
 
             plan.attachInput((Tuple) inp.result);
 
+            /*
             switch (compOperandType) {
+            case DataType.BOOLEAN:
+                res = comOp.getNext(dummyBool);
+                if (res.returnStatus != POStatus.STATUS_OK)
+                    continue;
+                break;
             case DataType.BYTEARRAY:
                 res = comOp.getNext(dummyDBA);
                 if (res.returnStatus != POStatus.STATUS_OK)
@@ -130,11 +136,15 @@
                 if (res.returnStatus != POStatus.STATUS_OK)
                     continue;
                 break;
-            }
 
-            if (res == null) {
-                return new Result();
+            default:
+                throw new RuntimeException("Unexpected type " +
+                    DataType.findTypeName(compOperandType));
             }
+            */
+            res = comOp.getNext(dummyBool);
+            if (res.returnStatus != POStatus.STATUS_OK) return res;
+
             if ((Boolean) res.result == true) {
                 return inp;
             }