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/02 23:23:38 UTC

svn commit: r644076 [2/3] - in /incubator/pig/branches/types: ./ src/org/apache/pig/impl/physicalLayer/ src/org/apache/pig/impl/physicalLayer/plans/ src/org/apache/pig/impl/physicalLayer/topLevelOperators/ src/org/apache/pig/impl/physicalLayer/topLevel...

Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Subtract.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Subtract.java?rev=644076&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Subtract.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Subtract.java Wed Apr  2 14:23:36 2008
@@ -0,0 +1,139 @@
+/*
+ * 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.arithmeticOperators;
+
+import org.apache.pig.backend.executionengine.ExecException;
+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;
+
+public class Subtract extends ArithmeticOperator {
+
+    public Subtract(OperatorKey k) {
+        super(k);
+    }
+
+    public Subtract(OperatorKey k, int rp) {
+        super(k, rp);
+    }
+
+    @Override
+    public void visit(ExprPlanVisitor v) throws ParseException {
+        v.visitSubtract(this);
+    }
+
+    @Override
+    public String name() {
+        return "Subtract - " + mKey.toString();
+    }
+
+    @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;
+        
+        res.result = new Double(left - right);
+        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;
+        
+        res.result = new Float(left - right);
+        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;
+        
+        res.result = new Integer(left - right);
+        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;
+        
+        res.result = new Long(left - right);
+        return res;
+    }
+
+}

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/ComparisonOperator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/ComparisonOperator.java?rev=644076&r1=644075&r2=644076&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/ComparisonOperator.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/ComparisonOperator.java Wed Apr  2 14:23:36 2008
@@ -1,3 +1,20 @@
+/*
+ * 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.pig.data.DataByteArray;
@@ -17,26 +34,26 @@
  *
  */
 public abstract class ComparisonOperator extends BinaryExpressionOperator {
-	//The result type for comparison operators is always
-	//Boolean. So the plans evaluating these should consider
-	//the type of the operands instead of the result.
-	//The result will be comunicated using the Status object.
-	//This is a slight abuse of the status object.
-	protected byte operandType;
-	
-	public ComparisonOperator(OperatorKey k) {
-		this(k,-1);
-	}
+    //The result type for comparison operators is always
+    //Boolean. So the plans evaluating these should consider
+    //the type of the operands instead of the result.
+    //The result will be comunicated using the Status object.
+    //This is a slight abuse of the status object.
+    protected byte operandType;
+    
+    public ComparisonOperator(OperatorKey k) {
+        this(k,-1);
+    }
 
-	public ComparisonOperator(OperatorKey k, int rp) {
-		super(k, rp);
-	}
+    public ComparisonOperator(OperatorKey k, int rp) {
+        super(k, rp);
+    }
 
-	public byte getOperandType() {
-		return operandType;
-	}
+    public byte getOperandType() {
+        return operandType;
+    }
 
-	public void setOperandType(byte operandType) {
-		this.operandType = operandType;
-	}
+    public void setOperandType(byte operandType) {
+        this.operandType = operandType;
+    }
 }

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=644076&r1=644075&r2=644076&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 Wed Apr  2 14:23:36 2008
@@ -1,3 +1,20 @@
+/*
+ * 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;
@@ -13,248 +30,248 @@
 
 public class GreaterThanExpr extends ComparisonOperator {
 
-	private final Log log = LogFactory.getLog(getClass());
-	
-	public GreaterThanExpr(OperatorKey k) {
-		this(k,-1);
-	}
-
-	public GreaterThanExpr(OperatorKey k, int rp) {
-		super(k, rp);
-	}
-
-	@Override
-	public String name() {
-		return "Greater Than - " + mKey.toString();
-	}
-
-	@Override
-	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;
-		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;
-			return res;
-		}
-		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;
-			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) { 
-			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;
-			return res;
-		}
-		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;
-			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) { 
-			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;
-			return res;
-		}
-		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;
-			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) { 
-			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;
-			return res;
-		}
-		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;
-			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) { 
-			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;
-			return res;
-		}
-		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;
-			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) { 
-			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;
-			return res;
-		}
-		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;
-			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 final Log log = LogFactory.getLog(getClass());
+    
+    public GreaterThanExpr(OperatorKey k) {
+        this(k,-1);
+    }
+
+    public GreaterThanExpr(OperatorKey k, int rp) {
+        super(k, rp);
+    }
+
+    @Override
+    public String name() {
+        return "Greater Than - " + mKey.toString();
+    }
+
+    @Override
+    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;
+        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;
+            return res;
+        }
+        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;
+            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) { 
+            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;
+            return res;
+        }
+        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;
+            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) { 
+            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;
+            return res;
+        }
+        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;
+            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) { 
+            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;
+            return res;
+        }
+        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;
+            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) { 
+            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;
+            return res;
+        }
+        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;
+            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) { 
+            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;
+            return res;
+        }
+        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;
+            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/util/operatorHelper.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/util/operatorHelper.java?rev=644076&r1=644075&r2=644076&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/util/operatorHelper.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/util/operatorHelper.java Wed Apr  2 14:23:36 2008
@@ -1,3 +1,20 @@
+/*
+ * 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.util;
 
 import java.util.HashMap;
@@ -6,29 +23,29 @@
 import org.apache.pig.data.DataType;
 
 public class operatorHelper {
-	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;
-	}
+    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;
+    }
 }

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestAdd.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestAdd.java?rev=644076&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestAdd.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestAdd.java Wed Apr  2 14:23:36 2008
@@ -0,0 +1,167 @@
+/*
+ * 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.test;
+
+
+import java.util.Map;
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+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.physicalLayer.POStatus;
+import org.apache.pig.impl.physicalLayer.Result;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.ConstantExpression;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.arithmeticOperators.Add;
+import org.apache.pig.test.utils.GenRandomData;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class TestAdd extends TestCase{
+
+    Random r = new Random();
+    ConstantExpression lt, rt;
+    Add op = new Add(new OperatorKey("", r.nextLong()));
+
+    @Before
+    public void setUp() throws Exception {
+        lt = new ConstantExpression(new OperatorKey("",r.nextLong()));
+        rt = new ConstantExpression(new OperatorKey("",r.nextLong()));
+    }
+
+    @Test
+    public void testOperator() throws ExecException{
+        //int TRIALS = 10;
+        byte[] types = { DataType.BAG, DataType.BOOLEAN, DataType.BYTEARRAY, DataType.CHARARRAY, 
+                DataType.DOUBLE, DataType.FLOAT, DataType.INTEGER, DataType.LONG, DataType.MAP, DataType.TUPLE};
+        //Map<Byte,String> map = GenRandomData.genTypeToNameMap();
+        System.out.println("Testing Add operator");
+        for(byte type : types) {
+            lt.setResultType(type);
+            rt.setResultType(type);
+            op.setLhs(lt);
+            op.setRhs(rt);
+
+            switch(type){
+            case DataType.BAG:
+                DataBag inpdb1 = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
+                DataBag inpdb2 = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
+                lt.setValue(inpdb1);
+                rt.setValue(inpdb2);
+                Result resdb = op.getNext(inpdb1);
+                assertEquals(resdb.returnStatus, POStatus.STATUS_ERR);
+                break;
+            case DataType.BOOLEAN:
+                Boolean inpb1 = r.nextBoolean();
+                Boolean inpb2 = r.nextBoolean();
+                lt.setValue(inpb1);
+                rt.setValue(inpb2);
+                Result resb = op.getNext(inpb1);
+                assertEquals(resb.returnStatus, POStatus.STATUS_ERR);
+                break;
+            case DataType.BYTEARRAY: {
+                DataByteArray inpba1 = GenRandomData.genRandDBA(r);
+                DataByteArray inpba2 = GenRandomData.genRandDBA(r);
+                lt.setValue(inpba1);
+                rt.setValue(inpba2);
+                Result resba = op.getNext(inpba1);
+                //DataByteArray expected = new DataByteArray(inpba1.toString() + inpba2.toString());
+                //assertEquals(expected, (DataByteArray)resba.result);
+                assertEquals(POStatus.STATUS_ERR, resba.returnStatus);
+                break;
+            }
+            case DataType.CHARARRAY: {
+                String inps1 = GenRandomData.genRandString(r);
+                String inps2 = GenRandomData.genRandString(r);
+                lt.setValue(inps1);
+                rt.setValue(inps2);
+                Result ress = op.getNext(inps1);
+                /*String expected = new String(inps1 + inps2);
+                assertEquals(expected, (String)ress.result);*/
+                assertEquals(POStatus.STATUS_ERR, ress.returnStatus);
+                break;
+            }
+            case DataType.DOUBLE: {
+                Double inpd1 = r.nextDouble();
+                Double inpd2 = r.nextDouble();
+                lt.setValue(inpd1);
+                rt.setValue(inpd2);
+                Result resd = op.getNext(inpd1);
+                Double expected = new Double(inpd1 + inpd2);
+                assertEquals(expected, (Double)resd.result);
+                break;
+            }
+            case DataType.FLOAT: {
+                Float inpf1 = r.nextFloat();
+                Float inpf2 = r.nextFloat();
+                lt.setValue(inpf1);
+                rt.setValue(inpf2);
+                Result resf = op.getNext(inpf1);
+                Float expected = new Float(inpf1 + inpf2);
+                assertEquals(expected, (Float)resf.result);
+                break;
+            }
+            case DataType.INTEGER: {
+                Integer inpi1 = r.nextInt();
+                Integer inpi2 = r.nextInt();
+                lt.setValue(inpi1);
+                rt.setValue(inpi2);
+                Result resi = op.getNext(inpi1);
+                Integer expected = new Integer(inpi1 + inpi2);
+                assertEquals(expected, (Integer) resi.result);
+                break;
+            }
+            case DataType.LONG: {
+                Long inpl1 = r.nextLong();
+                Long inpl2 = r.nextLong();
+                lt.setValue(inpl1);
+                rt.setValue(inpl2);
+                Result resl = op.getNext(inpl1);
+                Long expected = new Long(inpl1 + inpl2);
+                assertEquals(expected, (Long)resl.result);
+                break;
+            }
+            case DataType.MAP: {
+                Map<Integer,String> inpm1 = GenRandomData.genRandMap(r, 10);
+                Map<Integer,String> inpm2 = GenRandomData.genRandMap(r, 10);
+                lt.setValue(inpm1);
+                rt.setValue(inpm2);
+                Result resm = op.getNext(inpm1);
+                assertEquals(POStatus.STATUS_ERR, resm.returnStatus);
+                break;
+            }
+            case DataType.TUPLE: {
+                Tuple inpt1 = GenRandomData.genRandSmallBagTuple(r, 10, 100);
+                Tuple inpt2 = GenRandomData.genRandSmallBagTuple(r, 10, 100);
+                lt.setValue(inpt1);
+                rt.setValue(inpt2);
+                Result rest = op.getNext(inpt1);
+                assertEquals(POStatus.STATUS_ERR, rest.returnStatus);
+                break;
+            }
+            }
+        }
+    }
+}
+

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestConstExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestConstExpr.java?rev=644076&r1=644075&r2=644076&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestConstExpr.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestConstExpr.java Wed Apr  2 14:23:36 2008
@@ -1,3 +1,20 @@
+/*
+ * 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.test;
 
 import static org.junit.Assert.*;
@@ -18,105 +35,105 @@
 import org.junit.Test;
 
 public class TestConstExpr {
-	Random r = new Random();
-	ConstantExpression ce = (ConstantExpression) GenPhyOp.exprConst();
-	
-	@Before
-	public void setUp() throws Exception {
-	}
-
-	@After
-	public void tearDown() throws Exception {
-	}
-
-	@Test
-	public void testGetNextInteger() throws ExecException {
-		Integer inp = r.nextInt();
-		ce.setValue(inp);
-		Result resi = ce.getNext(inp);
-		Integer ret = (Integer)resi.result;
-		assertEquals(inp, ret);
-	}
-
-	@Test
-	public void testGetNextLong() throws ExecException {
-		Long inp = r.nextLong();
-		ce.setValue(inp);
-		Result resl = ce.getNext(inp);
-		Long ret = (Long)resl.result;
-		assertEquals(inp, ret);
-	}
-
-	@Test
-	public void testGetNextDouble() throws ExecException {
-		Double inp = r.nextDouble();
-		ce.setValue(inp);
-		Result resd = ce.getNext(inp);
-		Double ret = (Double)resd.result;
-		assertEquals(inp, ret);
-	}
-
-	@Test
-	public void testGetNextFloat() throws ExecException {
-		Float inp = r.nextFloat();
-		ce.setValue(inp);
-		Result resf = ce.getNext(inp);
-		Float ret = (Float)resf.result;
-		assertEquals(inp, ret);
-	}
-
-	@Test
-	public void testGetNextString() throws ExecException {
-		String inp = GenRandomData.genRandString(r);
-		ce.setValue(inp);
-		Result ress = ce.getNext(inp);
-		String ret = (String)ress.result;
-		assertEquals(inp, ret);
-	}
-
-	@Test
-	public void testGetNextDataByteArray() throws ExecException {
-		DataByteArray inp = GenRandomData.genRandDBA(r);
-		ce.setValue(inp);
-		Result resba = ce.getNext(inp);
-		DataByteArray ret = (DataByteArray)resba.result;
-		assertEquals(inp, ret);
-	}
-
-	@Test
-	public void testGetNextMap() throws ExecException {
-		Map<Integer,String> inp = GenRandomData.genRandMap(r, 10);
-		ce.setValue(inp);
-		Result resm = ce.getNext(inp);
-		Map<Integer,String> ret = (Map)resm.result;
-		assertEquals(inp, ret);
-	}
-
-	@Test
-	public void testGetNextBoolean() throws ExecException {
-		Boolean inp = r.nextBoolean();
-		ce.setValue(inp);
-		Result res = ce.getNext(inp);
-		Boolean ret = (Boolean)res.result;
-		assertEquals(inp, ret);
-	}
-
-	@Test
-	public void testGetNextTuple() throws ExecException {
-		Tuple inp = GenRandomData.genRandSmallBagTuple(r, 10, 100);
-		ce.setValue(inp);
-		Result rest = ce.getNext(inp);
-		Tuple ret = (Tuple)rest.result;
-		assertEquals(inp, ret);
-	}
-
-	@Test
-	public void testGetNextDataBag() throws ExecException {
-		DataBag inp = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
-		ce.setValue(inp);
-		Result res = ce.getNext(inp);
-		DataBag ret = (DataBag)res.result;
-		assertEquals(inp, ret);
-	}
+    Random r = new Random();
+    ConstantExpression ce = (ConstantExpression) GenPhyOp.exprConst();
+    
+    @Before
+    public void setUp() throws Exception {
+    }
+
+    @After
+    public void tearDown() throws Exception {
+    }
+
+    @Test
+    public void testGetNextInteger() throws ExecException {
+        Integer inp = r.nextInt();
+        ce.setValue(inp);
+        Result resi = ce.getNext(inp);
+        Integer ret = (Integer)resi.result;
+        assertEquals(inp, ret);
+    }
+
+    @Test
+    public void testGetNextLong() throws ExecException {
+        Long inp = r.nextLong();
+        ce.setValue(inp);
+        Result resl = ce.getNext(inp);
+        Long ret = (Long)resl.result;
+        assertEquals(inp, ret);
+    }
+
+    @Test
+    public void testGetNextDouble() throws ExecException {
+        Double inp = r.nextDouble();
+        ce.setValue(inp);
+        Result resd = ce.getNext(inp);
+        Double ret = (Double)resd.result;
+        assertEquals(inp, ret);
+    }
+
+    @Test
+    public void testGetNextFloat() throws ExecException {
+        Float inp = r.nextFloat();
+        ce.setValue(inp);
+        Result resf = ce.getNext(inp);
+        Float ret = (Float)resf.result;
+        assertEquals(inp, ret);
+    }
+
+    @Test
+    public void testGetNextString() throws ExecException {
+        String inp = GenRandomData.genRandString(r);
+        ce.setValue(inp);
+        Result ress = ce.getNext(inp);
+        String ret = (String)ress.result;
+        assertEquals(inp, ret);
+    }
+
+    @Test
+    public void testGetNextDataByteArray() throws ExecException {
+        DataByteArray inp = GenRandomData.genRandDBA(r);
+        ce.setValue(inp);
+        Result resba = ce.getNext(inp);
+        DataByteArray ret = (DataByteArray)resba.result;
+        assertEquals(inp, ret);
+    }
+
+    @Test
+    public void testGetNextMap() throws ExecException {
+        Map<Integer,String> inp = GenRandomData.genRandMap(r, 10);
+        ce.setValue(inp);
+        Result resm = ce.getNext(inp);
+        Map<Integer,String> ret = (Map)resm.result;
+        assertEquals(inp, ret);
+    }
+
+    @Test
+    public void testGetNextBoolean() throws ExecException {
+        Boolean inp = r.nextBoolean();
+        ce.setValue(inp);
+        Result res = ce.getNext(inp);
+        Boolean ret = (Boolean)res.result;
+        assertEquals(inp, ret);
+    }
+
+    @Test
+    public void testGetNextTuple() throws ExecException {
+        Tuple inp = GenRandomData.genRandSmallBagTuple(r, 10, 100);
+        ce.setValue(inp);
+        Result rest = ce.getNext(inp);
+        Tuple ret = (Tuple)rest.result;
+        assertEquals(inp, ret);
+    }
+
+    @Test
+    public void testGetNextDataBag() throws ExecException {
+        DataBag inp = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
+        ce.setValue(inp);
+        Result res = ce.getNext(inp);
+        DataBag ret = (DataBag)res.result;
+        assertEquals(inp, ret);
+    }
 
 }

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestDivide.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestDivide.java?rev=644076&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestDivide.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestDivide.java Wed Apr  2 14:23:36 2008
@@ -0,0 +1,167 @@
+/*
+ * 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.test;
+
+
+import java.util.Map;
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+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.physicalLayer.POStatus;
+import org.apache.pig.impl.physicalLayer.Result;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.ConstantExpression;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.arithmeticOperators.Divide;
+import org.apache.pig.test.utils.GenRandomData;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class TestDivide extends TestCase{
+
+    Random r = new Random();
+    ConstantExpression lt, rt;
+    Divide op = new Divide(new OperatorKey("", r.nextLong()));
+
+    @Before
+    public void setUp() throws Exception {
+        lt = new ConstantExpression(new OperatorKey("",r.nextLong()));
+        rt = new ConstantExpression(new OperatorKey("",r.nextLong()));
+    }
+
+    @Test
+    public void testOperator() throws ExecException{
+        //int TRIALS = 10;
+        byte[] types = { DataType.BAG, DataType.BOOLEAN, DataType.BYTEARRAY, DataType.CHARARRAY, 
+                DataType.DOUBLE, DataType.FLOAT, DataType.INTEGER, DataType.LONG, DataType.MAP, DataType.TUPLE};
+        //Map<Byte,String> map = GenRandomData.genTypeToNameMap();
+        System.out.println("Testing ADD operator");
+        for(byte type : types) {
+            lt.setResultType(type);
+            rt.setResultType(type);
+            op.setLhs(lt);
+            op.setRhs(rt);
+
+            switch(type){
+            case DataType.BAG:
+                DataBag inpdb1 = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
+                DataBag inpdb2 = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
+                lt.setValue(inpdb1);
+                rt.setValue(inpdb2);
+                Result resdb = op.getNext(inpdb1);
+                assertEquals(resdb.returnStatus, POStatus.STATUS_ERR);
+                break;
+            case DataType.BOOLEAN:
+                Boolean inpb1 = r.nextBoolean();
+                Boolean inpb2 = r.nextBoolean();
+                lt.setValue(inpb1);
+                rt.setValue(inpb2);
+                Result resb = op.getNext(inpb1);
+                assertEquals(resb.returnStatus, POStatus.STATUS_ERR);
+                break;
+            case DataType.BYTEARRAY: {
+                DataByteArray inpba1 = GenRandomData.genRandDBA(r);
+                DataByteArray inpba2 = GenRandomData.genRandDBA(r);
+                lt.setValue(inpba1);
+                rt.setValue(inpba2);
+                Result resba = op.getNext(inpba1);
+                //DataByteArray expected = new DataByteArray(inpba1.toString() + inpba2.toString());
+                //assertEquals(expected, (DataByteArray)resba.result);
+                assertEquals(POStatus.STATUS_ERR, resba.returnStatus);
+                break;
+            }
+            case DataType.CHARARRAY: {
+                String inps1 = GenRandomData.genRandString(r);
+                String inps2 = GenRandomData.genRandString(r);
+                lt.setValue(inps1);
+                rt.setValue(inps2);
+                Result ress = op.getNext(inps1);
+                /*String expected = new String(inps1 + inps2);
+                assertEquals(expected, (String)ress.result);*/
+                assertEquals(POStatus.STATUS_ERR, ress.returnStatus);
+                break;
+            }
+            case DataType.DOUBLE: {
+                Double inpd1 = r.nextDouble();
+                Double inpd2 = r.nextDouble();
+                lt.setValue(inpd1);
+                rt.setValue(inpd2);
+                Result resd = op.getNext(inpd1);
+                Double expected = new Double(inpd1 / inpd2);
+                assertEquals(expected, (Double)resd.result);
+                break;
+            }
+            case DataType.FLOAT: {
+                Float inpf1 = r.nextFloat();
+                Float inpf2 = r.nextFloat();
+                lt.setValue(inpf1);
+                rt.setValue(inpf2);
+                Result resf = op.getNext(inpf1);
+                Float expected = new Float(inpf1 / inpf2);
+                assertEquals(expected, (Float)resf.result);
+                break;
+            }
+            case DataType.INTEGER: {
+                Integer inpi1 = r.nextInt();
+                Integer inpi2 = r.nextInt();
+                lt.setValue(inpi1);
+                rt.setValue(inpi2);
+                Result resi = op.getNext(inpi1);
+                Integer expected = new Integer(inpi1 / inpi2);
+                assertEquals(expected, (Integer) resi.result);
+                break;
+            }
+            case DataType.LONG: {
+                Long inpl1 = r.nextLong();
+                Long inpl2 = r.nextLong();
+                lt.setValue(inpl1);
+                rt.setValue(inpl2);
+                Result resl = op.getNext(inpl1);
+                Long expected = new Long(inpl1 / inpl2);
+                assertEquals(expected, (Long)resl.result);
+                break;
+            }
+            case DataType.MAP: {
+                Map<Integer,String> inpm1 = GenRandomData.genRandMap(r, 10);
+                Map<Integer,String> inpm2 = GenRandomData.genRandMap(r, 10);
+                lt.setValue(inpm1);
+                rt.setValue(inpm2);
+                Result resm = op.getNext(inpm1);
+                assertEquals(POStatus.STATUS_ERR, resm.returnStatus);
+                break;
+            }
+            case DataType.TUPLE: {
+                Tuple inpt1 = GenRandomData.genRandSmallBagTuple(r, 10, 100);
+                Tuple inpt2 = GenRandomData.genRandSmallBagTuple(r, 10, 100);
+                lt.setValue(inpt1);
+                rt.setValue(inpt2);
+                Result rest = op.getNext(inpt1);
+                assertEquals(POStatus.STATUS_ERR, rest.returnStatus);
+                break;
+            }
+            }
+        }
+    }
+}
+

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestFilter.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestFilter.java?rev=644076&r1=644075&r2=644076&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestFilter.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestFilter.java Wed Apr  2 14:23:36 2008
@@ -1,3 +1,20 @@
+/*
+ * 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.test;
 
 import static org.junit.Assert.*;
@@ -16,30 +33,30 @@
 import org.junit.Test;
 
 public class TestFilter {
-	POFilter pass;
-	POFilter fail;
-	Tuple t;
-	
-	@Before
-	public void setUp() throws Exception {
-		pass = GenPhyOp.topFilterOpWithExPlan(50, 25);
-		fail = GenPhyOp.topFilterOpWithExPlan(25, 50);
-		
-		t = GenRandomData.genRandSmallBagTuple(new Random(), 10, 100);
-	}
-
-	@After
-	public void tearDown() throws Exception {
-	}
-
-	@Test
-	public void testGetNextTuple() throws ExecException {
-		pass.attachInput(t);
-		Result res = pass.getNext(t);
-		assertEquals(t, res.result);
-		fail.attachInput(t);
-		res = fail.getNext(t);
-		assertEquals(res.returnStatus, POStatus.STATUS_EOP);
-	}
+    POFilter pass;
+    POFilter fail;
+    Tuple t;
+    
+    @Before
+    public void setUp() throws Exception {
+        pass = GenPhyOp.topFilterOpWithExPlan(50, 25);
+        fail = GenPhyOp.topFilterOpWithExPlan(25, 50);
+        
+        t = GenRandomData.genRandSmallBagTuple(new Random(), 10, 100);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+    }
+
+    @Test
+    public void testGetNextTuple() throws ExecException {
+        pass.attachInput(t);
+        Result res = pass.getNext(t);
+        assertEquals(t, res.result);
+        fail.attachInput(t);
+        res = fail.getNext(t);
+        assertEquals(res.returnStatus, POStatus.STATUS_EOP);
+    }
 
 }

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestMod.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestMod.java?rev=644076&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestMod.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestMod.java Wed Apr  2 14:23:36 2008
@@ -0,0 +1,165 @@
+/*
+ * 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.test;
+
+
+import java.util.Map;
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+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.physicalLayer.POStatus;
+import org.apache.pig.impl.physicalLayer.Result;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.ConstantExpression;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.arithmeticOperators.Mod;
+import org.apache.pig.test.utils.GenRandomData;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class TestMod extends TestCase{
+
+    Random r = new Random();
+    ConstantExpression lt, rt;
+    Mod op = new Mod(new OperatorKey("", r.nextLong()));
+
+    @Before
+    public void setUp() throws Exception {
+        lt = new ConstantExpression(new OperatorKey("",r.nextLong()));
+        rt = new ConstantExpression(new OperatorKey("",r.nextLong()));
+    }
+
+    @Test
+    public void testOperator() throws ExecException{
+        //int TRIALS = 10;
+        byte[] types = { DataType.BAG, DataType.BOOLEAN, DataType.BYTEARRAY, DataType.CHARARRAY, 
+                DataType.DOUBLE, DataType.FLOAT, DataType.INTEGER, DataType.LONG, DataType.MAP, DataType.TUPLE};
+        //Map<Byte,String> map = GenRandomData.genTypeToNameMap();
+        System.out.println("Testing Mod operator");
+        for(byte type : types) {
+            lt.setResultType(type);
+            rt.setResultType(type);
+            op.setLhs(lt);
+            op.setRhs(rt);
+
+            switch(type){
+            case DataType.BAG:
+                DataBag inpdb1 = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
+                DataBag inpdb2 = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
+                lt.setValue(inpdb1);
+                rt.setValue(inpdb2);
+                Result resdb = op.getNext(inpdb1);
+                assertEquals(resdb.returnStatus, POStatus.STATUS_ERR);
+                break;
+            case DataType.BOOLEAN:
+                Boolean inpb1 = r.nextBoolean();
+                Boolean inpb2 = r.nextBoolean();
+                lt.setValue(inpb1);
+                rt.setValue(inpb2);
+                Result resb = op.getNext(inpb1);
+                assertEquals(resb.returnStatus, POStatus.STATUS_ERR);
+                break;
+            case DataType.BYTEARRAY: {
+                DataByteArray inpba1 = GenRandomData.genRandDBA(r);
+                DataByteArray inpba2 = GenRandomData.genRandDBA(r);
+                lt.setValue(inpba1);
+                rt.setValue(inpba2);
+                Result resba = op.getNext(inpba1);
+                //DataByteArray expected = new DataByteArray(inpba1.toString() + inpba2.toString());
+                //assertEquals(expected, (DataByteArray)resba.result);
+                assertEquals(POStatus.STATUS_ERR, resba.returnStatus);
+                break;
+            }
+            case DataType.CHARARRAY: {
+                String inps1 = GenRandomData.genRandString(r);
+                String inps2 = GenRandomData.genRandString(r);
+                lt.setValue(inps1);
+                rt.setValue(inps2);
+                Result ress = op.getNext(inps1);
+                /*String expected = new String(inps1 + inps2);
+                assertEquals(expected, (String)ress.result);*/
+                assertEquals(POStatus.STATUS_ERR, ress.returnStatus);
+                break;
+            }
+            case DataType.DOUBLE: {
+                Double inpd1 = r.nextDouble();
+                Double inpd2 = r.nextDouble();
+                lt.setValue(inpd1);
+                rt.setValue(inpd2);
+                Result resd = op.getNext(inpd1);
+                assertEquals(POStatus.STATUS_ERR, resd.returnStatus);
+                break;
+            }
+            case DataType.FLOAT: {
+                Float inpf1 = r.nextFloat();
+                Float inpf2 = r.nextFloat();
+                lt.setValue(inpf1);
+                rt.setValue(inpf2);
+                Result resf = op.getNext(inpf1);
+                assertEquals(POStatus.STATUS_ERR, resf.returnStatus);
+                break;
+            }
+            case DataType.INTEGER: {
+                Integer inpi1 = r.nextInt();
+                Integer inpi2 = r.nextInt();
+                lt.setValue(inpi1);
+                rt.setValue(inpi2);
+                Result resi = op.getNext(inpi1);
+                Integer expected = new Integer(inpi1 % inpi2);
+                assertEquals(expected, (Integer) resi.result);
+                break;
+            }
+            case DataType.LONG: {
+                Long inpl1 = r.nextLong();
+                Long inpl2 = r.nextLong();
+                lt.setValue(inpl1);
+                rt.setValue(inpl2);
+                Result resl = op.getNext(inpl1);
+                Long expected = new Long(inpl1 % inpl2);
+                assertEquals(expected, (Long)resl.result);
+                break;
+            }
+            case DataType.MAP: {
+                Map<Integer,String> inpm1 = GenRandomData.genRandMap(r, 10);
+                Map<Integer,String> inpm2 = GenRandomData.genRandMap(r, 10);
+                lt.setValue(inpm1);
+                rt.setValue(inpm2);
+                Result resm = op.getNext(inpm1);
+                assertEquals(POStatus.STATUS_ERR, resm.returnStatus);
+                break;
+            }
+            case DataType.TUPLE: {
+                Tuple inpt1 = GenRandomData.genRandSmallBagTuple(r, 10, 100);
+                Tuple inpt2 = GenRandomData.genRandSmallBagTuple(r, 10, 100);
+                lt.setValue(inpt1);
+                rt.setValue(inpt2);
+                Result rest = op.getNext(inpt1);
+                assertEquals(POStatus.STATUS_ERR, rest.returnStatus);
+                break;
+            }
+            }
+        }
+    }
+}
+

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestMultiply.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestMultiply.java?rev=644076&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestMultiply.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestMultiply.java Wed Apr  2 14:23:36 2008
@@ -0,0 +1,167 @@
+/*
+ * 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.test;
+
+
+import java.util.Map;
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+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.physicalLayer.POStatus;
+import org.apache.pig.impl.physicalLayer.Result;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.ConstantExpression;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.arithmeticOperators.Multiply;
+import org.apache.pig.test.utils.GenRandomData;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class TestMultiply extends TestCase{
+
+    Random r = new Random();
+    ConstantExpression lt, rt;
+    Multiply op = new Multiply(new OperatorKey("", r.nextLong()));
+
+    @Before
+    public void setUp() throws Exception {
+        lt = new ConstantExpression(new OperatorKey("",r.nextLong()));
+        rt = new ConstantExpression(new OperatorKey("",r.nextLong()));
+    }
+
+    @Test
+    public void testOperator() throws ExecException{
+        //int TRIALS = 10;
+        byte[] types = { DataType.BAG, DataType.BOOLEAN, DataType.BYTEARRAY, DataType.CHARARRAY, 
+                DataType.DOUBLE, DataType.FLOAT, DataType.INTEGER, DataType.LONG, DataType.MAP, DataType.TUPLE};
+        //Map<Byte,String> map = GenRandomData.genTypeToNameMap();
+        System.out.println("Testing ADD operator");
+        for(byte type : types) {
+            lt.setResultType(type);
+            rt.setResultType(type);
+            op.setLhs(lt);
+            op.setRhs(rt);
+
+            switch(type){
+            case DataType.BAG:
+                DataBag inpdb1 = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
+                DataBag inpdb2 = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
+                lt.setValue(inpdb1);
+                rt.setValue(inpdb2);
+                Result resdb = op.getNext(inpdb1);
+                assertEquals(resdb.returnStatus, POStatus.STATUS_ERR);
+                break;
+            case DataType.BOOLEAN:
+                Boolean inpb1 = r.nextBoolean();
+                Boolean inpb2 = r.nextBoolean();
+                lt.setValue(inpb1);
+                rt.setValue(inpb2);
+                Result resb = op.getNext(inpb1);
+                assertEquals(resb.returnStatus, POStatus.STATUS_ERR);
+                break;
+            case DataType.BYTEARRAY: {
+                DataByteArray inpba1 = GenRandomData.genRandDBA(r);
+                DataByteArray inpba2 = GenRandomData.genRandDBA(r);
+                lt.setValue(inpba1);
+                rt.setValue(inpba2);
+                Result resba = op.getNext(inpba1);
+                //DataByteArray expected = new DataByteArray(inpba1.toString() + inpba2.toString());
+                //assertEquals(expected, (DataByteArray)resba.result);
+                assertEquals(POStatus.STATUS_ERR, resba.returnStatus);
+                break;
+            }
+            case DataType.CHARARRAY: {
+                String inps1 = GenRandomData.genRandString(r);
+                String inps2 = GenRandomData.genRandString(r);
+                lt.setValue(inps1);
+                rt.setValue(inps2);
+                Result ress = op.getNext(inps1);
+                /*String expected = new String(inps1 + inps2);
+                assertEquals(expected, (String)ress.result);*/
+                assertEquals(POStatus.STATUS_ERR, ress.returnStatus);
+                break;
+            }
+            case DataType.DOUBLE: {
+                Double inpd1 = r.nextDouble();
+                Double inpd2 = r.nextDouble();
+                lt.setValue(inpd1);
+                rt.setValue(inpd2);
+                Result resd = op.getNext(inpd1);
+                Double expected = new Double(inpd1 * inpd2);
+                assertEquals(expected, (Double)resd.result);
+                break;
+            }
+            case DataType.FLOAT: {
+                Float inpf1 = r.nextFloat();
+                Float inpf2 = r.nextFloat();
+                lt.setValue(inpf1);
+                rt.setValue(inpf2);
+                Result resf = op.getNext(inpf1);
+                Float expected = new Float(inpf1 * inpf2);
+                assertEquals(expected, (Float)resf.result);
+                break;
+            }
+            case DataType.INTEGER: {
+                Integer inpi1 = r.nextInt();
+                Integer inpi2 = r.nextInt();
+                lt.setValue(inpi1);
+                rt.setValue(inpi2);
+                Result resi = op.getNext(inpi1);
+                Integer expected = new Integer(inpi1 * inpi2);
+                assertEquals(expected, (Integer) resi.result);
+                break;
+            }
+            case DataType.LONG: {
+                Long inpl1 = r.nextLong();
+                Long inpl2 = r.nextLong();
+                lt.setValue(inpl1);
+                rt.setValue(inpl2);
+                Result resl = op.getNext(inpl1);
+                Long expected = new Long(inpl1 * inpl2);
+                assertEquals(expected, (Long)resl.result);
+                break;
+            }
+            case DataType.MAP: {
+                Map<Integer,String> inpm1 = GenRandomData.genRandMap(r, 10);
+                Map<Integer,String> inpm2 = GenRandomData.genRandMap(r, 10);
+                lt.setValue(inpm1);
+                rt.setValue(inpm2);
+                Result resm = op.getNext(inpm1);
+                assertEquals(POStatus.STATUS_ERR, resm.returnStatus);
+                break;
+            }
+            case DataType.TUPLE: {
+                Tuple inpt1 = GenRandomData.genRandSmallBagTuple(r, 10, 100);
+                Tuple inpt2 = GenRandomData.genRandSmallBagTuple(r, 10, 100);
+                lt.setValue(inpt1);
+                rt.setValue(inpt2);
+                Result rest = op.getNext(inpt1);
+                assertEquals(POStatus.STATUS_ERR, rest.returnStatus);
+                break;
+            }
+            }
+        }
+    }
+}
+

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestPhyOp.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPhyOp.java?rev=644076&r1=644075&r2=644076&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestPhyOp.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestPhyOp.java Wed Apr  2 14:23:36 2008
@@ -1,3 +1,20 @@
+/*
+ * 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.test;
 
 import static org.junit.Assert.*;
@@ -21,48 +38,48 @@
 import org.junit.Test;
 
 public class TestPhyOp {
-	PhysicalOperator<PhyPlanVisitor> op;
-	PhysicalOperator<PhyPlanVisitor> inpOp;
-	Tuple t;
-
-	@Before
-	public void setUp() throws Exception {
-		op = GenPhyOp.topFilterOp();
-		inpOp = GenPhyOp.topFilterOpWithExPlan(25,10);
-		t = GenRandomData.genRandSmallBagTuple(new Random(), 10, 100);
-	}
-
-	@After
-	public void tearDown() throws Exception {
-	}
-
-	@Test
-	public void testProcessInput() throws ExecException {
-		//Stand-alone tests
-		Result res = op.processInput();
-		assertEquals(POStatus.STATUS_EOP, res.returnStatus);
-		op.attachInput(t);
-		res = op.processInput();
-		assertEquals(POStatus.STATUS_OK, res.returnStatus);
-		assertEquals(t, res.result);
-		op.detachInput();
-		res = op.processInput();
-		assertEquals(POStatus.STATUS_EOP, res.returnStatus);
-		
-		//With input operator
-		List<PhysicalOperator<PhyPlanVisitor>> inp = new ArrayList<PhysicalOperator<PhyPlanVisitor>>();
-		inp.add(inpOp);
-		op.setInputs(inp);
-		op.processInput();
-		assertEquals(POStatus.STATUS_EOP, res.returnStatus);
-		
-		inpOp.attachInput(t);
-		res = op.processInput();
-		assertEquals(POStatus.STATUS_OK, res.returnStatus);
-		assertEquals(t, res.result);
-		inpOp.detachInput();
-		res = op.processInput();
-		assertEquals(POStatus.STATUS_EOP, res.returnStatus);
-	}
+    PhysicalOperator<PhyPlanVisitor> op;
+    PhysicalOperator<PhyPlanVisitor> inpOp;
+    Tuple t;
+
+    @Before
+    public void setUp() throws Exception {
+        op = GenPhyOp.topFilterOp();
+        inpOp = GenPhyOp.topFilterOpWithExPlan(25,10);
+        t = GenRandomData.genRandSmallBagTuple(new Random(), 10, 100);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+    }
+
+    @Test
+    public void testProcessInput() throws ExecException {
+        //Stand-alone tests
+        Result res = op.processInput();
+        assertEquals(POStatus.STATUS_EOP, res.returnStatus);
+        op.attachInput(t);
+        res = op.processInput();
+        assertEquals(POStatus.STATUS_OK, res.returnStatus);
+        assertEquals(t, res.result);
+        op.detachInput();
+        res = op.processInput();
+        assertEquals(POStatus.STATUS_EOP, res.returnStatus);
+        
+        //With input operator
+        List<PhysicalOperator<PhyPlanVisitor>> inp = new ArrayList<PhysicalOperator<PhyPlanVisitor>>();
+        inp.add(inpOp);
+        op.setInputs(inp);
+        op.processInput();
+        assertEquals(POStatus.STATUS_EOP, res.returnStatus);
+        
+        inpOp.attachInput(t);
+        res = op.processInput();
+        assertEquals(POStatus.STATUS_OK, res.returnStatus);
+        assertEquals(t, res.result);
+        inpOp.detachInput();
+        res = op.processInput();
+        assertEquals(POStatus.STATUS_EOP, res.returnStatus);
+    }
 
 }

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestProject.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestProject.java?rev=644076&r1=644075&r2=644076&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestProject.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestProject.java Wed Apr  2 14:23:36 2008
@@ -1,3 +1,20 @@
+/*
+ * 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.test;
 
 import static org.junit.Assert.*;
@@ -18,37 +35,37 @@
 import org.junit.Test;
 
 public class TestProject {
-	Tuple typFinder;
-	Random r;
-	
-	Tuple t;
-	Result res;
-	POProject proj;
-	
-	@Before
-	public void setUp() throws Exception {
-		r = new Random();
-		typFinder = GenRandomData.genRandSmallBagTuple(r, 10, 100);
-		t = GenRandomData.genRandSmallBagTuple(r,10,100);
-		res = new Result();
-		proj = GenPhyOp.exprProject();
-	}
-
-	@After
-	public void tearDown() throws Exception {
-	}
-
-	@Test
-	public void testGetNext() throws ExecException, IOException {
-		proj.attachInput(t);
-		for(int j=0;j<t.size();j++){
-			proj.attachInput(t);
-			proj.setColumn(j);
-
-			res = proj.getNext();
-			assertEquals(POStatus.STATUS_OK, res.returnStatus);
-			assertEquals(t.get(j), res.result);
-		}
-	}
+    Tuple typFinder;
+    Random r;
+    
+    Tuple t;
+    Result res;
+    POProject proj;
+    
+    @Before
+    public void setUp() throws Exception {
+        r = new Random();
+        typFinder = GenRandomData.genRandSmallBagTuple(r, 10, 100);
+        t = GenRandomData.genRandSmallBagTuple(r,10,100);
+        res = new Result();
+        proj = GenPhyOp.exprProject();
+    }
+
+    @After
+    public void tearDown() throws Exception {
+    }
+
+    @Test
+    public void testGetNext() throws ExecException, IOException {
+        proj.attachInput(t);
+        for(int j=0;j<t.size();j++){
+            proj.attachInput(t);
+            proj.setColumn(j);
+
+            res = proj.getNext();
+            assertEquals(POStatus.STATUS_OK, res.returnStatus);
+            assertEquals(t.get(j), res.result);
+        }
+    }
 
 }

Added: incubator/pig/branches/types/test/org/apache/pig/test/TestSubtract.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestSubtract.java?rev=644076&view=auto
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestSubtract.java (added)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestSubtract.java Wed Apr  2 14:23:36 2008
@@ -0,0 +1,167 @@
+/*
+ * 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.test;
+
+
+import java.util.Map;
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+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.physicalLayer.POStatus;
+import org.apache.pig.impl.physicalLayer.Result;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.ConstantExpression;
+import org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.arithmeticOperators.Subtract;
+import org.apache.pig.test.utils.GenRandomData;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class TestSubtract extends TestCase{
+
+    Random r = new Random();
+    ConstantExpression lt, rt;
+    Subtract op = new Subtract(new OperatorKey("", r.nextLong()));
+
+    @Before
+    public void setUp() throws Exception {
+        lt = new ConstantExpression(new OperatorKey("",r.nextLong()));
+        rt = new ConstantExpression(new OperatorKey("",r.nextLong()));
+    }
+
+    @Test
+    public void testOperator() throws ExecException{
+        //int TRIALS = 10;
+        byte[] types = { DataType.BAG, DataType.BOOLEAN, DataType.BYTEARRAY, DataType.CHARARRAY, 
+                DataType.DOUBLE, DataType.FLOAT, DataType.INTEGER, DataType.LONG, DataType.MAP, DataType.TUPLE};
+        //Map<Byte,String> map = GenRandomData.genTypeToNameMap();
+        System.out.println("Testing ADD operator");
+        for(byte type : types) {
+            lt.setResultType(type);
+            rt.setResultType(type);
+            op.setLhs(lt);
+            op.setRhs(rt);
+
+            switch(type){
+            case DataType.BAG:
+                DataBag inpdb1 = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
+                DataBag inpdb2 = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
+                lt.setValue(inpdb1);
+                rt.setValue(inpdb2);
+                Result resdb = op.getNext(inpdb1);
+                assertEquals(resdb.returnStatus, POStatus.STATUS_ERR);
+                break;
+            case DataType.BOOLEAN:
+                Boolean inpb1 = r.nextBoolean();
+                Boolean inpb2 = r.nextBoolean();
+                lt.setValue(inpb1);
+                rt.setValue(inpb2);
+                Result resb = op.getNext(inpb1);
+                assertEquals(resb.returnStatus, POStatus.STATUS_ERR);
+                break;
+            case DataType.BYTEARRAY: {
+                DataByteArray inpba1 = GenRandomData.genRandDBA(r);
+                DataByteArray inpba2 = GenRandomData.genRandDBA(r);
+                lt.setValue(inpba1);
+                rt.setValue(inpba2);
+                Result resba = op.getNext(inpba1);
+                //DataByteArray expected = new DataByteArray(inpba1.toString() + inpba2.toString());
+                //assertEquals(expected, (DataByteArray)resba.result);
+                assertEquals(POStatus.STATUS_ERR, resba.returnStatus);
+                break;
+            }
+            case DataType.CHARARRAY: {
+                String inps1 = GenRandomData.genRandString(r);
+                String inps2 = GenRandomData.genRandString(r);
+                lt.setValue(inps1);
+                rt.setValue(inps2);
+                Result ress = op.getNext(inps1);
+                /*String expected = new String(inps1 + inps2);
+                assertEquals(expected, (String)ress.result);*/
+                assertEquals(POStatus.STATUS_ERR, ress.returnStatus);
+                break;
+            }
+            case DataType.DOUBLE: {
+                Double inpd1 = r.nextDouble();
+                Double inpd2 = r.nextDouble();
+                lt.setValue(inpd1);
+                rt.setValue(inpd2);
+                Result resd = op.getNext(inpd1);
+                Double expected = new Double(inpd1 - inpd2);
+                assertEquals(expected, (Double)resd.result);
+                break;
+            }
+            case DataType.FLOAT: {
+                Float inpf1 = r.nextFloat();
+                Float inpf2 = r.nextFloat();
+                lt.setValue(inpf1);
+                rt.setValue(inpf2);
+                Result resf = op.getNext(inpf1);
+                Float expected = new Float(inpf1 - inpf2);
+                assertEquals(expected, (Float)resf.result);
+                break;
+            }
+            case DataType.INTEGER: {
+                Integer inpi1 = r.nextInt();
+                Integer inpi2 = r.nextInt();
+                lt.setValue(inpi1);
+                rt.setValue(inpi2);
+                Result resi = op.getNext(inpi1);
+                Integer expected = new Integer(inpi1 - inpi2);
+                assertEquals(expected, (Integer) resi.result);
+                break;
+            }
+            case DataType.LONG: {
+                Long inpl1 = r.nextLong();
+                Long inpl2 = r.nextLong();
+                lt.setValue(inpl1);
+                rt.setValue(inpl2);
+                Result resl = op.getNext(inpl1);
+                Long expected = new Long(inpl1 - inpl2);
+                assertEquals(expected, (Long)resl.result);
+                break;
+            }
+            case DataType.MAP: {
+                Map<Integer,String> inpm1 = GenRandomData.genRandMap(r, 10);
+                Map<Integer,String> inpm2 = GenRandomData.genRandMap(r, 10);
+                lt.setValue(inpm1);
+                rt.setValue(inpm2);
+                Result resm = op.getNext(inpm1);
+                assertEquals(POStatus.STATUS_ERR, resm.returnStatus);
+                break;
+            }
+            case DataType.TUPLE: {
+                Tuple inpt1 = GenRandomData.genRandSmallBagTuple(r, 10, 100);
+                Tuple inpt2 = GenRandomData.genRandSmallBagTuple(r, 10, 100);
+                lt.setValue(inpt1);
+                rt.setValue(inpt2);
+                Result rest = op.getNext(inpt1);
+                assertEquals(POStatus.STATUS_ERR, rest.returnStatus);
+                break;
+            }
+            }
+        }
+    }
+}
+