You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ga...@apache.org on 2008/07/15 18:57:43 UTC

svn commit: r676968 - in /incubator/pig/branches/types: src/org/apache/pig/ src/org/apache/pig/builtin/ src/org/apache/pig/impl/mapReduceLayer/ src/org/apache/pig/impl/plan/ src/org/apache/pig/impl/util/ test/org/apache/pig/test/ test/org/apache/pig/te...

Author: gates
Date: Tue Jul 15 09:57:42 2008
New Revision: 676968

URL: http://svn.apache.org/viewvc?rev=676968&view=rev
Log:
PIG-304.  Added visitor to move local rearrange from reduce to next map.


Modified:
    incubator/pig/branches/types/src/org/apache/pig/Main.java
    incubator/pig/branches/types/src/org/apache/pig/builtin/TOKENIZE.java
    incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/JobControlCompiler.java
    incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java
    incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MapReduceOper.java
    incubator/pig/branches/types/src/org/apache/pig/impl/plan/OperatorPlan.java
    incubator/pig/branches/types/src/org/apache/pig/impl/util/WrappedIOException.java
    incubator/pig/branches/types/test/org/apache/pig/test/TestMRCompiler.java
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC1.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC10.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC11.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC12.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC13.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC14.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC15.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC16.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC2.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC3.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC4.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC5.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC6.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC7.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC8.gld
    incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC9.gld

Modified: incubator/pig/branches/types/src/org/apache/pig/Main.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/Main.java?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/Main.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/Main.java Tue Jul 15 09:57:42 2008
@@ -321,6 +321,7 @@
         usage();
         rc = 1;
     } catch (Throwable e) {
+        e.printStackTrace();
         log.error(e);
     } finally {
         PerformanceTimerFactory.getPerfTimerFactory().dumpTimers();

Modified: incubator/pig/branches/types/src/org/apache/pig/builtin/TOKENIZE.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/builtin/TOKENIZE.java?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/builtin/TOKENIZE.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/builtin/TOKENIZE.java Tue Jul 15 09:57:42 2008
@@ -38,8 +38,12 @@
     public DataBag exec(Tuple input) throws IOException {
         try {
             DataBag output = mBagFactory.newDefaultBag();
-            String str = (String)input.get(0);
-            StringTokenizer tok = new StringTokenizer(str, " \",()*", false);
+            Object o = input.get(0);
+            if (!(o instanceof String)) {
+                throw new IOException("Expected input to be chararray, but" +
+                    " got " + o.getClass().getName());
+            }
+            StringTokenizer tok = new StringTokenizer((String)o, " \",()*", false);
             while (tok.hasMoreTokens()) {
                 output.add(mTupleFactory.newTuple(tok.nextToken()));
             }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/JobControlCompiler.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/JobControlCompiler.java?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/JobControlCompiler.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/JobControlCompiler.java Tue Jul 15 09:57:42 2008
@@ -162,8 +162,8 @@
     }
     
     /**
-     * The method that creates the JobConf corresponding to a MapReduceOper
-     * Doesn't support Sort or Distinct jobs yet. The assumption is that
+     * The method that creates the JobConf corresponding to a MapReduceOper.
+     * The assumption is that
      * every MapReduceOper will have a load and a store. The JobConf removes
      * the load operator and serializes the input filespec so that PigInputFormat can
      * take over the creation of splits. It also removes the store operator
@@ -171,7 +171,7 @@
      * record writing. The remaining portion of the map plan and reduce plans are
      * serialized and stored for the PigMapReduce or PigMapOnly objects to take over
      * the actual running of the plans.
-     * The Mapper & Reducer classes and the required key value formats are set.
+     * The Mapper & Reducer classes and the required key value formats are set.
      * Checks if this is a map only job and uses PigMapOnly class as the mapper
      * and uses PigMapReduce otherwise.
      * If it is a Map Reduce job, it is bound to have a package operator. Remove it from

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MRCompiler.java Tue Jul 15 09:57:42 2008
@@ -39,6 +39,7 @@
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.io.FileSpec;
 import org.apache.pig.impl.mapReduceLayer.plans.MROperPlan;
+import org.apache.pig.impl.mapReduceLayer.plans.MROpPlanVisitor;
 import org.apache.pig.impl.mapReduceLayer.plans.UDFFinder;
 import org.apache.pig.impl.physicalLayer.PhysicalOperator;
 import org.apache.pig.impl.physicalLayer.expressionOperators.ConstantExpression;
@@ -181,9 +182,30 @@
     public MROperPlan compile() throws IOException, PlanException, VisitorException {
         List<PhysicalOperator> leaves = plan.getLeaves();
         POStore store = (POStore)leaves.get(0);
-System.out.println("store file is " + store.getSFile());
         FileLocalizer.registerDeleteOnFail(store.getSFile().getFileName(), pigContext);
         compile(store);
+
+        // I'm quite certain this is not the best way to do this.  The issue
+        // is that for jobs that take multiple map reduce passes, for
+        // non-sort jobs, the POLocalRearrange is being put into the reduce
+        // of MR job n, with the map for MR job n+1 empty and the POPackage
+        // in reduce of MR job n+1.  This causes problems in the collect of
+        // the map MR job n+1.  To resolve this, the following visitor
+        // walks the resulting compiled jobs, looks for the pattern described
+        // above, and then moves the POLocalRearrange to the map of MR job
+        // n+1.  It seems to me there are two possible better solutions:
+        // 1) Change the logic in this compiler to put POLocalRearrange in
+        // the correct place to begin with instead of patching it up later.
+        // I'd do this but I don't fully understand the logic here and it's
+        // complex.
+        // 2) Change our map reduce execution to have a reduce only mode.  In
+        // this case the map would not even try to parse the input, it would
+        // just be 100% pass through.  I suspect this might be better though
+        // I don't fully understand the consequences of this.
+        // Given these issues, the following works for now, and we can fine
+        // tune it when Shravan returns.
+        RearrangeAdjuster ra = new RearrangeAdjuster(MRPlan);
+        ra.visit();
         
         return MRPlan;
     }
@@ -282,7 +304,8 @@
             } else if (mro.isMapDone() && !mro.isReduceDone()) {
                 mro.reducePlan.addAsLeaf(op);
             } else {
-                log.warn("Both map and reduce phases have been done. This is unexpected while compiling!");
+                log.error("Both map and reduce phases have been done. This is unexpected while compiling!");
+                throw new PlanException("Both map and reduce phases have been done. This is unexpected while compiling!");
             }
             curMROp = mro;
         } else {
@@ -409,7 +432,8 @@
             mro.reducePlan.addAsLeaf(str);
             mro.setReduceDone(true);
         } else {
-            log.warn("Both map and reduce phases have been done. This is unexpected while compiling!");
+            log.error("Both map and reduce phases have been done. This is unexpected while compiling!");
+            throw new PlanException("Both map and reduce phases have been done. This is unexpected while compiling!");
         }
         return mro;
     }
@@ -485,7 +509,7 @@
             } else if (mro.isMapDone() && !mro.isReduceDone()) {
                 ret.add(mro);
             } else {
-                log.warn(
+                log.error(
                         "Both map and reduce phases have been done. This is unexpected for a merge!");
                 throw new PlanException(
                         "Both map and reduce phases have been done. This is unexpected for a merge!");
@@ -957,39 +981,77 @@
 //        mro.requestedParallelism = rp;
         return mro;
     }
-    
-    /*
-    public static void main(String[] args) throws PlanException, IOException, ExecException, VisitorException {
-        PigContext pc = new PigContext();
-        pc.connect();
-        MRCompiler comp = new MRCompiler(null, pc);
-        Random r = new Random();
-        List<PhysicalPlan> sortPlans = new LinkedList<PhysicalPlan>();
-        POProject pr1 = new POProject(new OperatorKey("", r.nextLong()), -1, 1);
-        pr1.setResultType(DataType.INTEGER);
-        PhysicalPlan expPlan = new PhysicalPlan();
-        expPlan.add(pr1);
-        sortPlans.add(expPlan);
-        List<Boolean> mAscCols = new LinkedList<Boolean>();
-        mAscCols.add(false);
-        MapReduceOper pj = comp.getMROp();
-        POLoad ld = comp.getLoad();
-        pj.mapPlan.add(ld);
-
-        //POSort op = new POSort(new OperatorKey("", r.nextLong()), -1, null,
-        //      sortPlans, mAscCols, null);
-        PODistinct op = new PODistinct(new OperatorKey("", r.nextLong()),
-                -1, null);
-        pj.mapPlan.addAsLeaf(op);
-        
-        POStore st = comp.getStore();
-        pj.mapPlan.addAsLeaf(st);
-        
-        MRCompiler c1 = new MRCompiler(pj.mapPlan,pc);
-        c1.compile();
-        MROperPlan plan = c1.getMRPlan();
-        PlanPrinter<MapReduceOper, MROperPlan> pp = new PlanPrinter<MapReduceOper, MROperPlan>(plan);
-        pp.print(System.out);
+
+    private class RearrangeAdjuster extends MROpPlanVisitor {
+
+        RearrangeAdjuster(MROperPlan plan) {
+            super(plan, new DepthFirstWalker<MapReduceOper, MROperPlan>(plan));
+        }
+
+        @Override
+        public void visitMROp(MapReduceOper mr) throws VisitorException {
+            // Look for map reduce operators whose reduce starts in a local
+            // rearrange.  If it has a successor and that predecessor's map
+            // plan is just a load, push the porearrange to the successor.
+            // Else, throw an error.
+            if (mr.reducePlan.isEmpty()) return;
+            List<PhysicalOperator> mpLeaves = mr.reducePlan.getLeaves();
+            if (mpLeaves.size() != 1) {
+                String msg = new String("Expected reduce to have single leaf");
+                log.error(msg);
+                throw new VisitorException(msg);
+            }
+            PhysicalOperator mpLeaf = mpLeaves.get(0);
+            if (!(mpLeaf instanceof POStore)) {
+                String msg = new String("Expected leaf of reduce plan to " +
+                    "always be POStore!");
+                log.error(msg);
+                throw new VisitorException(msg);
+            }
+            List<PhysicalOperator> preds =
+                mr.reducePlan.getPredecessors(mpLeaf);
+            if (preds == null) return;
+            if (preds.size() > 1) {
+                String msg = new String("Expected mr to have single predecessor");
+                log.error(msg);
+                throw new VisitorException(msg);
+            }
+            PhysicalOperator pred = preds.get(0);
+            if (!(pred instanceof POLocalRearrange)) return;
+
+            // Next question, does the next MROper have an empty map?
+            List<MapReduceOper> succs = mPlan.getSuccessors(mr);
+            if (succs == null) {
+                String msg = new String("Found mro with POLocalRearrange as"
+                    + " last oper but with no succesor!");
+                log.error(msg);
+                throw new VisitorException(msg);
+            }
+            if (succs.size() > 1) {
+                String msg = new String("Expected mr to have single successor");
+                log.error(msg);
+                throw new VisitorException(msg);
+            }
+            MapReduceOper succ = succs.get(0);
+            List<PhysicalOperator> succMpLeaves = succ.mapPlan.getLeaves();
+            List<PhysicalOperator> succMpRoots = succ.mapPlan.getRoots();
+            if (succMpLeaves == null || succMpLeaves.size() > 1 ||
+                    succMpRoots == null || succMpRoots.size() > 1 ||
+                    succMpLeaves.get(0) != succMpRoots.get(0)) {
+                log.warn("Expected to find subsequent map " +
+                    "with just a load, but didn't");
+                return;
+            }
+            PhysicalOperator load = succMpRoots.get(0);
+
+            try {
+                mr.reducePlan.removeAndReconnect(pred);
+                succ.mapPlan.add(pred);
+                succ.mapPlan.connect(load, pred);
+            } catch (PlanException pe) {
+                throw new VisitorException(pe);
+            }
+        }
     }
-    */
+    
 }

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MapReduceOper.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MapReduceOper.java?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MapReduceOper.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/mapReduceLayer/MapReduceOper.java Tue Jul 15 09:57:42 2008
@@ -186,38 +186,6 @@
         return new POUnion(new OperatorKey(scope,nig.getNextNodeId(scope)));
     }
     
-    /**
-     * Utility method for the MRCompiler that
-     * creates a union operator and adds it as
-     * a leaf to the map plan. Used when there
-     * is more than one input in order to process
-     * them in a single map job  
-     * @param mapDone
-     * @throws IOException
-     *//*
-    public void setMapDoneAndMerge(boolean mapDone) throws IOException{
-        this.mapDone = mapDone;
-        if(mapDone){
-            mapPlan.addAsLeaf(GenPhyOp.topUnionOp());
-        }
-    }
-    
-    *//**
-     * Same as the above method but checks to see
-     * there is more than one input. It is for 
-     * optimization purposes as a single input
-     * pipeline really doesn't need the union operator
-     * to execute in a single job.
-     * @param mapDone
-     * @throws IOException
-     *//*
-    public void setMapDoneAndChkdMerge(boolean mapDone) throws IOException {
-        this.mapDone = mapDone;
-        if (mapDone && mapPlan.getLeaves().size()!=1) {
-            mapPlan.addAsLeaf(GenPhyOp.topUnionOp());
-        }
-    }*/
-
     public boolean isReduceDone() {
         return reduceDone;
     }

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=676968&r1=676967&r2=676968&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 Jul 15 09:57:42 2008
@@ -378,6 +378,45 @@
         connect(newNode, before);
     }
 
+    /**
+     * Remove a node in a way that connects the node's predecessor (if any)
+     * with the node's successor (if any).  This function does not handle the
+     * case where the node has multiple predecessors or successors.
+     * @param node Node to be removed
+     * @throws PlanException if the node has more than one predecessor or
+     * successor.
+     */
+    public void removeAndReconnect(E node) throws PlanException {
+        List<E> preds = getPredecessors(node);
+        E pred = null;
+        if (preds != null) {
+            if (preds.size() > 1) {
+                PlanException pe = new PlanException("Attempt to remove " +
+                    " and reconnect for node with multiple predecessors.");
+                log.error(pe.getMessage());
+                throw pe;
+            }
+            pred = preds.get(0);
+            disconnect(pred, node);
+        }
+
+        List<E> succs = getSuccessors(node);
+        E succ = null;
+        if (succs != null) {
+            if (succs.size() > 1) {
+                PlanException pe = new PlanException("Attempt to remove " +
+                    " and reconnect for node with multiple successors.");
+                log.error(pe.getMessage());
+                throw pe;
+            }
+            succ = succs.get(0);
+            disconnect(node, succ);
+        }
+
+        remove(node);
+        if (pred != null && succ != null) connect(pred, succ);
+    }
+
     public void dump(PrintStream ps) {
         ps.println("Ops");
         for (E op : mOps.keySet()) {

Modified: incubator/pig/branches/types/src/org/apache/pig/impl/util/WrappedIOException.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/src/org/apache/pig/impl/util/WrappedIOException.java?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/src/org/apache/pig/impl/util/WrappedIOException.java (original)
+++ incubator/pig/branches/types/src/org/apache/pig/impl/util/WrappedIOException.java Tue Jul 15 09:57:42 2008
@@ -31,7 +31,9 @@
     }
     
     public static IOException wrap(final String message, final Throwable e) {
-        final IOException wrappedException = new IOException(message);
+        final IOException wrappedException = new IOException(message + " [" +
+            e.getMessage() + "]");
+        wrappedException.setStackTrace(e.getStackTrace());
         wrappedException.initCause(e);
         return wrappedException;
     }

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestMRCompiler.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestMRCompiler.java?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestMRCompiler.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestMRCompiler.java Tue Jul 15 09:57:42 2008
@@ -57,14 +57,10 @@
 import org.junit.Test;
 
 public class TestMRCompiler extends junit.framework.TestCase {
-    static PhysicalPlan php = new PhysicalPlan();
-
 //    MiniCluster cluster = MiniCluster.buildCluster();
     
     static PigContext pc;
 
-    static String[] tests;
-
     static final int MAX_SIZE = 100000;
 
     static final long SEED = 1013;
@@ -86,19 +82,6 @@
         GenPhyOp.setR(r);
         
         GenPhyOp.setPc(pc);
-        int numTests = 16;
-//        int numTests = 9;
-        tests = new String[numTests];
-        int cnt = -1;
-
-        for (int i = 1; i <= 9; i++)
-            tests[++cnt] = "intTestSim" + i;
-        tests[++cnt] = "intTestRun1";
-        tests[++cnt] = "intTestRun2";
-        for (int i = 1; i <= 3; i++)
-            tests[++cnt] = "intTestSpl" + i;
-        tests[++cnt] = "intTestSortUDF1";
-        tests[++cnt] = "intTestDistinct1";
     }
 
     @After
@@ -106,18 +89,8 @@
     }
     
     @Test
-    public void testCompile() throws ExecException, VisitorException,
-            ExecException, PlanException, SecurityException,
-            IllegalArgumentException, NoSuchMethodException,
-            IllegalAccessException, InvocationTargetException, IOException {
-        setUp();
-//        generate();
-        intTest();
-    }
-
-    public static void intTestRun1() throws ExecException, PlanException, VisitorException,
-            ExecException {
-        php = new PhysicalPlan();
+    public void testRun1() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
 
         PhysicalPlan part1 = new PhysicalPlan();
         POLoad lC = GenPhyOp.topLoadOp();
@@ -221,12 +194,11 @@
 
         php.add(st);
         php.connect(unABC, st);
-
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC10.gld");
     }
 
-    public static void intTestRun2() throws ExecException, PlanException, VisitorException,
-            ExecException {
-        php = new PhysicalPlan();
+    public void testRun2() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
 
         PhysicalPlan part1 = new PhysicalPlan();
         POLoad lC = GenPhyOp.topLoadOp();
@@ -313,11 +285,11 @@
 
         php.add(st);
         php.connect(unABC, st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC11.gld");
     }
 
-    public static void intTestSpl1() throws ExecException, VisitorException, PlanException,
-            ExecException, IOException {
-        php = new PhysicalPlan();
+    public void testSpl1() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
 
         POLoad lA = GenPhyOp.topLoadOp();
         POSplit spl = GenPhyOp.topSplitOp();
@@ -351,12 +323,12 @@
         POStore st = GenPhyOp.topStoreOp();
         php.add(st);
         php.connect(pk, st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC12.gld");
 
     }
 
-    public static void intTestSpl2() throws ExecException, VisitorException, PlanException,
-            ExecException, IOException {
-        php = new PhysicalPlan();
+    public void testSpl2() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
 
         POLoad lA = GenPhyOp.topLoadOp();
         POSplit spl = GenPhyOp.topSplitOp();
@@ -399,12 +371,12 @@
 
         POStore st = GenPhyOp.topStoreOp();
         php.addAsLeaf(st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC13.gld");
 
     }
 
-    public static void intTestSpl3() throws ExecException, VisitorException, PlanException,
-            ExecException, IOException {
-        php = new PhysicalPlan();
+    public void testSpl3() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
 
         POLoad lA = GenPhyOp.topLoadOp();
         POSplit spl = GenPhyOp.topSplitOp();
@@ -478,17 +450,14 @@
 
         POStore st = GenPhyOp.topStoreOp();
         php.addAsLeaf(st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC14.gld");
 
     }
 
-    /**
-     * Tests Single input case for both blocking and non-blocking
-     * with both map and reduce phases
-     * @throws PlanException
-     * @throws ExecException
-     */
-    public static void intTestSim1() throws PlanException, ExecException {
-        php = new PhysicalPlan();
+     // Tests Single input case for both blocking and non-blocking
+     // with both map and reduce phases
+    public void testSim1() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
         POLoad ld = GenPhyOp.topLoadOp();
         php.add(ld);
         PhysicalPlan grpChain1 = GenPhyOp.grpChain();
@@ -513,11 +482,12 @@
         php.add(st);
 
         php.connect(fl, st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC1.gld");
 
     }
 
-    public static void intTestSim2() throws ExecException, PlanException {
-        php = new PhysicalPlan();
+    public void testSim2() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
 
         PhysicalPlan ldGrpChain1 = GenPhyOp.loadedGrpChain();
         PhysicalPlan ldGrpChain2 = GenPhyOp.loadedGrpChain();
@@ -532,11 +502,12 @@
         php.add(st);
 
         php.connect(un, st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC2.gld");
 
     }
 
-    public static void intTestSim3() throws ExecException, PlanException {
-        php = new PhysicalPlan();
+    public void testSim3() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
 
         PhysicalPlan ldGrpChain1 = GenPhyOp.loadedGrpChain();
         PhysicalPlan ldGrpChain2 = GenPhyOp.loadedGrpChain();
@@ -556,10 +527,11 @@
         php.add(st);
 
         php.connect(un, st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC3.gld");
     }
 
-    public static void intTestSim4() throws ExecException, PlanException {
-        php = new PhysicalPlan();
+    public void intTestSim4() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
 
         PhysicalPlan ldGrpChain1 = GenPhyOp.loadedGrpChain();
         PhysicalPlan ldGrpChain2 = GenPhyOp.loadedGrpChain();
@@ -583,10 +555,11 @@
         php.add(st);
 
         php.connect(un, st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC4.gld");
     }
 
-    public static void intTestSim5() throws ExecException, PlanException {
-        php = new PhysicalPlan();
+    public void testSim5() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
         PhysicalPlan ldFil1 = GenPhyOp.loadedFilter();
         PhysicalPlan ldFil2 = GenPhyOp.loadedFilter();
         php.merge(ldFil1);
@@ -599,10 +572,11 @@
         php.add(st);
 
         php.connect(un, st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC5.gld");
     }
 
-    public static void intTestSim6() throws ExecException, PlanException {
-        php = new PhysicalPlan();
+    public void testSim6() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
 
         PhysicalPlan ldGrpChain1 = GenPhyOp.loadedGrpChain();
         PhysicalPlan ldGrpChain2 = GenPhyOp.loadedGrpChain();
@@ -624,11 +598,12 @@
 
         POStore st = GenPhyOp.topStoreOp();
         php.addAsLeaf(st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC6.gld");
 
     }
 
-    public static void intTestSim7() throws ExecException, PlanException {
-        php = new PhysicalPlan();
+    public void testSim7() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
 
         PhysicalPlan ldGrpChain1 = GenPhyOp.loadedGrpChain();
         PhysicalPlan ldGrpChain2 = GenPhyOp.loadedGrpChain();
@@ -655,10 +630,11 @@
 
         POStore st = GenPhyOp.topStoreOp();
         php.addAsLeaf(st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC7.gld");
     }
 
-    public static void intTestSim8() throws ExecException, PlanException {
-        php = new PhysicalPlan();
+    public void testSim8() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
 
         PhysicalPlan ldGrpChain1 = GenPhyOp.loadedGrpChain();
         PhysicalPlan ldGrpChain2 = GenPhyOp.loadedGrpChain();
@@ -689,10 +665,11 @@
 
         POStore st = GenPhyOp.topStoreOp();
         php.addAsLeaf(st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC8.gld");
     }
 
-    public static void intTestSim9() throws ExecException, PlanException {
-        php = new PhysicalPlan();
+    public void testSim9() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
 
         POGlobalRearrange gr = GenPhyOp.topGlobalRearrangeOp();
         php.addAsLeaf(gr);
@@ -711,10 +688,11 @@
 
         POStore st = GenPhyOp.topStoreOp();
         php.addAsLeaf(st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC9.gld");
     }
     
-    public static void intTestSortUDF1() throws PlanException, ExecException{
-        php = new PhysicalPlan();
+    public void testSortUDF1() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
         PhysicalPlan ldFil1 = GenPhyOp.loadedFilter();
         php.merge(ldFil1);
         
@@ -772,10 +750,11 @@
         
         POStore st = GenPhyOp.topStoreOp();
         php.addAsLeaf(st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC15.gld");
     }
     
-    public static void intTestDistinct1() throws PlanException, ExecException{
-        php = new PhysicalPlan();
+    public void testDistinct1() throws Exception {
+        PhysicalPlan php = new PhysicalPlan();
         PhysicalPlan ldFil1 = GenPhyOp.loadedFilter();
         php.merge(ldFil1);
         
@@ -794,6 +773,7 @@
         php.addAsLeaf(op1);
         POStore st = GenPhyOp.topStoreOp();
         php.addAsLeaf(st);
+        run(php, "test/org/apache/pig/test/data/GoldenFiles/MRC16.gld");
     }
     
     public static class WeirdComparator extends ComparisonFunc {
@@ -815,77 +795,34 @@
 
     }
 
-    public static void generate() throws SecurityException, NoSuchMethodException,
-            IllegalArgumentException, IllegalAccessException,
-            InvocationTargetException, VisitorException, ExecException,
-            PlanException, IOException {
-        for (int i = 0; i < tests.length; i++) {
-            Method m = TestMRCompiler.class.getMethod(tests[i], null);
-            m.invoke(null, null);
-
-            System.out.println("The plan being compiled under " + tests[i]
-                    + ":");
-            PlanPrinter phpp = new PlanPrinter(php);
-            phpp.visit();
-
-            System.out.println();
-            System.out.println("The compiled plan:");
-
-            MRCompiler comp = new MRCompiler(php, pc);
-            comp.compile();
-
-            MROperPlan mrp = comp.getMRPlan();
-            PlanPrinter ppp = new PlanPrinter(mrp);
-            ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-            ppp.print(baos);
-
-            FileOutputStream fos = new FileOutputStream("test/org/apache/pig/test/data/GoldenFiles/MRC"
-                    + (i + 1) + ".gld");
-            fos.write(baos.toByteArray());
-
-            System.out.println(baos.toString());
-            System.out.println("--------------------------------------");
-        }
-    }
-
-    public static void intTest() throws SecurityException, NoSuchMethodException,
-            IllegalArgumentException, IllegalAccessException,
-            InvocationTargetException, ExecException, PlanException,
-            VisitorException, IOException {
+    private void run(PhysicalPlan pp, String expectedFile) throws Exception {
         String compiledPlan, goldenPlan = null;
         int MAX_SIZE = 100000;
-        for (int i = 0; i < tests.length; i++) {
-            Method m = TestMRCompiler.class.getMethod(tests[i], null);
-            m.invoke(null, null);
-            MRCompiler comp = new MRCompiler(php, pc);
-            comp.compile();
-
-            MROperPlan mrp = comp.getMRPlan();
-            PlanPrinter ppp = new PlanPrinter(mrp);
-            ByteArrayOutputStream baos = new ByteArrayOutputStream();
-            ppp.print(baos);
-            compiledPlan = baos.toString();
-
-            FileInputStream fis = new FileInputStream("test/org/apache/pig/test/data/GoldenFiles/MRC"
-                    + (i + 1) + ".gld");
-            byte[] b = new byte[MAX_SIZE];
-            int len = fis.read(b);
-            goldenPlan = new String(b, 0, len);
-
-            php.explain(System.out);
-            System.out.println();
-            System.out.println("<<<" + compiledPlan + ">>>");
-            System.out.println("-------------");
-            System.out.println("Golden");
-            System.out.println("<<<" + goldenPlan + ">>>");
-            System.out.println("-------------");
-            //System.out.println(compiledPlan.compareTo(goldenPlan)==0);
-            //assertEquals(true, compiledPlan.compareTo(goldenPlan) == 0);
-            assertEquals(goldenPlan, compiledPlan);
-        }
+        MRCompiler comp = new MRCompiler(pp, pc);
+        comp.compile();
+
+        MROperPlan mrp = comp.getMRPlan();
+        PlanPrinter ppp = new PlanPrinter(mrp);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        ppp.print(baos);
+        compiledPlan = baos.toString();
+
+        FileInputStream fis = new FileInputStream(expectedFile);
+        byte[] b = new byte[MAX_SIZE];
+        int len = fis.read(b);
+        goldenPlan = new String(b, 0, len);
+
+        pp.explain(System.out);
+        System.out.println();
+        System.out.println("<<<" + compiledPlan + ">>>");
+        System.out.println("-------------");
+        System.out.println("Golden");
+        System.out.println("<<<" + goldenPlan + ">>>");
+        System.out.println("-------------");
+        assertEquals(goldenPlan, compiledPlan);
     }
 
+
     
 
 }

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC1.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC1.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC1.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC1.gld Tue Jul 15 09:57:42 2008
@@ -1,17 +1,17 @@
-MapReduce(-1) - -3:
-|   Store(DummyFil:DummyLdr) - -6748240903696823165
+MapReduce(-1) - -78:
+|   Store(DummyFil:DummyLdr) - --7868505214447593853
 |   |
-|   |---Filter[tuple] - -7469509242284658386
+|   |---Filter[tuple] - --5177745552827005198
 |       |
-|       |---Package[tuple]{Unknown} - --7212359720440714287
-|   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -2
+|       |---Package[tuple]{Unknown} - --6430355428631435461
+|   Local Rearrange[tuple]{Unknown} - -8729990799028586395
+|   |
+|   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -77
 |
-|---MapReduce(-1) - -0:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -1
+|---MapReduce(-1) - -75:
+    |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -76
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - -8345455294066939854
-    |       |
-    |       |---Package[tuple]{Unknown} - -3248199015665744565
-    |   Local Rearrange[tuple]{Unknown} - -6520791719738296531
+    |   |---Package[tuple]{Unknown} - -4721502244557927278
+    |   Local Rearrange[tuple]{Unknown} - --7681398237172009051
     |   |
-    |   |---Load(DummyFil:DummyLdr) - --5314747545448923824
\ No newline at end of file
+    |   |---Load(DummyFil:DummyLdr) - -6620645493024302760
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC10.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC10.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC10.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC10.gld Tue Jul 15 09:57:42 2008
@@ -1,57 +1,57 @@
-MapReduce(-1) - -72:
+MapReduce(-1) - -11:
 Reduce Plan Empty
-|   Store(DummyFil:DummyLdr) - --2325244147060806375
+|   Store(DummyFil:DummyLdr) - --2449990780759860228
 |   |
-|   |---Union[tuple] - --8240903279973257769
+|   |---Union[tuple] - -4061122832660258194
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -73
+|       |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -12
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -75
+|       |---Load(/tmp/temp-1456742965/tmp774375955:org.apache.pig.builtin.BinStorage) - -14
 |
-|---MapReduce(-1) - -63:
-|   |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -74
+|---MapReduce(-1) - -2:
+|   |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -13
 |   |   |
-|   |   |---Package[tuple]{Unknown} - --6957591191795645940
-|   |   Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -64
-|   |   
-|   |   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -66
-|   |
-|   |---MapReduce(20) - -61:
-|   |   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -65
-|   |   |   |
-|   |   |   |---Local Rearrange[tuple]{Unknown} - --5190071275724378681
-|   |   |       |
-|   |   |       |---Package[tuple]{Unknown} - --1158897849427419546
-|   |   |   Local Rearrange[tuple]{Unknown} - --2057425961601007773
+|   |   |---Filter[tuple] - --171729478481529687
+|   |       |
+|   |       |---Package[tuple]{Unknown} - -3527883492192621891
+|   |   Union[tuple] - -3
+|   |   |
+|   |   |---Local Rearrange[tuple]{Unknown} - --2655303127943013956
 |   |   |   |
-|   |   |   |---Filter[tuple] - --1963949992380707373
-|   |   |       |
-|   |   |       |---Load(DummyFil:DummyLdr) - --8361563503038121624
-|   |
-|   |---MapReduce(30) - -62:
-|       |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -67
-|       |   |
-|       |   |---Local Rearrange[tuple]{Unknown} - -580704905698185869
-|       |       |
-|       |       |---Package[tuple]{Unknown} - --5758282087831209061
-|       |   Local Rearrange[tuple]{Unknown} - --6061281703859425960
-|       |   |
-|       |   |---Load(DummyFil:DummyLdr) - -5515425171581967372
+|   |   |   |---Load(DummyFil:DummyLdr) - --3833933141637499382
+|   |   |
+|   |   |---Local Rearrange[tuple]{Unknown} - -7473175511145418837
+|   |       |
+|   |       |---Filter[tuple] - --6402314745592504008
+|   |           |
+|   |           |---Load(DummyFil:DummyLdr) - --838807233869503381
 |
-|---MapReduce(-1) - -70:
-    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -76
-    |   |
-    |   |---Filter[tuple] - --1194577301115518934
-    |       |
-    |       |---Package[tuple]{Unknown} - --8027742474430787324
-    |   Union[tuple] - -71
+|---MapReduce(-1) - -6:
+    |   Store(/tmp/temp-1456742965/tmp774375955:org.apache.pig.builtin.BinStorage) - -15
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - --5768272326302808468
+    |   |---Package[tuple]{Unknown} - -990040854696137546
+    |   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -7
+    |   
+    |   Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -9
+    |
+    |---MapReduce(30) - -4:
+    |   |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -8
     |   |   |
-    |   |   |---Load(DummyFil:DummyLdr) - -6435233330554227045
-    |   |
-    |   |---Local Rearrange[tuple]{Unknown} - --2829086180578157275
-    |       |
-    |       |---Filter[tuple] - --5371705948522929752
-    |           |
-    |           |---Load(DummyFil:DummyLdr) - --4168060277593001906
\ No newline at end of file
+    |   |   |---Local Rearrange[tuple]{Unknown} - --5623550231721294978
+    |   |       |
+    |   |       |---Package[tuple]{Unknown} - --6259721534861268730
+    |   |   Local Rearrange[tuple]{Unknown} - --7212359720440714287
+    |   |   |
+    |   |   |---Load(DummyFil:DummyLdr) - -6748240903696823165
+    |
+    |---MapReduce(20) - -5:
+        |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -10
+        |   |
+        |   |---Local Rearrange[tuple]{Unknown} - -5679595123645092366
+        |       |
+        |       |---Package[tuple]{Unknown} - -8345455294066939854
+        |   Local Rearrange[tuple]{Unknown} - -2043312794799763441
+        |   |
+        |   |---Filter[tuple] - -6520791719738296531
+        |       |
+        |       |---Load(DummyFil:DummyLdr) - --5314747545448923824
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC11.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC11.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC11.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC11.gld Tue Jul 15 09:57:42 2008
@@ -1,49 +1,49 @@
-MapReduce(-1) - -87:
+MapReduce(-1) - -26:
 Reduce Plan Empty
-|   Store(DummyFil:DummyLdr) - -7219092146411102494
+|   Store(DummyFil:DummyLdr) - -2288440041178982968
 |   |
-|   |---Union[tuple] - -4266652190633805129
+|   |---Union[tuple] - -913150185705910016
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -88
+|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -27
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -90
+|       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -29
 |
-|---MapReduce(-1) - -79:
-|   |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -89
+|---MapReduce(-1) - -18:
+|   |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -28
 |   |   |
-|   |   |---Package[tuple]{Unknown} - --2530083186736419352
-|   |   Union[tuple] - -80
+|   |   |---Filter[tuple] - -3055962796670893316
+|   |       |
+|   |       |---Package[tuple]{Unknown} - -3943647700946858188
+|   |   Union[tuple] - -19
 |   |   |
-|   |   |---Local Rearrange[tuple]{Unknown} - --3008031181120208412
+|   |   |---Local Rearrange[tuple]{Unknown} - -2833954415250116776
 |   |   |   |
-|   |   |   |---Filter[tuple] - -3737603423295312892
+|   |   |   |---Filter[tuple] - --4083686173685839766
 |   |   |       |
-|   |   |       |---Load(DummyFil:DummyLdr) - --5123390619301085966
+|   |   |       |---Load(DummyFil:DummyLdr) - --5733160635931065595
 |   |   |
-|   |   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -81
-|   |
-|   |---MapReduce(-1) - -77:
-|       |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -82
-|       |   |
-|       |   |---Package[tuple]{Unknown} - --4776862487779398064
-|       |   Local Rearrange[tuple]{Unknown} - --3624738819541751735
-|       |   |
-|       |   |---Load(DummyFil:DummyLdr) - -3719468117239527682
+|   |   |---Local Rearrange[tuple]{Unknown} - -4962214768762054129
+|   |       |
+|   |       |---Load(DummyFil:DummyLdr) - --1115934782004129477
 |
-|---MapReduce(-1) - -85:
-    |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -91
+|---MapReduce(-1) - -22:
+    |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -30
     |   |
-    |   |---Filter[tuple] - -7127277012934370361
-    |       |
-    |       |---Package[tuple]{Unknown} - --1036765666284482668
-    |   Union[tuple] - -86
+    |   |---Package[tuple]{Unknown} - -6108697957385222190
+    |   Union[tuple] - -23
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - -810744320494301308
-    |   |   |
-    |   |   |---Filter[tuple] - -7860460526876129822
-    |   |       |
-    |   |       |---Load(DummyFil:DummyLdr) - --2426993543147308005
+    |   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -24
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - -3683227376238667289
+    |   |---Local Rearrange[tuple]{Unknown} - -7490898804471997380
     |       |
-    |       |---Load(DummyFil:DummyLdr) - --5111685507913827932
\ No newline at end of file
+    |       |---Filter[tuple] - -3720949273928245639
+    |           |
+    |           |---Load(DummyFil:DummyLdr) - --8248982303554009
+    |
+    |---MapReduce(-1) - -20:
+        |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -25
+        |   |
+        |   |---Package[tuple]{Unknown} - --8723304958939002625
+        |   Local Rearrange[tuple]{Unknown} - -689137294940608050
+        |   |
+        |   |---Load(DummyFil:DummyLdr) - -3742910951635599848
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC12.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC12.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC12.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC12.gld Tue Jul 15 09:57:42 2008
@@ -1,23 +1,23 @@
-MapReduce(-1) - -98:
-|   Store(DummyFil:DummyLdr) - -913122466036599874
+MapReduce(-1) - -37:
+|   Store(DummyFil:DummyLdr) - -340188275237520979
 |   |
-|   |---Package[tuple]{Unknown} - -8515950928528386562
-|   Union[tuple] - -99
+|   |---Package[tuple]{Unknown} - --1008013035164995818
+|   Union[tuple] - -38
 |   |
-|   |---Local Rearrange[tuple]{Unknown} - --7092666336106039025
+|   |---Local Rearrange[tuple]{Unknown} - -3900012572437255236
 |   |   |
-|   |   |---Filter[tuple] - --506205335735031954
+|   |   |---Filter[tuple] - -7391599663069134339
 |   |       |
-|   |       |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -94
+|   |       |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -33
 |   |
-|   |---Local Rearrange[tuple]{Unknown} - -8957152010410190649
+|   |---Local Rearrange[tuple]{Unknown} - -8760996681222683693
 |       |
-|       |---Filter[tuple] - --2465563180022385815
+|       |---Filter[tuple] - -956528893337238225
 |           |
-|           |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -96
+|           |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -35
 |
-|---MapReduce(-1) - -92:
+|---MapReduce(-1) - -31:
     Reduce Plan Empty
-    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -93
+    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -32
     |   |
-    |   |---Load(DummyFil:DummyLdr) - --90170124379688035
\ No newline at end of file
+    |   |---Load(DummyFil:DummyLdr) - --5840807574562740596
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC13.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC13.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC13.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC13.gld Tue Jul 15 09:57:42 2008
@@ -1,37 +1,37 @@
-MapReduce(-1) - -113:
+MapReduce(-1) - -52:
 Reduce Plan Empty
-|   Store(DummyFil:DummyLdr) - -3777104251028634198
+|   Store(DummyFil:DummyLdr) - -8531323295221133204
 |   |
-|   |---Union[tuple] - -4636081797237532954
+|   |---Union[tuple] - -3515731110955624708
 |       |
-|       |---Filter[tuple] - --4160312061837144266
+|       |---Filter[tuple] - -1351115285619019432
 |       |   |
-|       |   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -109
+|       |   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -48
 |       |
-|       |---Filter[tuple] - -2997708366016271267
+|       |---Filter[tuple] - -6869916393445222618
 |           |
-|           |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -111
+|           |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -50
 |
-|---MapReduce(-1) - -106:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -108
+|---MapReduce(-1) - -45:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -47
     |   |
-    |   |---Package[tuple]{Unknown} - --1195368474144577564
-    |   Union[tuple] - -107
+    |   |---Package[tuple]{Unknown} - --1596982214453270401
+    |   Union[tuple] - -46
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - --8706893458091286727
+    |   |---Local Rearrange[tuple]{Unknown} - --6728652914243238289
     |   |   |
-    |   |   |---Filter[tuple] - -6693653468874366867
+    |   |   |---Filter[tuple] - --3509976263222494134
     |   |       |
-    |   |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -102
+    |   |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -41
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - --8512881738335808286
+    |   |---Local Rearrange[tuple]{Unknown} - --4071762447953696591
     |       |
-    |       |---Filter[tuple] - --3347320220748403847
+    |       |---Filter[tuple] - -7123161826157220327
     |           |
-    |           |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -104
+    |           |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -43
     |
-    |---MapReduce(-1) - -100:
+    |---MapReduce(-1) - -39:
         Reduce Plan Empty
-        |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -101
+        |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -40
         |   |
-        |   |---Load(DummyFil:DummyLdr) - -5969405952749301492
\ No newline at end of file
+        |   |---Load(DummyFil:DummyLdr) - --5941754500044119766
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC14.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC14.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC14.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC14.gld Tue Jul 15 09:57:42 2008
@@ -1,65 +1,65 @@
-MapReduce(100) - -135:
+MapReduce(100) - -74:
 Reduce Plan Empty
-|   Store(DummyFil:DummyLdr) - --7335024873119453444
+|   Store(DummyFil:DummyLdr) - -4747874603499386537
 |   |
-|   |---Union[tuple] - --4213306726552862637
+|   |---Union[tuple] - --7231511798448197372
 |       |
-|       |---Filter[tuple] - --4248200967728536480
+|       |---Filter[tuple] - --7091196022628815072
 |       |   |
-|       |   |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -131
+|       |   |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -70
 |       |
-|       |---Filter[tuple] - -4589138876054328603
+|       |---Filter[tuple] - -5984722006991913573
 |           |
-|           |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -133
+|           |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -72
 |
-|---MapReduce(40) - -128:
-    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -130
+|---MapReduce(40) - -67:
+    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -69
     |   |
-    |   |---Package[tuple]{Unknown} - --1848504978980807369
-    |   Union[tuple] - -129
+    |   |---Package[tuple]{Unknown} - -8067897495211048679
+    |   Union[tuple] - -68
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - --5908426805312852480
+    |   |---Local Rearrange[tuple]{Unknown} - --8754957318949788629
     |   |   |
-    |   |   |---Filter[tuple] - --8479692259657755370
+    |   |   |---Filter[tuple] - -551977918718096509
     |   |       |
-    |   |       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -119
+    |   |       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -58
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - -5965044993061572808
+    |   |---Local Rearrange[tuple]{Unknown} - -6411189422016119355
     |   |   |
-    |   |   |---Filter[tuple] - --2247078527532208396
+    |   |   |---Filter[tuple] - -2056844330428347996
     |   |       |
-    |   |       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -121
+    |   |       |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -63
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - -8914265632748254170
+    |   |---Local Rearrange[tuple]{Unknown} - -7225341208466719305
     |       |
-    |       |---Filter[tuple] - -9193928674704944093
+    |       |---Filter[tuple] - --7987250529196129674
     |           |
-    |           |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -126
+    |           |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -65
     |
-    |---MapReduce(20) - -117:
+    |---MapReduce(10) - -56:
     |   Reduce Plan Empty
-    |   |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -118
+    |   |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -57
     |   |   |
-    |   |   |---Filter[tuple] - --2771799342014688397
+    |   |   |---Filter[tuple] - -2351257193210757136
     |   |       |
-    |   |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -116
+    |   |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -55
     |   |
-    |   |---MapReduce(-1) - -114:
+    |   |---MapReduce(-1) - -53:
     |       Reduce Plan Empty
-    |       |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -115
+    |       |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -54
     |       |   |
-    |       |   |---Load(DummyFil:DummyLdr) - -7955165653808522445
+    |       |   |---Load(DummyFil:DummyLdr) - -728209667098611077
     |
-    |---MapReduce(10) - -124:
+    |---MapReduce(20) - -61:
         Reduce Plan Empty
-        |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -125
+        |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -62
         |   |
-        |   |---Filter[tuple] - --997494992827460962
+        |   |---Filter[tuple] - --1369132270304461038
         |       |
-        |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -123
+        |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -60
         |
-        |---MapReduce(-1) - -114:
+        |---MapReduce(-1) - -53:
             Reduce Plan Empty
-            |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -115
+            |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -54
             |   |
-            |   |---Load(DummyFil:DummyLdr) - -7955165653808522445
\ No newline at end of file
+            |   |---Load(DummyFil:DummyLdr) - -728209667098611077
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC15.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC15.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC15.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC15.gld Tue Jul 15 09:57:42 2008
@@ -1,89 +1,89 @@
-MapReduce(1,GFCross) - -165:
-|   Store(DummyFil:DummyLdr) - -6139496040975471496
+MapReduce(1,GFCross) - -156:
+|   Store(DummyFil:DummyLdr) - --5683415113785058706
 |   |
-|   |---New For Each(false)[tuple] - -5878058249424113280
+|   |---New For Each(false)[tuple] - --8002381389674382470
 |       |   |
-|       |   POUserFunc(org.apache.pig.impl.builtin.GFCross)[tuple] - -7085374960058856883
+|       |   POUserFunc(org.apache.pig.impl.builtin.GFCross)[tuple] - -727770031531364881
 |       |   |
-|       |   |---Project[tuple][*] - -7856319821130535798
+|       |   |---Project[tuple][*] - -7128285064986147947
 |       |
-|       |---Package[tuple]{Unknown} - -7398260302074824818
-|   Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -164
+|       |---Package[tuple]{Unknown} - --885269774183211482
+|   Local Rearrange[tuple]{Unknown} - --776319888013965510
+|   |
+|   |---Load(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -155
 |
-|---MapReduce(1,AVG) - -162:
-    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -163
+|---MapReduce(1,AVG) - -153:
+    |   Store(/tmp/temp-1456742965/tmp-586682361:org.apache.pig.builtin.BinStorage) - -154
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - -7973970339130605847
+    |   |---New For Each(false)[tuple] - -7965768498188214494
+    |       |   |
+    |       |   POUserFunc(org.apache.pig.builtin.AVG)[tuple] - --4867358672373443663
+    |       |   |
+    |       |   |---Project[tuple][*] - -31712229583931650
     |       |
-    |       |---New For Each(false)[tuple] - --6860857982727545176
-    |           |   |
-    |           |   POUserFunc(org.apache.pig.builtin.AVG)[tuple] - -2082992246427879202
-    |           |   |
-    |           |   |---Project[tuple][*] - -737965411848544085
-    |           |
-    |           |---Package[tuple]{Unknown} - --3148893660811981376
-    |   Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -161
+    |       |---Package[tuple]{Unknown} - --7335024873119453444
+    |   Local Rearrange[tuple]{Unknown} - -4589138876054328603
+    |   |
+    |   |---Load(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -152
     |
-    |---MapReduce(20,TestMRCompiler$WeirdComparator,FindQuantiles,SUM) - -154:
-        |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -160
+    |---MapReduce(20,TestMRCompiler$WeirdComparator,FindQuantiles,SUM) - -145:
+        |   Store(/tmp/temp-1456742965/tmp-26634357:org.apache.pig.builtin.BinStorage) - -151
         |   |
-        |   |---Local Rearrange[tuple]{Unknown} - --7926255547935388282
+        |   |---New For Each(false,false)[tuple] - --4248200967728536480
+        |       |   |
+        |       |   POUserFunc(org.apache.pig.impl.builtin.FindQuantiles)[tuple] - -8767305735755351861
+        |       |   |
+        |       |   |---Project[tuple][*] - --5908426805312852480
+        |       |   |
+        |       |   POUserFunc(org.apache.pig.builtin.SUM)[tuple] - -4744867519639813717
+        |       |   |
+        |       |   |---Project[tuple][*] - --1848504978980807369
         |       |
-        |       |---New For Each(false,false)[tuple] - -8639002377389679348
-        |           |   |
-        |           |   POUserFunc(org.apache.pig.impl.builtin.FindQuantiles)[tuple] - -727770031531364881
-        |           |   |
-        |           |   |---Project[tuple][*] - -7128285064986147947
-        |           |   |
-        |           |   POUserFunc(org.apache.pig.builtin.SUM)[tuple] - --5683415113785058706
+        |       |---New For Each(true)[tuple] - -150
         |           |   |
-        |           |   |---Project[tuple][*] - --8002381389674382470
+        |           |   Project[bag][1] - -149
         |           |
-        |           |---New For Each(true)[tuple] - -159
-        |               |   |
-        |               |   Project[bag][1] - -158
-        |               |
-        |               |---Package[tuple]{tuple} - -157
-        |   Local Rearrange[tuple]{tuple} - -156
+        |           |---Package[tuple]{tuple} - -148
+        |   Local Rearrange[tuple]{tuple} - -147
         |   |   |
-        |   |   Project[tuple][*] - -155
+        |   |   Project[tuple][*] - -146
         |   |
-        |   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -153
+        |   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -144
         |
-        |---MapReduce(-1,FindQuantiles,TestMRCompiler$WeirdComparator) - -139:
-            |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -152
+        |---MapReduce(-1,FindQuantiles,TestMRCompiler$WeirdComparator) - -130:
+            |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -143
             |   |
-            |   |---New For Each(false)[tuple] - -151
+            |   |---New For Each(false)[tuple] - -142
             |       |   |
-            |       |   POUserFunc(org.apache.pig.impl.builtin.FindQuantiles)[tuple] - -150
+            |       |   POUserFunc(org.apache.pig.impl.builtin.FindQuantiles)[tuple] - -141
             |       |   |
-            |       |   |---Project[tuple][*] - -149
+            |       |   |---Project[tuple][*] - -140
             |       |
-            |       |---New For Each(false,false)[tuple] - -148
+            |       |---New For Each(false,false)[tuple] - -139
             |           |   |
-            |           |   Constant(20) - -147
+            |           |   Constant(20) - -138
             |           |   |
-            |           |   POSort[bag](org.apache.pig.test.TestMRCompiler$WeirdComparator) - --776319888013965510
+            |           |   POSort[bag](org.apache.pig.test.TestMRCompiler$WeirdComparator) - --8479692259657755370
             |           |   |   |
-            |           |   |   Project[tuple][*] - -146
+            |           |   |   Project[tuple][*] - -137
             |           |   |
-            |           |   |---Project[tuple][1] - -145
+            |           |   |---Project[tuple][1] - -136
             |           |
-            |           |---Package[tuple]{chararray} - -144
-            |   Local Rearrange[tuple]{chararray} - -143
+            |           |---Package[tuple]{chararray} - -135
+            |   Local Rearrange[tuple]{chararray} - -134
             |   |   |
-            |   |   Constant(all) - -142
+            |   |   Constant(all) - -133
             |   |
-            |   |---New For Each(true)[tuple] - -141
+            |   |---New For Each(true)[tuple] - -132
             |       |   |
-            |       |   Project[tuple][*] - -140
+            |       |   Project[tuple][*] - -131
             |       |
-            |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.impl.builtin.RandomSampleLoader) - -138
+            |       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.impl.builtin.RandomSampleLoader) - -129
             |
-            |---MapReduce(-1) - -136:
+            |---MapReduce(-1) - -127:
                 Reduce Plan Empty
-                |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -137
+                |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -128
                 |   |
-                |   |---Filter[tuple] - --4867358672373443663
+                |   |---Filter[tuple] - -9193928674704944093
                 |       |
-                |       |---Load(DummyFil:DummyLdr) - -31712229583931650
\ No newline at end of file
+                |       |---Load(DummyFil:DummyLdr) - --4238531569995320849
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC16.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC16.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC16.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC16.gld Tue Jul 15 09:57:42 2008
@@ -1,37 +1,37 @@
-MapReduce(-1) - -179:
-|   Store(DummyFil:DummyLdr) - --2783416442434419494
+MapReduce(-1) - -170:
+|   Store(DummyFil:DummyLdr) - -7973970339130605847
 |   |
-|   |---New For Each(false)[bag] - -182
+|   |---New For Each(false)[bag] - -173
 |       |   |
-|       |   Project[tuple][0] - -181
+|       |   Project[tuple][0] - -172
 |       |
-|       |---Package[tuple]{tuple} - -180
-|   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -178
+|       |---Package[tuple]{tuple} - -171
+|   Local Rearrange[tuple]{tuple} - -167
+|   |   |
+|   |   Project[tuple][*] - -166
+|   |
+|   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -169
 |
-|---MapReduce(-1) - -174:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -177
+|---MapReduce(-1) - -165:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -168
+    |   |
+    |   |---Package[tuple]{Unknown} - -2082992246427879202
+    |   Local Rearrange[tuple]{Unknown} - --3148893660811981376
     |   |
-    |   |---Local Rearrange[tuple]{tuple} - -176
-    |       |   |
-    |       |   Project[tuple][*] - -175
-    |       |
-    |       |---Package[tuple]{Unknown} - -7714111612268358662
-    |   Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -173
+    |   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -164
     |
-    |---MapReduce(-1) - -166:
-        |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -172
+    |---MapReduce(-1) - -157:
+        |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -163
         |   |
-        |   |---Local Rearrange[tuple]{Unknown} - -5321755951016030071
+        |   |---New For Each(false)[bag] - -162
+        |       |   |
+        |       |   Project[tuple][0] - -161
         |       |
-        |       |---New For Each(false)[bag] - -171
-        |           |   |
-        |           |   Project[tuple][0] - -170
-        |           |
-        |           |---Package[tuple]{tuple} - -169
-        |   Local Rearrange[tuple]{tuple} - -168
+        |       |---Package[tuple]{tuple} - -160
+        |   Local Rearrange[tuple]{tuple} - -159
         |   |   |
-        |   |   Project[tuple][*] - -167
+        |   |   Project[tuple][*] - -158
         |   |
-        |   |---Filter[tuple] - -2197807331204639125
+        |   |---Filter[tuple] - --7926255547935388282
         |       |
-        |       |---Load(DummyFil:DummyLdr) - --8447156979210379761
\ No newline at end of file
+        |       |---Load(DummyFil:DummyLdr) - -8639002377389679348
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC2.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC2.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC2.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC2.gld Tue Jul 15 09:57:42 2008
@@ -1,25 +1,25 @@
-MapReduce(-1) - -6:
+MapReduce(-1) - -81:
 Reduce Plan Empty
-|   Store(DummyFil:DummyLdr) - -8637487025682524492
+|   Store(DummyFil:DummyLdr) - -7034055442337723447
 |   |
-|   |---Union[tuple] - --2655303127943013956
+|   |---Union[tuple] - -2910267342587130351
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -7
+|       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -82
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -9
+|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -84
 |
-|---MapReduce(-1) - -4:
-|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -8
+|---MapReduce(-1) - -79:
+|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -83
 |   |   |
-|   |   |---Package[tuple]{Unknown} - --8216215966586363937
-|   |   Local Rearrange[tuple]{Unknown} - -5679595123645092366
+|   |   |---Package[tuple]{Unknown} - -370083002387034672
+|   |   Local Rearrange[tuple]{Unknown} - -6305091296204163466
 |   |   |
-|   |   |---Load(DummyFil:DummyLdr) - -990040854696137546
+|   |   |---Load(DummyFil:DummyLdr) - -1236980712440527800
 |
-|---MapReduce(-1) - -5:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -10
+|---MapReduce(-1) - -80:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -85
     |   |
-    |   |---Package[tuple]{Unknown} - -7473175511145418837
-    |   Local Rearrange[tuple]{Unknown} - --3833933141637499382
+    |   |---Package[tuple]{Unknown} - -7123718043666602037
+    |   Local Rearrange[tuple]{Unknown} - --6380139708906526553
     |   |
-    |   |---Load(DummyFil:DummyLdr) - --838807233869503381
\ No newline at end of file
+    |   |---Load(DummyFil:DummyLdr) - --2136667002319908593
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC3.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC3.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC3.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC3.gld Tue Jul 15 09:57:42 2008
@@ -1,29 +1,29 @@
-MapReduce(-1) - -14:
+MapReduce(-1) - -89:
 Reduce Plan Empty
-|   Store(DummyFil:DummyLdr) - -5879589559059467399
+|   Store(DummyFil:DummyLdr) - --5190071275724378681
 |   |
-|   |---Union[tuple] - -6108697957385222190
+|   |---Union[tuple] - --3396897091865664764
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -15
+|       |---Filter[tuple] - --6957591191795645940
+|       |   |
+|       |   |---Load(DummyFil:DummyLdr) - -5515425171581967372
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -17
+|       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -90
 |       |
-|       |---Filter[tuple] - -689137294940608050
-|           |
-|           |---Load(DummyFil:DummyLdr) - --8723304958939002625
+|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -92
 |
-|---MapReduce(-1) - -11:
-|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -16
+|---MapReduce(-1) - -87:
+|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -91
 |   |   |
-|   |   |---Package[tuple]{Unknown} - --2449990780759860228
-|   |   Local Rearrange[tuple]{Unknown} - --171729478481529687
+|   |   |---Package[tuple]{Unknown} - --6061281703859425960
+|   |   Local Rearrange[tuple]{Unknown} - --1158897849427419546
 |   |   |
-|   |   |---Load(DummyFil:DummyLdr) - -3527883492192621891
+|   |   |---Load(DummyFil:DummyLdr) - -3709512757404691843
 |
-|---MapReduce(-1) - -12:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -18
+|---MapReduce(-1) - -88:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -93
     |   |
-    |   |---Package[tuple]{Unknown} - --961974426096759446
-    |   Local Rearrange[tuple]{Unknown} - -3720949273928245639
+    |   |---Package[tuple]{Unknown} - --2057425961601007773
+    |   Local Rearrange[tuple]{Unknown} - --8361563503038121624
     |   |
-    |   |---Load(DummyFil:DummyLdr) - --8248982303554009
\ No newline at end of file
+    |   |---Load(DummyFil:DummyLdr) - -7506868571066332964
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC4.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC4.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC4.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC4.gld Tue Jul 15 09:57:42 2008
@@ -1,33 +1,29 @@
-MapReduce(-1) - -23:
+MapReduce(-1) - -89:
 Reduce Plan Empty
-|   Store(DummyFil:DummyLdr) - -956528893337238225
+|   Store(DummyFil:DummyLdr) - --5190071275724378681
 |   |
-|   |---Union[tuple] - -3055962796670893316
+|   |---Union[tuple] - --3396897091865664764
 |       |
-|       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -24
-|       |
-|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -26
-|       |
-|       |---Filter[tuple] - -2288440041178982968
+|       |---Filter[tuple] - --6957591191795645940
 |       |   |
-|       |   |---Load(DummyFil:DummyLdr) - -913150185705910016
+|       |   |---Load(DummyFil:DummyLdr) - -5515425171581967372
+|       |
+|       |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -90
 |       |
-|       |---Filter[tuple] - -4024598850351165272
-|           |
-|           |---Load(DummyFil:DummyLdr) - --5840807574562740596
+|       |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -92
 |
-|---MapReduce(-1) - -19:
-|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -25
+|---MapReduce(-1) - -87:
+|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -91
 |   |   |
-|   |   |---Package[tuple]{Unknown} - --4083686173685839766
-|   |   Local Rearrange[tuple]{Unknown} - --5733160635931065595
+|   |   |---Package[tuple]{Unknown} - --6061281703859425960
+|   |   Local Rearrange[tuple]{Unknown} - --1158897849427419546
 |   |   |
-|   |   |---Load(DummyFil:DummyLdr) - -3742910951635599848
+|   |   |---Load(DummyFil:DummyLdr) - -3709512757404691843
 |
-|---MapReduce(-1) - -21:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -27
+|---MapReduce(-1) - -88:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -93
     |   |
-    |   |---Package[tuple]{Unknown} - -3943647700946858188
-    |   Local Rearrange[tuple]{Unknown} - -4962214768762054129
+    |   |---Package[tuple]{Unknown} - --2057425961601007773
+    |   Local Rearrange[tuple]{Unknown} - --8361563503038121624
     |   |
-    |   |---Load(DummyFil:DummyLdr) - -2833954415250116776
\ No newline at end of file
+    |   |---Load(DummyFil:DummyLdr) - -7506868571066332964
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC5.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC5.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC5.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC5.gld Tue Jul 15 09:57:42 2008
@@ -1,13 +1,13 @@
-MapReduce(-1) - -30:
+MapReduce(-1) - -96:
 Reduce Plan Empty
-|   Store(DummyFil:DummyLdr) - -340188275237520979
+|   Store(DummyFil:DummyLdr) - --2829086180578157275
 |   |
-|   |---Union[tuple] - --1008013035164995818
+|   |---Union[tuple] - --5371705948522929752
 |       |
-|       |---Filter[tuple] - -2626971498845605352
+|       |---Filter[tuple] - -6435233330554227045
 |       |   |
-|       |   |---Load(DummyFil:DummyLdr) - -3900012572437255236
+|       |   |---Load(DummyFil:DummyLdr) - --4168060277593001906
 |       |
-|       |---Filter[tuple] - -8760996681222683693
+|       |---Filter[tuple] - -7483213803049293823
 |           |
-|           |---Load(DummyFil:DummyLdr) - -7391599663069134339
\ No newline at end of file
+|           |---Load(DummyFil:DummyLdr) - -580704905698185869
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC6.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC6.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC6.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC6.gld Tue Jul 15 09:57:42 2008
@@ -1,27 +1,27 @@
-MapReduce(-1) - -33:
-|   Store(DummyFil:DummyLdr) - -8531323295221133204
+MapReduce(-1) - -99:
+|   Store(DummyFil:DummyLdr) - --3624738819541751735
 |   |
-|   |---Package[tuple]{Unknown} - -3515731110955624708
-|   Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -34
+|   |---Package[tuple]{Unknown} - --4776862487779398064
+|   Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -100
 |   
-|   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -36
+|   Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -102
 |
-|---MapReduce(-1) - -31:
-|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -35
+|---MapReduce(-1) - -97:
+|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -101
 |   |   |
-|   |   |---Local Rearrange[tuple]{Unknown} - --7797378030368227515
+|   |   |---Local Rearrange[tuple]{Unknown} - --3671186819751472084
 |   |       |
-|   |       |---Package[tuple]{Unknown} - --3509976263222494134
-|   |   Local Rearrange[tuple]{Unknown} - --6487150145665307346
+|   |       |---Package[tuple]{Unknown} - -3737603423295312892
+|   |   Local Rearrange[tuple]{Unknown} - --2325244147060806375
 |   |   |
-|   |   |---Load(DummyFil:DummyLdr) - --5941754500044119766
+|   |   |---Load(DummyFil:DummyLdr) - --8240903279973257769
 |
-|---MapReduce(-1) - -32:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -37
+|---MapReduce(-1) - -98:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -103
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - -6869916393445222618
+    |   |---Local Rearrange[tuple]{Unknown} - --3008031181120208412
     |       |
-    |       |---Package[tuple]{Unknown} - --1596982214453270401
-    |   Local Rearrange[tuple]{Unknown} - --6728652914243238289
+    |       |---Package[tuple]{Unknown} - --1194577301115518934
+    |   Local Rearrange[tuple]{Unknown} - --2626287810923037076
     |   |
-    |   |---Load(DummyFil:DummyLdr) - --4071762447953696591
\ No newline at end of file
+    |   |---Load(DummyFil:DummyLdr) - --5768272326302808468
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC7.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC7.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC7.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC7.gld Tue Jul 15 09:57:42 2008
@@ -1,33 +1,33 @@
-MapReduce(-1) - -41:
-|   Store(DummyFil:DummyLdr) - -8205341881606740757
+MapReduce(-1) - -107:
+|   Store(DummyFil:DummyLdr) - --506205335735031954
 |   |
-|   |---Package[tuple]{Unknown} - -8067897495211048679
-|   Union[tuple] - -42
+|   |---Package[tuple]{Unknown} - --2216790928607759809
+|   Union[tuple] - -108
 |   |
-|   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -43
+|   |---Filter[tuple] - --90170124379688035
+|   |   |
+|   |   |---Load(DummyFil:DummyLdr) - -7219092146411102494
 |   |
-|   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -45
+|   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -109
 |   |
-|   |---Filter[tuple] - -8816201798088731301
-|       |
-|       |---Load(DummyFil:DummyLdr) - -6411189422016119355
+|   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -111
 |
-|---MapReduce(-1) - -38:
-|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -44
+|---MapReduce(-1) - -104:
+|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -110
 |   |   |
-|   |   |---Local Rearrange[tuple]{Unknown} - --8754957318949788629
+|   |   |---Local Rearrange[tuple]{Unknown} - --1036765666284482668
 |   |       |
-|   |       |---Package[tuple]{Unknown} - --7987250529196129674
-|   |   Local Rearrange[tuple]{Unknown} - --7150209077138491584
+|   |       |---Package[tuple]{Unknown} - --5111685507913827932
+|   |   Local Rearrange[tuple]{Unknown} - -3719468117239527682
 |   |   |
-|   |   |---Load(DummyFil:DummyLdr) - --1607475648664293401
+|   |   |---Load(DummyFil:DummyLdr) - -1761200043915140459
 |
-|---MapReduce(-1) - -39:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -46
+|---MapReduce(-1) - -106:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -112
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - -2056844330428347996
+    |   |---Local Rearrange[tuple]{Unknown} - -7127277012934370361
     |       |
-    |       |---Package[tuple]{Unknown} - --1369132270304461038
-    |   Local Rearrange[tuple]{Unknown} - --3476413267732334825
+    |       |---Package[tuple]{Unknown} - --8622295867288126988
+    |   Local Rearrange[tuple]{Unknown} - -810744320494301308
     |   |
-    |   |---Load(DummyFil:DummyLdr) - -728209667098611077
\ No newline at end of file
+    |   |---Load(DummyFil:DummyLdr) - -7860460526876129822
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC8.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC8.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC8.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC8.gld Tue Jul 15 09:57:42 2008
@@ -1,37 +1,37 @@
-MapReduce(-1) - -51:
-|   Store(DummyFil:DummyLdr) - -370083002387034672
+MapReduce(-1) - -117:
+|   Store(DummyFil:DummyLdr) - --4160312061837144266
 |   |
-|   |---Package[tuple]{Unknown} - --7096529877081178520
-|   Union[tuple] - -52
+|   |---Package[tuple]{Unknown} - -2997708366016271267
+|   Union[tuple] - -118
 |   |
-|   |---Filter[tuple] - --7868505214447593853
-|   |   |
-|   |   |---Load(DummyFil:DummyLdr) - --5177745552827005198
+|   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -119
 |   |
-|   |---Load(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -53
+|   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -121
 |   |
-|   |---Load(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -55
+|   |---Filter[tuple] - -3615014757987062850
+|   |   |
+|   |   |---Load(DummyFil:DummyLdr) - --8706893458091286727
 |   |
-|   |---Filter[tuple] - -6305091296204163466
+|   |---Filter[tuple] - -9019572209815819418
 |       |
-|       |---Load(DummyFil:DummyLdr) - -1236980712440527800
+|       |---Load(DummyFil:DummyLdr) - --1195368474144577564
 |
-|---MapReduce(-1) - -49:
-|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -54
+|---MapReduce(-1) - -113:
+|   |   Store(/tmp/temp-1456742965/tmp-1456742965:org.apache.pig.builtin.BinStorage) - -120
 |   |   |
-|   |   |---Local Rearrange[tuple]{Unknown} - -6790164331631653200
+|   |   |---Local Rearrange[tuple]{Unknown} - --3347320220748403847
 |   |       |
-|   |       |---Package[tuple]{Unknown} - -4721502244557927278
-|   |   Local Rearrange[tuple]{Unknown} - --7681398237172009051
+|   |       |---Package[tuple]{Unknown} - --203604194309414000
+|   |   Local Rearrange[tuple]{Unknown} - --7092666336106039025
 |   |   |
-|   |   |---Load(DummyFil:DummyLdr) - -6620645493024302760
+|   |   |---Load(DummyFil:DummyLdr) - --2465563180022385815
 |
-|---MapReduce(-1) - -50:
-    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -56
+|---MapReduce(-1) - -115:
+    |   Store(/tmp/temp-1456742965/tmp2077335416:org.apache.pig.builtin.BinStorage) - -122
     |   |
-    |   |---Local Rearrange[tuple]{Unknown} - -8729990799028586395
+    |   |---Local Rearrange[tuple]{Unknown} - -6693653468874366867
     |       |
-    |       |---Package[tuple]{Unknown} - -4747874603499386537
-    |   Local Rearrange[tuple]{Unknown} - -5984722006991913573
+    |       |---Package[tuple]{Unknown} - -8554221887998033529
+    |   Local Rearrange[tuple]{Unknown} - -913122466036599874
     |   |
-    |   |---Load(DummyFil:DummyLdr) - --7091196022628815072
\ No newline at end of file
+    |   |---Load(DummyFil:DummyLdr) - -8515950928528386562
\ No newline at end of file

Modified: incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC9.gld
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC9.gld?rev=676968&r1=676967&r2=676968&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC9.gld (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/data/GoldenFiles/MRC9.gld Tue Jul 15 09:57:42 2008
@@ -1,13 +1,13 @@
-MapReduce(-1) - -59:
-|   Store(DummyFil:DummyLdr) - -7506868571066332964
+MapReduce(-1) - -125:
+|   Store(DummyFil:DummyLdr) - --6047015131487356012
 |   |
-|   |---Package[tuple]{Unknown} - -7034055442337723447
-|   Union[tuple] - -60
+|   |---Package[tuple]{Unknown} - --2771799342014688397
+|   Union[tuple] - -126
 |   |
-|   |---Filter[tuple] - -2468262982799101733
+|   |---Filter[tuple] - --997494992827460962
 |   |   |
-|   |   |---Load(DummyFil:DummyLdr) - --6380139708906526553
+|   |   |---Load(DummyFil:DummyLdr) - --1524666447399813617
 |   |
-|   |---Filter[tuple] - -2910267342587130351
+|   |---Filter[tuple] - -7955165653808522445
 |       |
-|       |---Load(DummyFil:DummyLdr) - -7123718043666602037
\ No newline at end of file
+|       |---Load(DummyFil:DummyLdr) - -3777104251028634198
\ No newline at end of file