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/05/13 23:11:24 UTC

svn commit: r656011 [3/5] - in /incubator/pig/branches/types: ./ lib/ src/org/apache/pig/ src/org/apache/pig/backend/hadoop/executionengine/ src/org/apache/pig/data/ src/org/apache/pig/impl/io/ src/org/apache/pig/impl/logicalLayer/ src/org/apache/pig/i...

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POLoad.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POLoad.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POLoad.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POLoad.java Tue May 13 14:11:21 2008
@@ -45,6 +45,10 @@
  *
  */
 public class POLoad extends PhysicalOperator<PhyPlanVisitor> {
+    /**
+     * 
+     */
+//    private static final long serialVersionUID = 1L;
     // The user defined load function or a default load function
     LoadFunc loader = null;
     // The filespec on which the operator is based
@@ -78,7 +82,7 @@
      * 3) Binding to the input stream
      * @throws IOException
      */
-    private void setUp() throws IOException{
+    public void setUp() throws IOException{
         String filename = lFile.getFileName();
         loader = (LoadFunc)PigContext.instantiateFuncFromSpec(lFile.getFuncSpec());
         
@@ -92,7 +96,7 @@
      * using this method
      * @throws IOException
      */
-    private void tearDown() throws IOException{
+    public void tearDown() throws IOException{
         is.close();
         setUpDone = false;
     }
@@ -136,7 +140,10 @@
 
     @Override
     public String name() {
-        return "Load - " + mKey.toString();
+        if(lFile!=null)
+            return "Load" + "(" + lFile.toString() + ")" + " - " + mKey.toString();
+        else
+            return "Load" + "(" + "DummyFil:DummyLdr" + ")" + " - " + mKey.toString();
     }
 
     @Override

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POLocalRearrange.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POLocalRearrange.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POLocalRearrange.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POLocalRearrange.java Tue May 13 14:11:21 2008
@@ -40,6 +40,11 @@
  */
 public class POLocalRearrange extends PhysicalOperator<PhyPlanVisitor> {
 
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
     private Log log = LogFactory.getLog(getClass());
 
     PhysicalPlan<PhysicalOperator> plan;
@@ -53,6 +58,8 @@
     //as the generate can potentially return multiple tuples for
     //same call.
     private boolean processingPlan = false;
+    
+    byte keyType;
 
     public POLocalRearrange(OperatorKey k) {
         this(k, -1, null);
@@ -191,4 +198,12 @@
         this.plan = plan;
         gen = (POGenerate) plan.getLeaves().get(0);
     }
+
+    public byte getKeyType() {
+        return keyType;
+    }
+
+    public void setKeyType(byte keyType) {
+        this.keyType = keyType;
+    }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POPackage.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POPackage.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POPackage.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POPackage.java Tue May 13 14:11:21 2008
@@ -23,9 +23,8 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.DataBag;
-import org.apache.pig.data.DefaultBagFactory;
-import org.apache.pig.data.DefaultTuple;
 import org.apache.pig.data.IndexedTuple;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.data.TupleFactory;
@@ -48,6 +47,11 @@
  * bags based on the index.
  */
 public class POPackage extends PhysicalOperator<PhyPlanVisitor> {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
     //The iterator of indexed Tuples
     //that is typically provided by
     //Hadoop
@@ -55,6 +59,9 @@
     
     //The key being worked on
     Object key;
+    
+    //key's type
+    byte keyType;
 
     //The number of inputs to this
     //co-group
@@ -145,6 +152,7 @@
      */
     @Override
     public Result getNext(Tuple t) throws ExecException {
+        //Should be removed once we start integration/perf testing
         if(indTupIter==null){
             throw new ExecException("Incorrect usage of the Package operator. " +
                     "No input has been attached.");
@@ -153,7 +161,7 @@
         //Create numInputs bags
         DataBag[] dbs = new DataBag[numInputs];
         for (int i = 0; i < numInputs; i++) {
-            dbs[i] = DefaultBagFactory.getInstance().newDefaultBag();
+            dbs[i] = BagFactory.getInstance().newDefaultBag();
         }
         
         //For each indexed tup in the inp, sort them
@@ -168,24 +176,19 @@
         //the key and all the above constructed bags
         //and return it.
         Tuple res;
-        try{
-            res = TupleFactory.getInstance().newTuple(numInputs+1);
-            res.set(0,key);
-            int i=-1;
-            for (DataBag bag : dbs) {
-                if(inner[++i]){
-                    if(bag.size()==0){
-                        detachInput();
-                        Result r = new Result();
-                        r.returnStatus = POStatus.STATUS_NULL;
-                        return r;
-                    }
+        res = TupleFactory.getInstance().newTuple(numInputs+1);
+        res.set(0,key);
+        int i=-1;
+        for (DataBag bag : dbs) {
+            if(inner[++i]){
+                if(bag.size()==0){
+                    detachInput();
+                    Result r = new Result();
+                    r.returnStatus = POStatus.STATUS_NULL;
+                    return r;
                 }
-                res.set(i+1,bag);
             }
-        }catch(ExecException e){
-            log.error("Received error while constructing the output tuple");
-            return new Result();
+            res.set(i+1,bag);
         }
         detachInput();
         Result r = new Result();
@@ -194,7 +197,11 @@
         return r;
     }
 
+    public byte getKeyType() {
+        return keyType;
+    }
 
-
-    
+    public void setKeyType(byte keyType) {
+        this.keyType = keyType;
+    }
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POSort.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POSort.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POSort.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POSort.java Tue May 13 14:11:21 2008
@@ -245,4 +245,8 @@
 		v.visitSort(this);
 	}
 
+    public List<ExprPlan> getSortPlans() {
+        return sortPlans;
+    }
+
 }

Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POSplit.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POSplit.java?rev=656011&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POSplit.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POSplit.java Tue May 13 14:11:21 2008
@@ -0,0 +1,120 @@
+/*
+ * 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;
+
+import java.util.List;
+
+import org.apache.pig.impl.io.FileSpec;
+import org.apache.pig.impl.logicalLayer.OperatorKey;
+import org.apache.pig.impl.physicalLayer.plans.PhyPlanVisitor;
+import org.apache.pig.impl.plan.VisitorException;
+
+/**
+ * The MapReduce Split operator.
+ * The assumption here is that
+ * the logical to physical translation
+ * will create this dummy operator with
+ * just the filename using which the input
+ * branch will be stored and used for loading
+ * Also the translation should make sure that
+ * appropriate filter operators are configured
+ * as outputs of this operator using the conditions
+ * specified in the LOSplit. So LOSplit will be converted
+ * into:
+ * 
+ *     |        |           |
+ *  Filter1  Filter2 ... Filter3
+ *     |        |    ...    |
+ *     |        |    ...    |
+ *     ---- POSplit -... ----
+ * This is different than the existing implementation
+ * where the POSplit writes to sidefiles after filtering
+ * and then loads the appropirate file.
+ * 
+ * The approach followed here is as good as the old
+ * approach if not better in many cases because
+ * of the availablity of attachinInputs. An optimization
+ * that can ensue is if there are multiple loads that
+ * load the same file, they can be merged into one and 
+ * then the operators that take input from the load 
+ * can be stored. This can be used when
+ * the mapPlan executes to read the file only once and
+ * attach the resulting tuple as inputs to all the 
+ * operators that take input from this load.
+ * 
+ * In some cases where the conditions are exclusive and
+ * some outputs are ignored, this approach can be worse.
+ * But this leads to easier management of the Split and
+ * also allows to reuse this data stored from the split
+ * job whenever necessary.
+ */
+public class POSplit extends PhysicalOperator<PhyPlanVisitor> {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+    //The filespec that is used to store
+    //and load the output of the split job
+    //which is the job containing the split
+    FileSpec splitStore;
+
+    public POSplit(OperatorKey k) {
+        this(k,-1,null);
+    }
+
+    public POSplit(OperatorKey k, int rp) {
+        this(k,rp,null);
+    }
+
+    public POSplit(OperatorKey k, List<PhysicalOperator> inp) {
+        this(k,-1,null);
+    }
+
+    public POSplit(OperatorKey k, int rp, List<PhysicalOperator> inp) {
+        super(k, rp, inp);
+    }
+
+    @Override
+    public void visit(PhyPlanVisitor v) throws VisitorException {
+        v.visitSplit(this);
+    }
+
+    @Override
+    public String name() {
+        return "Split - " + mKey.toString();
+    }
+
+    @Override
+    public boolean supportsMultipleInputs() {
+        return false;
+    }
+
+    @Override
+    public boolean supportsMultipleOutputs() {
+        return true;
+    }
+
+    public FileSpec getSplitStore() {
+        return splitStore;
+    }
+
+    public void setSplitStore(FileSpec splitStore) {
+        this.splitStore = splitStore;
+    }
+
+}

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POStore.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POStore.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POStore.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POStore.java Tue May 13 14:11:21 2008
@@ -44,6 +44,10 @@
  *
  */
 public class POStore extends PhysicalOperator<PhyPlanVisitor> {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
     // The user defined load function or a default load function
     private StoreFunc storer;
     // The filespec on which the operator is based
@@ -153,7 +157,10 @@
 
     @Override
     public String name() {
-        return "Store - " + mKey.toString();
+        if(sFile!=null)
+            return "Store" + "(" + sFile.toString() + ")" + " - " + mKey.toString();
+        else
+            return "Store" + "(" + "DummyFil:DummyLdr" + ")" + " - " + mKey.toString();
     }
 
     @Override

Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POUnion.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POUnion.java?rev=656011&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POUnion.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POUnion.java Tue May 13 14:11:21 2008
@@ -0,0 +1,140 @@
+/*
+ * 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;
+
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.pig.backend.executionengine.ExecException;
+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.plans.PhyPlanVisitor;
+import org.apache.pig.impl.plan.VisitorException;
+
+/**
+ * The union operator that combines the two inputs into a single
+ * stream. Note that this doesn't eliminate duplicate tuples.
+ * The Operator will also be added to every map plan which processes
+ * more than one input. This just pulls out data from the piepline
+ * using the proposed single threaded shared execution model. By shared
+ * execution I mean, one input to the Union operator is called
+ * once and the execution moves to the next non-drained input till
+ * all the inputs are drained.
+ *
+ */
+public class POUnion extends PhysicalOperator<PhyPlanVisitor> {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
+    //Used for efficiently shifting between non-drained
+    //inputs
+    BitSet done;
+    
+    //The index of the last input that was read
+    int lastInd = 0;
+
+    public POUnion(OperatorKey k) {
+        this(k, -1, null);
+    }
+
+    public POUnion(OperatorKey k, int rp) {
+        this(k, rp, null);
+    }
+
+    public POUnion(OperatorKey k, List<PhysicalOperator> inp) {
+        this(k, -1, inp);
+    }
+
+    public POUnion(OperatorKey k, int rp, List<PhysicalOperator> inp) {
+        super(k, rp, inp);
+    }
+
+    @Override
+    public void setInputs(List<PhysicalOperator> inputs) {
+        super.setInputs(inputs);
+        done = new BitSet(inputs.size());
+    }
+
+    @Override
+    public void visit(PhyPlanVisitor v) throws VisitorException {
+        v.visitUnion(this);
+    }
+
+    @Override
+    public String name() {
+        return "Union - " + mKey.toString();
+    }
+
+    @Override
+    public boolean supportsMultipleInputs() {
+        return true;
+    }
+
+    @Override
+    public boolean supportsMultipleOutputs() {
+        return false;
+    }
+
+    public void clearDone() {
+        done.clear();
+    }
+
+    /**
+     * The code below, tries to follow our single threaded 
+     * shared execution model with execution being passed
+     * around each non-drained input
+     */
+    @Override
+    public Result getNext(Tuple t) throws ExecException {
+        while(true){
+            if (done.nextClearBit(0) >= inputs.size()) {
+                res = new Result();
+                res.returnStatus = POStatus.STATUS_EOP;
+                clearDone();
+                return res;
+            }
+            if(lastInd >= inputs.size() || done.nextClearBit(lastInd) >= inputs.size())
+                lastInd = 0;
+            int ind = done.nextClearBit(lastInd);
+            Result res;
+            
+            while(true){
+                res = inputs.get(ind).getNext(t);
+                if(res.returnStatus == POStatus.STATUS_NULL)
+                    continue;
+                
+                lastInd = ind + 1;
+                
+                if(res.returnStatus == POStatus.STATUS_ERR)
+                    return new Result();
+                
+                if (res.returnStatus == POStatus.STATUS_OK)
+                    return res;
+                
+                if (res.returnStatus == POStatus.STATUS_EOP) {
+                    done.set(ind);
+                    break;
+                }
+            }
+        }
+    }
+}

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POUserFunc.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POUserFunc.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POUserFunc.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/POUserFunc.java Tue May 13 14:11:21 2008
@@ -256,8 +256,10 @@
 
 	@Override
 	public String name() {
-
-		return "POUserFunc - " + mKey.toString();
+	    if(funcSpec!=null)
+	        return "POUserFunc" + "(" + funcSpec + ")" + " - " + mKey.toString();
+        else
+            return "POUserFunc" + "(" + "DummySpec" + ")" + " - " + mKey.toString();
 	}
 
 	@Override

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/PhysicalOperator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/PhysicalOperator.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/PhysicalOperator.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/PhysicalOperator.java Tue May 13 14:11:21 2008
@@ -27,11 +27,11 @@
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.OperatorKey;
-import org.apache.pig.impl.plan.VisitorException;
 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.Operator;
+import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.backend.executionengine.ExecException;
 
 /**
@@ -62,7 +62,7 @@
 
     private Log log = LogFactory.getLog(getClass());
 
-    static final long serialVersionUID = 1L;
+    protected static final long serialVersionUID = 1L;
 
     // The degree of parallelism requested
     protected int requestedParallelism;
@@ -199,7 +199,7 @@
     public Result processInput() throws ExecException {
         Result res = new Result();
         Tuple inpValue = null;
-        if (input == null && inputs == null) {
+        if (input == null && (inputs == null || inputs.size()==0)) {
 //            log.warn("No inputs found. Signaling End of Processing.");
             res.returnStatus = POStatus.STATUS_EOP;
             return res;
@@ -213,8 +213,8 @@
             return res;
         }
     }
-    
-    public abstract void visit(V v) throws VisitorException ;
+
+    public abstract void visit(V v) throws VisitorException;
 
     public Result getNext(Integer i) throws ExecException {
         return res;

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ConstantExpression.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ConstantExpression.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ConstantExpression.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ConstantExpression.java Tue May 13 14:11:21 2008
@@ -19,17 +19,15 @@
 
 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.Tuple;
 import org.apache.pig.impl.logicalLayer.OperatorKey;
-import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
-import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.impl.plan.VisitorException;
 
 
 /**
@@ -39,7 +37,12 @@
  */
 public class ConstantExpression extends ExpressionOperator {
     
-    private Log log = LogFactory.getLog(getClass());
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
+//    private Log log = LogFactory.getLog(getClass());
     
     //The value that this constant represents
     Object value;
@@ -57,7 +60,10 @@
 
     @Override
     public String name() {
-        return "Constant(" + value.toString() +") - " + mKey.toString();
+        if(value!=null)
+            return "Constant(" + value.toString() +") - " + mKey.toString();
+        else
+            return "Constant(" + "DummyVal" +") - " + mKey.toString();
     }
 
     @Override

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ExpressionOperator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ExpressionOperator.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ExpressionOperator.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/ExpressionOperator.java Tue May 13 14:11:21 2008
@@ -19,9 +19,9 @@
 
 
 import org.apache.pig.impl.logicalLayer.OperatorKey;
-import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
 import org.apache.pig.impl.physicalLayer.topLevelOperators.PhysicalOperator;
+import org.apache.pig.impl.plan.VisitorException;
 
 /**
  * A base class for all types of expressions. All expression
@@ -30,6 +30,7 @@
  */
 
 public abstract class ExpressionOperator extends PhysicalOperator<ExprPlanVisitor> {
+    private static final long serialVersionUID = 1L;
     
     public ExpressionOperator(OperatorKey k) {
         this(k,-1);

Added: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POCast.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POCast.java?rev=656011&view=auto
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POCast.java (added)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POCast.java Tue May 13 14:11:21 2008
@@ -0,0 +1,101 @@
+/*
+ * 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;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.data.DataByteArray;
+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.plans.ExprPlanVisitor;
+import org.apache.pig.impl.plan.VisitorException;
+
+/**
+ * This is just a cast that converts DataByteArray into either
+ * String or Integer. Just added it for testing the POUnion. 
+ * Need the full operator implementation.
+ */
+public class POCast extends ExpressionOperator {
+
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
+    public POCast(OperatorKey k) {
+        super(k);
+        // TODO Auto-generated constructor stub
+    }
+
+    public POCast(OperatorKey k, int rp) {
+        super(k, rp);
+        // TODO Auto-generated constructor stub
+    }
+
+    @Override
+    public void visit(ExprPlanVisitor v) throws VisitorException {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public String name() {
+        return "Cast - " + mKey.toString();
+    }
+
+    @Override
+    public boolean supportsMultipleInputs() {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public Result getNext(Integer i) throws ExecException {
+        Result res = inputs.get(0).getNext(i);
+
+        if(res.returnStatus != POStatus.STATUS_OK){
+            return res;
+        }
+        
+        if(res.result instanceof DataByteArray){
+            String rslt = ((DataByteArray)res.result).toString();
+            res.result = Integer.parseInt(rslt.trim());
+            return res;
+        }
+        return new Result();
+    }
+
+    @Override
+    public Result getNext(String s) throws ExecException {
+        Result res = inputs.get(0).getNext(s);
+
+        if(res.returnStatus != POStatus.STATUS_OK){
+            return res;
+        }
+        
+        if(res.result instanceof DataByteArray){
+            String rslt = ((DataByteArray)res.result).toString();
+            res.result = rslt;
+            return res;
+        }
+        return new Result();
+    }
+    
+    
+
+}

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POProject.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POProject.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POProject.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/POProject.java Tue May 13 14:11:21 2008
@@ -17,21 +17,20 @@
  */
 package org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators;
 
-import java.io.IOException;
 import java.util.Iterator;
 import java.util.Map;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+/*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.Tuple;
 import org.apache.pig.impl.logicalLayer.OperatorKey;
-import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
+import org.apache.pig.impl.plan.VisitorException;
 
 /**
  * Implements the overloaded form of the project operator.
@@ -42,7 +41,12 @@
  */
 public class POProject extends ExpressionOperator {
     
-    private Log log = LogFactory.getLog(getClass());
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
+//    private Log log = LogFactory.getLog(getClass());
     
     //The column to project
     int column = 0;
@@ -116,12 +120,7 @@
         if(res.returnStatus != POStatus.STATUS_OK){
             return res;
         }
-        try {
-            res.result = ((Tuple)res.result).get(column);
-        } catch (ExecException e) {
-            res.returnStatus = POStatus.STATUS_ERR;
-            log.warn(e.getMessage());
-        }
+        res.result = ((Tuple)res.result).get(column);
         return res;
     }
 
@@ -188,24 +187,19 @@
             inpValue = (Tuple)res.result;
             res.result = null;
             
-            try {
-                Object ret = inpValue.get(column);
-                if(overloaded){
-                    DataBag retBag = (DataBag)ret;
-                    bagIterator = retBag.iterator();
-                    if(bagIterator.hasNext()){
-                        processingBagOfTuples = true;
-                        res.result = bagIterator.next();
-                    }
-                }
-                else {
-                    res.result = (Tuple)ret;
+            Object ret = inpValue.get(column);
+            if(overloaded){
+                DataBag retBag = (DataBag)ret;
+                bagIterator = retBag.iterator();
+                if(bagIterator.hasNext()){
+                    processingBagOfTuples = true;
+                    res.result = bagIterator.next();
                 }
-                return res;
-            } catch (ExecException e) {
-                res.returnStatus = POStatus.STATUS_ERR;
-                log.error(e.getMessage());
             }
+            else {
+                res.result = (Tuple)ret;
+            }
+            return res;
         }
         if(bagIterator.hasNext()){
             res.result = bagIterator.next();

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/BinaryExpressionOperator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/BinaryExpressionOperator.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/BinaryExpressionOperator.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/BinaryExpressionOperator.java Tue May 13 14:11:21 2008
@@ -28,6 +28,8 @@
  *
  */
 public abstract class BinaryExpressionOperator extends ExpressionOperator {
+    private static final long serialVersionUID = 1L;
+    
     protected ExpressionOperator lhs;
     protected ExpressionOperator rhs;
     

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Add.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Add.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Add.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Add.java Tue May 13 14:11:21 2008
@@ -19,13 +19,18 @@
 
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.impl.logicalLayer.OperatorKey;
-import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
+import org.apache.pig.impl.plan.VisitorException;
 
 public class Add extends ArithmeticOperator {
 
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
     public Add(OperatorKey k) {
         super(k);
     }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Divide.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Divide.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Divide.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Divide.java Tue May 13 14:11:21 2008
@@ -19,13 +19,18 @@
 
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.impl.logicalLayer.OperatorKey;
-import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
+import org.apache.pig.impl.plan.VisitorException;
 
 public class Divide extends ArithmeticOperator {
 
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
     public Divide(OperatorKey k) {
         super(k);
     }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Mod.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Mod.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Mod.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Mod.java Tue May 13 14:11:21 2008
@@ -19,13 +19,18 @@
 
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.impl.logicalLayer.OperatorKey;
-import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
+import org.apache.pig.impl.plan.VisitorException;
 
 public class Mod extends ArithmeticOperator {
 
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
     public Mod(OperatorKey k) {
         super(k);
     }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Multiply.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Multiply.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Multiply.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Multiply.java Tue May 13 14:11:21 2008
@@ -19,13 +19,18 @@
 
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.impl.logicalLayer.OperatorKey;
-import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
+import org.apache.pig.impl.plan.VisitorException;
 
 public class Multiply extends ArithmeticOperator {
 
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
     public Multiply(OperatorKey k) {
         super(k);
     }

Modified: 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=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Subtract.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/arithmeticOperators/Subtract.java Tue May 13 14:11:21 2008
@@ -19,13 +19,18 @@
 
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.impl.logicalLayer.OperatorKey;
-import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
+import org.apache.pig.impl.plan.VisitorException;
 
 public class Subtract extends ArithmeticOperator {
 
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
     public Subtract(OperatorKey k) {
         super(k);
     }

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=656011&r1=656010&r2=656011&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 Tue May 13 14:11:21 2008
@@ -17,10 +17,7 @@
  */
 package org.apache.pig.impl.physicalLayer.topLevelOperators.expressionOperators.binaryExprOps.comparators;
 
-import org.apache.pig.data.DataByteArray;
 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.binaryExprOps.BinaryExpressionOperator;
 
 /**

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/EqualToExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/EqualToExpr.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/EqualToExpr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/EqualToExpr.java Tue May 13 14:11:21 2008
@@ -21,12 +21,12 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.OperatorKey;
-import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
@@ -34,6 +34,10 @@
 
 public class EqualToExpr extends ComparisonOperator {
 
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
     private final Log log = LogFactory.getLog(getClass());
 
     public EqualToExpr(OperatorKey k) {

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GTOrEqualToExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GTOrEqualToExpr.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GTOrEqualToExpr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/GTOrEqualToExpr.java Tue May 13 14:11:21 2008
@@ -24,12 +24,16 @@
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
-import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.backend.executionengine.ExecException;
 
 
 public class GTOrEqualToExpr extends ComparisonOperator {
 
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
     private final Log log = LogFactory.getLog(getClass());
     
     public GTOrEqualToExpr(OperatorKey k) {

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=656011&r1=656010&r2=656011&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 Tue May 13 14:11:21 2008
@@ -21,14 +21,18 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.impl.logicalLayer.OperatorKey;
-import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
+import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.backend.executionengine.ExecException;
 
 public class GreaterThanExpr extends ComparisonOperator {
 
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
     private final Log log = LogFactory.getLog(getClass());
 
     public GreaterThanExpr(OperatorKey k) {

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LTOrEqualToExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LTOrEqualToExpr.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LTOrEqualToExpr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LTOrEqualToExpr.java Tue May 13 14:11:21 2008
@@ -24,10 +24,14 @@
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
-import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.backend.executionengine.ExecException;
 
 public class LTOrEqualToExpr extends ComparisonOperator {
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
     private final Log log = LogFactory.getLog(getClass());
 
     public LTOrEqualToExpr(OperatorKey k) {

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LessThanExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LessThanExpr.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LessThanExpr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/LessThanExpr.java Tue May 13 14:11:21 2008
@@ -24,11 +24,15 @@
 import org.apache.pig.impl.physicalLayer.POStatus;
 import org.apache.pig.impl.physicalLayer.Result;
 import org.apache.pig.impl.physicalLayer.plans.ExprPlanVisitor;
-import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.backend.executionengine.ExecException;
 
 public class LessThanExpr extends ComparisonOperator {
 
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
     private final Log log = LogFactory.getLog(getClass());
 
     public LessThanExpr(OperatorKey k) {

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/NotEqualToExpr.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/NotEqualToExpr.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/NotEqualToExpr.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/physicalLayer/topLevelOperators/expressionOperators/binaryExprOps/comparators/NotEqualToExpr.java Tue May 13 14:11:21 2008
@@ -34,6 +34,10 @@
 
 public class NotEqualToExpr extends ComparisonOperator {
 
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
     private final Log log = LogFactory.getLog(getClass());
 
     public NotEqualToExpr(OperatorKey k) {

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/plan/CompilationMessageCollector.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/plan/CompilationMessageCollector.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/plan/CompilationMessageCollector.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/plan/CompilationMessageCollector.java Tue May 13 14:11:21 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.plan ;
 
 import java.util.Iterator;

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/plan/Operator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/plan/Operator.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/plan/Operator.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/plan/Operator.java Tue May 13 14:11:21 2008
@@ -28,7 +28,7 @@
 /**
  * Base class for all types of operators.
  */
-abstract public class Operator<V extends PlanVisitor> implements Serializable {
+abstract public class Operator<V extends PlanVisitor> implements Serializable, Comparable<Operator> {
     private static final long serialVersionUID = 1L;
 
     /**
@@ -87,4 +87,36 @@
 
         return msg.toString();
     }
+    
+    /**
+     * Compares to Operators based on their opKey
+     */
+    @Override
+    public boolean equals(Object obj) {
+        if(obj instanceof Operator){
+            Operator opObj = (Operator)obj;
+            if(obj==this)
+                return true;
+            return mKey.equals(opObj);
+        }
+        return false;
+    }
+    
+    /**
+     * Needed to ensure that the list iterators'
+     * outputs are deterministic. Without this
+     * we are totally at object id's mercy.
+     */
+    @Override
+    public int hashCode() {
+        return mKey.hashCode();
+    }
+
+    public int compareTo(Operator o) {
+        return mKey.compareTo(o.mKey);
+    }
+    
+    
+    
+    
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/plan/OperatorPlan.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/plan/OperatorPlan.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/plan/OperatorPlan.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/plan/OperatorPlan.java Tue May 13 14:11:21 2008
@@ -24,6 +24,7 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.pig.impl.logicalLayer.OperatorKey;
 import org.apache.commons.logging.Log;
@@ -261,7 +262,83 @@
         }
     }
     
-    public boolean isSingleLeafPlan() {
+    /**
+     * Merges the operators in the incoming operPlan with
+     * this plan's operators. By merging I mean just making
+     * a combined graph with each one as a component
+     * It doesn't support merging of shared plans
+     * @param inpPlan
+     * @return
+     * @throws PlanException
+     */
+    public OperatorPlan<E> merge(OperatorPlan<E> inpPlan) throws PlanException {
+        Map<E, OperatorKey> inpOps = inpPlan.mOps;
+        Set<E> curOpsKeySet = mOps.keySet();
+        for (Map.Entry<E, OperatorKey> mapEnt : inpOps.entrySet()) {
+            if (curOpsKeySet.contains(mapEnt.getKey()))
+                throw new PlanException(
+                        "There are operators that are shared across the plans. Merge of "
+                                + "mutually exclusive plans is the only supported merge.");
+            mOps.put(mapEnt.getKey(), mapEnt.getValue());
+        }
+
+        Map<OperatorKey, E> inpKeys = inpPlan.mKeys;
+        Set<OperatorKey> curOKKeySet = mKeys.keySet();
+        for (Map.Entry<OperatorKey, E> mapEnt : inpKeys.entrySet()) {
+            if (curOKKeySet.contains(mapEnt.getKey()))
+                throw new PlanException(
+                        "There are operators that are shared across the plans. Merge of "
+                                + "mutually exclusive plans is the only supported merge.");
+            mKeys.put(mapEnt.getKey(), mapEnt.getValue());
+        }
+
+        MultiMap<E, E> inpFromEdges = inpPlan.mFromEdges;
+        Set<E> curFEKeySet = mFromEdges.keySet();
+        for (E fromEdg : inpFromEdges.keySet()) {
+            if (curFEKeySet.contains(fromEdg))
+                throw new PlanException(
+                        "There are operators that are shared across the plans. Merge of "
+                                + "mutually exclusive plans is the only supported merge.");
+            for (E e : inpFromEdges.get(fromEdg)) {
+                mFromEdges.put(fromEdg, e);
+            }
+        }
+
+        MultiMap<E, E> inpToEdges = inpPlan.mToEdges;
+        Set<E> curTEKeySet = mToEdges.keySet();
+        for (E toEdg : inpToEdges.keySet()) {
+            if (curTEKeySet.contains(toEdg))
+                throw new PlanException(
+                        "There are operators that are shared across the plans. Merge of "
+                                + "mutually exclusive plans is the only supported merge.");
+            for (E e : inpToEdges.get(toEdg)) {
+                mToEdges.put(toEdg, e);
+            }
+        }
+
+        markDirty();
+        return this;
+    }
+    
+    /**
+     * Utility method heavily used in the MRCompiler
+     * Adds the leaf operator to the plan and connects
+     * all existing leaves to the new leaf
+     * @param leaf
+     * @throws PlanException 
+     */
+    public void addAsLeaf(E leaf) throws PlanException {
+        List<E> ret = new ArrayList<E>();
+        for (E operator : getLeaves()) {
+            ret.add(operator);
+        }
+        add(leaf);
+        for (E oper : ret) {
+            connect(oper, leaf);
+        }
+    }
+	
+	public boolean isSingleLeafPlan() {
         List<E> tmpList = getLeaves() ;
         return tmpList.size() == 1 ;
     }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanValidationException.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanValidationException.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanValidationException.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanValidationException.java Tue May 13 14:11:21 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.plan;
 
 import org.apache.pig.impl.logicalLayer.FrontendException;

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanValidationExecutor.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanValidationExecutor.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanValidationExecutor.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanValidationExecutor.java Tue May 13 14:11:21 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.plan;
 
 /***

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanValidator.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanValidator.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanValidator.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/plan/PlanValidator.java Tue May 13 14:11:21 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.plan;
 
 import org.apache.pig.impl.plan.CompilationMessageCollector.MessageType;

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/util/JarManager.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/util/JarManager.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/util/JarManager.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/util/JarManager.java Tue May 13 14:11:21 2008
@@ -40,6 +40,7 @@
 
 //import org.apache.pig.backend.hadoop.executionengine.mapreduceExec.PigMapReduce;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.mapReduceLayer.PigMapReduce;
 
 
 public class JarManager {
@@ -97,12 +98,9 @@
     public static void createJar(OutputStream os, List<String> funcs, PigContext pigContext) throws ClassNotFoundException, IOException {
         Vector<JarListEntry> jarList = new Vector<JarListEntry>();
         for(String toSend: pigPackagesToSend) {
-//          TODO FIX Need to uncomment this with the right logic
-//            addContainingJar(jarList, PigMapReduce.class, toSend, pigContext);
+            addContainingJar(jarList, PigMapReduce.class, toSend, pigContext);
         }
-//      TODO FIX Need to uncomment this with the right logic
-//        ClassLoader pigClassLoader = PigMapReduce.class.getClassLoader();
-        ClassLoader pigClassLoader = PigContext.class.getClassLoader();
+        ClassLoader pigClassLoader = PigMapReduce.class.getClassLoader();
         
         for (String func: funcs) {
             Class clazz = pigContext.getClassForAlias(func);
@@ -151,9 +149,7 @@
         for (int i = 0; i < pigContext.extraJars.size(); i++) {
             urls[i + passedJar] = new URL("file:" + pigContext.extraJars.get(i));
         }
-//      TODO FIX Need to uncomment this with the right logic
-//        return new URLClassLoader(urls, PigMapReduce.class.getClassLoader());
-        return new URLClassLoader(urls, PigContext.class.getClassLoader());
+        return new URLClassLoader(urls, PigMapReduce.class.getClassLoader());
     }
     
 

Modified: incubator/pig/branches/types/src/org/apache/pig/tools/grunt/GruntParser.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/tools/grunt/GruntParser.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/tools/grunt/GruntParser.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/tools/grunt/GruntParser.java Tue May 13 14:11:21 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.tools.grunt;
 
 import java.io.IOException;

Modified: incubator/pig/branches/types/test/org/apache/pig/test/OrdAsc.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/OrdAsc.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/OrdAsc.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/OrdAsc.java Tue May 13 14:11:21 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 org.apache.pig.data.Tuple;

Modified: incubator/pig/branches/types/test/org/apache/pig/test/OrdDesc.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/OrdDesc.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/OrdDesc.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/OrdDesc.java Tue May 13 14:11:21 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 org.apache.pig.data.Tuple;

Modified: incubator/pig/branches/types/test/org/apache/pig/test/OrdDescNumeric.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/OrdDescNumeric.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/OrdDescNumeric.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/OrdDescNumeric.java Tue May 13 14:11:21 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;
 

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestCombiner.java Tue May 13 14:11:21 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 java.io.File;

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestGrunt.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestGrunt.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestGrunt.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestGrunt.java Tue May 13 14:11:21 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 org.junit.Test;

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestLoad.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestLoad.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestLoad.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestLoad.java Tue May 13 14:11:21 2008
@@ -17,12 +17,10 @@
  */
 package org.apache.pig.test;
 
-import static org.junit.Assert.*;
-
 import java.io.BufferedReader;
+import java.io.File;
 import java.io.FileReader;
 
-import org.apache.hadoop.fs.BufferedFSInputStream;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.DataBag;
@@ -50,8 +48,9 @@
     static MiniCluster cluster = MiniCluster.buildCluster();
     @Before
     public void setUp() throws Exception {
-        
-        inpFSpec = new FileSpec("file:////etc/passwd",PigStorage.class.getName()+"(':')");
+        String curDir = System.getProperty("user.dir");
+        String inpDir = curDir + File.separatorChar + "test/org/apache/pig/test/data/InputFiles/";
+        inpFSpec = new FileSpec("file:" + inpDir + "passwd",PigStorage.class.getName()+"(':')");
         pc = new PigContext();
         pc.connect();
         
@@ -60,7 +59,7 @@
         ld.setPc(pc);
         
         inpDB = DefaultBagFactory.getInstance().newDefaultBag();
-        BufferedReader br = new BufferedReader(new FileReader("/etc/passwd"));
+        BufferedReader br = new BufferedReader(new FileReader("test/org/apache/pig/test/data/InputFiles/passwd"));
         
         for(String line = br.readLine();line!=null;line=br.readLine()){
             String[] flds = line.split(":",-1);

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestLocalRearrange.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestLocalRearrange.java?rev=656011&r1=656010&r2=656011&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestLocalRearrange.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestLocalRearrange.java Tue May 13 14:11:21 2008
@@ -58,7 +58,7 @@
     public void setUp() throws Exception {
         Random r = new Random();
         db = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
-        lr = GenPhyOp.topLocalRearrangeOPWithPlan(0,0,db.iterator().next());
+        lr = GenPhyOp.topLocalRearrangeOPWithPlanPlain(0,0,db.iterator().next());
         POProject proj = GenPhyOp.exprProject();
         proj.setColumn(0);
         proj.setResultType(DataType.TUPLE);