You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ro...@apache.org on 2014/12/09 21:40:04 UTC

svn commit: r1644173 - in /pig/trunk: ./ src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/ src/org/apache/pig/backend/hadoop/executionengine/tez/ src/org/apache/pig/backend/hadoop/executionengine/tez/plan/ src/org/apache/pig/backen...

Author: rohini
Date: Tue Dec  9 20:40:03 2014
New Revision: 1644173

URL: http://svn.apache.org/r1644173
Log:
PIG-4338: Fix test failures with JDK8 (rohini)

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/build.xml
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/TezPrinter.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/operator/POShuffledValueInputTez.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/operator/POValueOutputTez.java
    pig/trunk/src/org/apache/pig/impl/plan/DependencyOrderWalker.java
    pig/trunk/test/org/apache/pig/builtin/TestAvroStorage.java
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Limit-2.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Limit-3.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-1-OPTOFF.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-2-OPTOFF.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-2.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-3.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-4-OPTOFF.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-5-OPTOFF.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Order-1.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Rank-1.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Rank-2.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-SkewJoin-1.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-1-OPTOFF.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-10-OPTOFF.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-2-OPTOFF.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-2.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-4-OPTOFF.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-6-OPTOFF.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-6.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-7-OPTOFF.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-7.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-8-OPTOFF.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-8.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-9-OPTOFF.gld
    pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-9.gld
    pig/trunk/test/org/apache/pig/tez/TestTezJobControlCompiler.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Tue Dec  9 20:40:03 2014
@@ -28,6 +28,8 @@ PIG-4333: Split BigData tests into multi
  
 BUG FIXES
 
+PIG-4338: Fix test failures with JDK8 (rohini)
+
 PIG-4351: TestPigRunner.simpleTest2 fail on trunk (daijy)
 
 PIG-4350: Port local mode tests to Tez - part2 (daijy)

Modified: pig/trunk/build.xml
URL: http://svn.apache.org/viewvc/pig/trunk/build.xml?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/build.xml (original)
+++ pig/trunk/build.xml Tue Dec  9 20:40:03 2014
@@ -61,7 +61,7 @@
     <property name="javac.debug" value="on" />
     <property name="javac.optimize" value="on" />
     <property name="javac.deprecation" value="off" />
-    <property name="javac.version" value="1.5" />
+    <property name="javac.version" value="1.7" />
     <property name="javac.args" value="" />
     <condition property="javac.args.warnings" value="-Xmaxwarns 1000000 -Xlint -Xlint:-deprecation" else="-Xmaxwarns 1000000">
         <isset property="all.warnings" />

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java Tue Dec  9 20:40:03 2014
@@ -23,6 +23,7 @@ import java.io.OutputStream;
 import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -229,7 +230,10 @@ public class PhysicalPlan extends Operat
         // clone.
         Map<PhysicalOperator, PhysicalOperator> matches =
             new HashMap<PhysicalOperator, PhysicalOperator>(mOps.size());
-        for (PhysicalOperator op : mOps.keySet()) {
+        // Sorting just so that explain output (scope ids) is same in jdk7 and jdk8
+        List<PhysicalOperator> opsToClone = new ArrayList<PhysicalOperator>(mOps.keySet());
+        Collections.sort(opsToClone);
+        for (PhysicalOperator op : opsToClone) {
             PhysicalOperator c = op.clone();
             clone.add(c);
             if (opmap != null)

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java Tue Dec  9 20:40:03 2014
@@ -98,11 +98,15 @@ public class TezLauncher extends Launche
                 .setUncaughtExceptionHandler(new JobControlThreadExceptionHandler())
                 .build();
         }
-        executor = Executors.newSingleThreadExecutor(namedThreadFactory);
     }
 
     @Override
     public PigStats launchPig(PhysicalPlan php, String grpName, PigContext pc) throws Exception {
+        synchronized (this) {
+            if (executor == null) {
+                executor = Executors.newSingleThreadExecutor(namedThreadFactory);
+            }
+        }
         if (pc.getExecType().isLocal()) {
             pc.getProperties().setProperty(TezConfiguration.TEZ_LOCAL_MODE, "true");
             pc.getProperties().setProperty(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, "true");

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/TezPrinter.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/TezPrinter.java?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/TezPrinter.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/TezPrinter.java Tue Dec  9 20:40:03 2014
@@ -18,8 +18,9 @@
 package org.apache.pig.backend.hadoop.executionengine.tez.plan;
 
 import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
-import java.util.Map.Entry;
 
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
@@ -42,7 +43,7 @@ public class TezPrinter extends TezOpPla
      * @param plan tez plan to print
      */
     public TezPrinter(PrintStream ps, TezOperPlan plan) {
-        super(plan, new DependencyOrderWalker<TezOperator, TezOperPlan>(plan));
+        super(plan, new DependencyOrderWalker<TezOperator, TezOperPlan>(plan, true));
         mStream = ps;
     }
 
@@ -62,13 +63,16 @@ public class TezPrinter extends TezOpPla
             mStream.println("Tez vertex " + tezOper.getOperatorKey().toString());
         }
         if (tezOper.inEdges.size() > 0) {
-            for (Entry<OperatorKey, TezEdgeDescriptor> inEdge : tezOper.inEdges.entrySet()) {
+            List<OperatorKey> inEdges = new ArrayList<OperatorKey>(tezOper.inEdges.keySet());
+            Collections.sort(inEdges);
+            for (OperatorKey inEdge : inEdges) {
                 //TODO: Print other edge properties like custom partitioner
-                if (!inEdge.getValue().combinePlan.isEmpty()) {
-                    mStream.println("# Combine plan on edge <" + inEdge.getKey() + ">");
+                TezEdgeDescriptor edgeDesc = tezOper.inEdges.get(inEdge);
+                if (!edgeDesc.combinePlan.isEmpty()) {
+                    mStream.println("# Combine plan on edge <" + inEdge + ">");
                     PlanPrinter<PhysicalOperator, PhysicalPlan> printer =
                             new PlanPrinter<PhysicalOperator, PhysicalPlan>(
-                                    inEdge.getValue().combinePlan, mStream);
+                                    edgeDesc.combinePlan, mStream);
                     printer.setVerbose(isVerbose);
                     printer.visit();
                     mStream.println();
@@ -93,7 +97,7 @@ public class TezPrinter extends TezOpPla
         StringBuffer buf;
 
         public TezGraphPrinter(TezOperPlan plan) {
-            super(plan, new DependencyOrderWalker<TezOperator, TezOperPlan>(plan));
+            super(plan, new DependencyOrderWalker<TezOperator, TezOperPlan>(plan, true));
             buf = new StringBuffer();
         }
 
@@ -106,6 +110,7 @@ public class TezPrinter extends TezOpPla
             }
             List<TezOperator> succs = mPlan.getSuccessors(tezOper);
             if (succs != null) {
+                Collections.sort(succs);
                 buf.append("\t->\t");
                 for (TezOperator op : succs) {
                     if (op.isVertexGroup()) {

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/operator/POShuffledValueInputTez.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/operator/POShuffledValueInputTez.java?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/operator/POShuffledValueInputTez.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/operator/POShuffledValueInputTez.java Tue Dec  9 20:40:03 2014
@@ -20,6 +20,7 @@ package org.apache.pig.backend.hadoop.ex
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -160,6 +161,8 @@ public class POShuffledValueInputTez ext
 
     @Override
     public String name() {
-        return "POShuffledValueInputTez - " + mKey.toString() + "\t<-\t " + inputKeys;
+        List<String> inputKeyList = new ArrayList<String>(inputKeys);
+        Collections.sort(inputKeyList);
+        return "POShuffledValueInputTez - " + mKey.toString() + "\t<-\t " + inputKeyList;
     }
 }

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/operator/POValueOutputTez.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/operator/POValueOutputTez.java?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/operator/POValueOutputTez.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/operator/POValueOutputTez.java Tue Dec  9 20:40:03 2014
@@ -21,6 +21,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -189,7 +190,9 @@ public class POValueOutputTez extends Ph
 
     @Override
     public String name() {
-        return "POValueOutputTez - " + mKey.toString() + "\t->\t " + outputKeys;
+        List<String> outputKeyList = new ArrayList<String>(outputKeys);
+        Collections.sort(outputKeyList);
+        return "POValueOutputTez - " + mKey.toString() + "\t->\t " + outputKeyList;
     }
 
     public static class EmptyWritable implements Writable {

Modified: pig/trunk/src/org/apache/pig/impl/plan/DependencyOrderWalker.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/impl/plan/DependencyOrderWalker.java?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/impl/plan/DependencyOrderWalker.java (original)
+++ pig/trunk/src/org/apache/pig/impl/plan/DependencyOrderWalker.java Tue Dec  9 20:40:03 2014
@@ -19,6 +19,7 @@ package org.apache.pig.impl.plan;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -37,18 +38,31 @@ import org.apache.pig.impl.util.Utils;
 public class DependencyOrderWalker <O extends Operator, P extends OperatorPlan<O>>
     extends PlanWalker<O, P> {
 
+    private final boolean walkLeavesInOrder;
+
     /**
      * @param plan Plan for this walker to traverse.
      */
     public DependencyOrderWalker(P plan) {
+        this(plan, false);
+    }
+
+    /**
+     * @param plan Plan for this walker to traverse.
+     * @param boolean walkLeavesInOrder Sort the leaves before walking
+     */
+    public DependencyOrderWalker(P plan, boolean walkLeavesInOrder) {
         super(plan);
+        this.walkLeavesInOrder = walkLeavesInOrder;
     }
 
+
     /**
      * Begin traversing the graph.
      * @param visitor Visitor this walker is being used by.
      * @throws VisitorException if an error is encountered while walking.
      */
+    @Override
     @SuppressWarnings("unchecked")
     public void walk(PlanVisitor<O, P> visitor) throws VisitorException {
         // This is highly inefficient, but our graphs are small so it should be okay.
@@ -63,6 +77,9 @@ public class DependencyOrderWalker <O ex
         Set<O> seen = new HashSet<O>();
         List<O> leaves = mPlan.getLeaves();
         if (leaves == null) return;
+        if (walkLeavesInOrder) {
+            Collections.sort(leaves);
+        }
         for (O op : leaves) {
             doAllPredecessors(op, seen, fifo);
         }
@@ -71,8 +88,9 @@ public class DependencyOrderWalker <O ex
         }
     }
 
-    public PlanWalker<O, P> spawnChildWalker(P plan) { 
-        return new DependencyOrderWalker<O, P>(plan);
+    @Override
+    public PlanWalker<O, P> spawnChildWalker(P plan) {
+        return new DependencyOrderWalker<O, P>(plan, walkLeavesInOrder);
     }
 
     protected void doAllPredecessors(O node,

Modified: pig/trunk/test/org/apache/pig/builtin/TestAvroStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/builtin/TestAvroStorage.java?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/builtin/TestAvroStorage.java (original)
+++ pig/trunk/test/org/apache/pig/builtin/TestAvroStorage.java Tue Dec  9 20:40:03 2014
@@ -16,6 +16,25 @@
  */
 package org.apache.pig.builtin;
 
+import static org.apache.pig.builtin.mock.Storage.resetData;
+import static org.apache.pig.builtin.mock.Storage.tuple;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
 import org.apache.avro.file.DataFileStream;
 import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericData.Record;
@@ -51,25 +70,6 @@ import org.junit.Test;
 
 import com.google.common.collect.ImmutableMap;
 
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.io.PrintWriter;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-import static junit.framework.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.apache.pig.builtin.mock.Storage.resetData;
-import static org.apache.pig.builtin.mock.Storage.tuple;
-
 public class TestAvroStorage {
 
     final protected static Log LOG = LogFactory.getLog(TestAvroStorage.class);
@@ -119,6 +119,7 @@ public class TestAvroStorage {
     private static PigServer pigServerLocal = null;
 
     public static final PathFilter hiddenPathFilter = new PathFilter() {
+        @Override
         public boolean accept(Path p) {
           String name = p.getName();
           return !name.startsWith("_") && !name.startsWith(".");
@@ -325,9 +326,9 @@ public class TestAvroStorage {
         return outbasedir + st[index].getMethodName();
     }
 
-    @Test public void testLoadRecordsOfStringArrays() throws Exception {
+    @Test
+    public void testLoadRecordsOfStringArrays() throws Exception {
       final String input = basedir + "data/avro/uncompressed/recordsOfStringArrays.avro";
-      final String check = input;
       testAvroStorage(true, basedir + "code/pig/dump.pig",
           ImmutableMap.of(
               "INFILE",             input,
@@ -337,7 +338,8 @@ public class TestAvroStorage {
     }
 
 
-    @Test public void testLoadRecords() throws Exception {
+    @Test
+    public void testLoadRecords() throws Exception {
       final String input = basedir + "data/avro/uncompressed/records.avro";
       final String check = basedir + "data/avro/uncompressed/recordsAsOutputByPig.avro";
       testAvroStorage(true, basedir + "code/pig/identity_ao2.pig",
@@ -350,7 +352,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsWithSimpleUnion() throws Exception {
+    @Test
+    public void testLoadRecordsWithSimpleUnion() throws Exception {
         final String input = basedir + "data/avro/uncompressed/recordsWithSimpleUnion.avro";
         final String check = basedir + "data/avro/uncompressed/recordsWithSimpleUnionOutput.avro";
         testAvroStorage(true, basedir + "code/pig/identity_ao2.pig",
@@ -363,7 +366,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testProjection() throws Exception {
+    @Test
+    public void testProjection() throws Exception {
         final String input = basedir + "data/avro/uncompressed/records.avro";
         final String check = basedir + "data/avro/uncompressed/projectionTest.avro";
         testAvroStorage(true, basedir + "code/pig/projection_test.pig",
@@ -376,7 +380,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testProjectionWithSchema() throws Exception {
+    @Test
+    public void testProjectionWithSchema() throws Exception {
         final String input = basedir + "data/avro/uncompressed/records.avro";
         final String check = basedir + "data/avro/uncompressed/projectionTestWithSchema.avro";
         testAvroStorage(true, basedir + "code/pig/projection_test_with_schema.pig",
@@ -390,7 +395,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testDates() throws Exception {
+    @Test
+    public void testDates() throws Exception {
       final String input = basedir + "data/avro/uncompressed/records.avro";
       final String check = basedir + "data/avro/uncompressed/recordsAsOutputByPigWithDates.avro";
       testAvroStorage(true, basedir + "code/pig/with_dates.pig",
@@ -403,7 +409,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsSpecifyFullSchema() throws Exception {
+    @Test
+    public void testLoadRecordsSpecifyFullSchema() throws Exception {
       final String input = basedir + "data/avro/uncompressed/records.avro";
       final String check = basedir + "data/avro/uncompressed/recordsAsOutputByPig.avro";
       final String schema = loadFileIntoString(basedir + "schema/records.avsc");
@@ -418,7 +425,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsSpecifyFullSchemaFromFile() throws Exception {
+    @Test
+    public void testLoadRecordsSpecifyFullSchemaFromFile() throws Exception {
       final String input = basedir + "data/avro/uncompressed/records.avro";
       final String check = basedir + "data/avro/uncompressed/recordsAsOutputByPig.avro";
       testAvroStorage(true, basedir + "code/pig/identity.pig",
@@ -432,7 +440,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsSpecifySubSchema() throws Exception {
+    @Test
+    public void testLoadRecordsSpecifySubSchema() throws Exception {
       final String input = basedir + "data/avro/uncompressed/records.avro";
       final String check = basedir + "data/avro/uncompressed/recordsSubSchema.avro";
       testAvroStorage(true, basedir + "code/pig/identity_ai1_ao2.pig",
@@ -446,7 +455,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsSpecifySubSchemaFromFile() throws Exception {
+    @Test
+    public void testLoadRecordsSpecifySubSchemaFromFile() throws Exception {
       final String input = basedir + "data/avro/uncompressed/records.avro";
       final String check = basedir + "data/avro/uncompressed/recordsSubSchema.avro";
       testAvroStorage(true, basedir + "code/pig/identity_blank_first_args.pig",
@@ -459,7 +469,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsSpecifySubSchemaFromExampleFile() throws Exception {
+    @Test
+    public void testLoadRecordsSpecifySubSchemaFromExampleFile() throws Exception {
       final String input = basedir + "data/avro/uncompressed/records.avro";
       final String check = basedir + "data/avro/uncompressed/recordsSubSchema.avro";
       testAvroStorage(true, basedir + "code/pig/identity_blank_first_args.pig",
@@ -472,7 +483,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsOfArrays() throws Exception {
+    @Test
+    public void testLoadRecordsOfArrays() throws Exception {
       final String input = basedir + "data/avro/uncompressed/recordsOfArrays.avro";
       final String check = input;
       testAvroStorage(true, basedir + "code/pig/identity_just_ao2.pig",
@@ -484,7 +496,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsOfArraysOfRecords() throws Exception {
+    @Test
+    public void testLoadRecordsOfArraysOfRecords() throws Exception {
       final String input = basedir + "data/avro/uncompressed/recordsOfArraysOfRecords.avro";
       final String check = input;
       testAvroStorage(true, basedir + "code/pig/identity_just_ao2.pig",
@@ -496,7 +509,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsWithEnums() throws Exception {
+    @Test
+    public void testLoadRecordsWithEnums() throws Exception {
       final String input = basedir + "data/avro/uncompressed/recordsWithEnums.avro";
       final String check = input;
       testAvroStorage(true, basedir + "code/pig/identity_just_ao2.pig",
@@ -508,7 +522,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsWithFixed() throws Exception {
+    @Test
+    public void testLoadRecordsWithFixed() throws Exception {
       final String input = basedir + "data/avro/uncompressed/recordsWithFixed.avro";
       final String check = input;
       testAvroStorage(true, basedir + "code/pig/identity_just_ao2.pig",
@@ -520,7 +535,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsWithMaps() throws Exception {
+    @Test
+    public void testLoadRecordsWithMaps() throws Exception {
       final String input = basedir + "data/avro/uncompressed/recordsWithMaps.avro";
       final String check = input;
       testAvroStorage(true, basedir + "code/pig/identity_just_ao2.pig",
@@ -532,7 +548,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsWithMapsOfRecords() throws Exception {
+    @Test
+    public void testLoadRecordsWithMapsOfRecords() throws Exception {
       final String input = basedir + "data/avro/uncompressed/recordsWithMapsOfRecords.avro";
       final String check = input;
       testAvroStorage(true, basedir + "code/pig/identity_just_ao2.pig",
@@ -544,7 +561,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsWithMapsOfArrayOfRecords() throws Exception {
+    @Test
+    public void testLoadRecordsWithMapsOfArrayOfRecords() throws Exception {
         final String input = basedir + "data/avro/uncompressed/recordsWithMapsOfArrayOfRecords.avro";
         final String check = input;
         testAvroStorage(true, basedir + "code/pig/identity_just_ao2.pig",
@@ -556,7 +574,8 @@ public class TestAvroStorage {
         verifyResults(createOutputName(),check);
       }
 
-    @Test public void testLoadRecordsWithNullableUnions() throws Exception {
+    @Test
+    public void testLoadRecordsWithNullableUnions() throws Exception {
       final String input = basedir + "data/avro/uncompressed/recordsWithNullableUnions.avro";
       final String check = input;
       testAvroStorage(true, basedir + "code/pig/identity_just_ao2.pig",
@@ -568,7 +587,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadDeflateCompressedRecords() throws Exception {
+    @Test
+    public void testLoadDeflateCompressedRecords() throws Exception {
       final String input = basedir + "data/avro/compressed/deflate/records.avro";
       final String check = basedir + "data/avro/uncompressed/recordsAsOutputByPig.avro";
       testAvroStorage(true, basedir + "code/pig/identity_ao2.pig",
@@ -581,7 +601,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadSnappyCompressedRecords() throws Exception {
+    @Test
+    public void testLoadSnappyCompressedRecords() throws Exception {
       final String input = basedir + "data/avro/compressed/snappy/records.avro";
       final String check = basedir + "data/avro/uncompressed/recordsAsOutputByPig.avro";
       testAvroStorage(true, basedir + "code/pig/identity_ao2.pig",
@@ -594,7 +615,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testStoreDeflateCompressedRecords() throws Exception {
+    @Test
+    public void testStoreDeflateCompressedRecords() throws Exception {
       final String input = basedir + "data/avro/uncompressed/records.avro";
       final String check = basedir + "data/avro/compressed/deflate/recordsAsOutputByPig.avro";
       testAvroStorage(true, basedir + "code/pig/identity_codec.pig",
@@ -610,7 +632,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testStoreSnappyCompressedRecords() throws Exception {
+    @Test
+    public void testStoreSnappyCompressedRecords() throws Exception {
       final String input = basedir + "data/avro/uncompressed/records.avro";
       final String check = basedir + "data/avro/compressed/snappy/recordsAsOutputByPig.avro";
       testAvroStorage(true, basedir + "code/pig/identity_codec.pig",
@@ -626,7 +649,8 @@ public class TestAvroStorage {
      verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecursiveRecords() throws Exception {
+    @Test
+    public void testLoadRecursiveRecords() throws Exception {
       final String input = basedir + "data/avro/uncompressed/recursiveRecord.avro";
       testAvroStorage(false, basedir + "code/pig/recursive_tests.pig",
           ImmutableMap.of(
@@ -638,7 +662,8 @@ public class TestAvroStorage {
         );
     }
 
-    @Test public void testLoadRecursiveRecordsOptionOn() throws Exception {
+    @Test
+    public void testLoadRecursiveRecordsOptionOn() throws Exception {
       final String input = basedir + "data/avro/uncompressed/recursiveRecord.avro";
       final String check = basedir + "data/avro/uncompressed/recordsSubSchemaNullable.avro";
       testAvroStorage(true, basedir + "code/pig/recursive_tests.pig",
@@ -652,7 +677,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadRecordsWithRepeatedSubRecords() throws Exception {
+    @Test
+    public void testLoadRecordsWithRepeatedSubRecords() throws Exception {
       final String input = basedir + "data/avro/uncompressed/recordWithRepeatedSubRecords.avro";
       final String check = basedir + "data/avro/uncompressed/recordWithRepeatedSubRecords.avro";
       testAvroStorage(true, basedir + "code/pig/identity_just_ao2.pig",
@@ -664,7 +690,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadDirectory() throws Exception {
+    @Test
+    public void testLoadDirectory() throws Exception {
       final String input = basedir + "data/avro/uncompressed/testdirectory";
       final String check = basedir + "data/avro/uncompressed/testDirectoryCounts.avro";
       testAvroStorage(true, basedir + "code/pig/directory_test.pig",
@@ -677,7 +704,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadGlob() throws Exception {
+    @Test
+    public void testLoadGlob() throws Exception {
       final String input = basedir + "data/avro/uncompressed/testdirectory/part-m-0000*";
       final String check = basedir + "data/avro/uncompressed/testDirectoryCounts.avro";
       testAvroStorage(true, basedir + "code/pig/directory_test.pig",
@@ -690,7 +718,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testPartialLoadGlob() throws Exception {
+    @Test
+    public void testPartialLoadGlob() throws Exception {
       final String input = basedir + "data/avro/uncompressed/testdirectory/part-m-0000{0,2,4,6}.avro";
       final String check = basedir + "data/avro/uncompressed/evenFileNameTestDirectoryCounts.avro";
       testAvroStorage(true, basedir + "code/pig/directory_test.pig",
@@ -703,7 +732,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testSeparatedByComma() throws Exception {
+    @Test
+    public void testSeparatedByComma() throws Exception {
         final String temp = basedir
                 + "data/avro/uncompressed/testdirectory/part-m-0000";
         StringBuffer sb = new StringBuffer();
@@ -722,7 +752,8 @@ public class TestAvroStorage {
         verifyResults(createOutputName(), check);
     }
 
-    @Test public void testDoubleUnderscore() throws Exception {
+    @Test
+    public void testDoubleUnderscore() throws Exception {
       final String input = basedir + "data/avro/uncompressed/records.avro";
       final String check = basedir + "data/avro/uncompressed/recordsWithDoubleUnderscores.avro";
       testAvroStorage(true, basedir + "code/pig/namesWithDoubleColons.pig",
@@ -735,7 +766,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testDoubleUnderscoreNoFlag() throws Exception {
+    @Test
+    public void testDoubleUnderscoreNoFlag() throws Exception {
       final String input = basedir + "data/avro/uncompressed/records.avro";
       testAvroStorage(false, basedir + "code/pig/namesWithDoubleColons.pig",
           ImmutableMap.of(
@@ -746,7 +778,8 @@ public class TestAvroStorage {
         );
     }
 
-    @Test public void testLoadArrays() throws Exception {
+    @Test
+    public void testLoadArrays() throws Exception {
       final String input = basedir + "data/avro/uncompressed/arrays.avro";
       final String check = basedir + "data/avro/uncompressed/arraysAsOutputByPig.avro";
       testAvroStorage(true, basedir + "code/pig/identity_ao2.pig",
@@ -759,7 +792,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadTrevniRecords() throws Exception {
+    @Test
+    public void testLoadTrevniRecords() throws Exception {
       final String input = basedir + "data/trevni/uncompressed/simpleRecordsTrevni.trevni";
       final String check = basedir + "data/avro/uncompressed/simpleRecordsTrevni.avro";
       testAvroStorage(true, basedir + "code/pig/trevni_to_avro.pig",
@@ -771,7 +805,8 @@ public class TestAvroStorage {
       verifyResults(createOutputName(),check);
     }
 
-    @Test public void testLoadAndSaveTrevniRecords() throws Exception {
+    @Test
+    public void testLoadAndSaveTrevniRecords() throws Exception {
       final String input = basedir + "data/trevni/uncompressed/simpleRecordsTrevni.trevni";
       final String check = basedir + "data/avro/uncompressed/simpleRecordsTrevni.avro";
 
@@ -946,7 +981,8 @@ public class TestAvroStorage {
                 new Comparator<GenericData.Record>() {
                     @Override
                     public int compare(Record o1, Record o2) {
-                        return o1.toString().compareTo(o2.toString());
+                        //return o1.compareTo(o2); throws AvroRuntimeException: Can't compare maps!
+                        return o1.equals(o2) ? 0 : o1.toString().compareTo(o2.toString());
                     }}
                 );
         FileSystem fs = FileSystem.getLocal(new Configuration());

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Limit-2.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Limit-2.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Limit-2.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Limit-2.gld Tue Dec  9 20:40:03 2014
@@ -4,7 +4,7 @@
 #--------------------------------------------------
 # TEZ DAG plan: pig-0_scope-0
 #--------------------------------------------------
-Tez vertex scope-12	->	Tez vertex scope-33,Tez vertex scope-22,
+Tez vertex scope-12	->	Tez vertex scope-22,Tez vertex scope-33,
 Tez vertex scope-22	->	Tez vertex scope-33,
 Tez vertex scope-33	->	Tez vertex scope-35,
 Tez vertex scope-35	->	Tez vertex scope-46,

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Limit-3.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Limit-3.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Limit-3.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Limit-3.gld Tue Dec  9 20:40:03 2014
@@ -4,10 +4,10 @@
 #--------------------------------------------------
 # TEZ DAG plan: pig-0_scope-0
 #--------------------------------------------------
-Tez vertex scope-33	->	Tez vertex scope-37,Tez vertex scope-60,Tez vertex scope-49,
+Tez vertex scope-33	->	Tez vertex scope-37,Tez vertex scope-49,Tez vertex scope-60,
 Tez vertex scope-49	->	Tez vertex scope-60,
 Tez vertex scope-60	->	Tez vertex scope-62,
-Tez vertex scope-37	->	Tez vertex scope-67,Tez vertex scope-62,
+Tez vertex scope-37	->	Tez vertex scope-62,Tez vertex scope-67,
 Tez vertex scope-62	->	Tez vertex scope-67,
 Tez vertex scope-67
 
@@ -102,7 +102,7 @@ g: Local Rearrange[tuple]{chararray}(fal
     |
     |---g: Package(CombinerPackager)[tuple]{chararray} - scope-81
 # Plan on vertex
-POValueOutputTez - scope-70	->	 [scope-67, scope-62]
+POValueOutputTez - scope-70	->	 [scope-62, scope-67]
 |
 |---h: New For Each(false)[bag] - scope-23
     |   |

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-1-OPTOFF.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-1-OPTOFF.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-1-OPTOFF.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-1-OPTOFF.gld Tue Dec  9 20:40:03 2014
@@ -6,12 +6,12 @@
 #--------------------------------------------------
 Tez vertex scope-57	->	Tez vertex scope-59,Tez vertex scope-61,Tez vertex scope-63,
 Tez vertex scope-59
-Tez vertex scope-63
 Tez vertex scope-61
+Tez vertex scope-63
 
 Tez vertex scope-57
 # Plan on vertex
-POValueOutputTez - scope-58	->	 [scope-59, scope-63, scope-61]
+POValueOutputTez - scope-58	->	 [scope-59, scope-61, scope-63]
 |
 |---a: New For Each(false,false)[bag] - scope-40
     |   |
@@ -37,19 +37,6 @@ b: Store(file:///tmp/output/b:org.apache
     |   |---Constant(5) - scope-44
     |
     |---POValueInputTez - scope-60	<-	 scope-57
-Tez vertex scope-63
-# Plan on vertex
-d: Store(file:///tmp/output/d:org.apache.pig.builtin.PigStorage) - scope-56
-|
-|---d: Filter[bag] - scope-52
-    |   |
-    |   Greater Than[boolean] - scope-55
-    |   |
-    |   |---Project[int][0] - scope-53
-    |   |
-    |   |---Constant(10) - scope-54
-    |
-    |---POValueInputTez - scope-64	<-	 scope-57
 Tez vertex scope-61
 # Plan on vertex
 c: Store(file:///tmp/output/c:org.apache.pig.builtin.PigStorage) - scope-51
@@ -63,3 +50,16 @@ c: Store(file:///tmp/output/c:org.apache
     |   |---Constant(10) - scope-49
     |
     |---POValueInputTez - scope-62	<-	 scope-57
+Tez vertex scope-63
+# Plan on vertex
+d: Store(file:///tmp/output/d:org.apache.pig.builtin.PigStorage) - scope-56
+|
+|---d: Filter[bag] - scope-52
+    |   |
+    |   Greater Than[boolean] - scope-55
+    |   |
+    |   |---Project[int][0] - scope-53
+    |   |
+    |   |---Constant(10) - scope-54
+    |
+    |---POValueInputTez - scope-64	<-	 scope-57

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-2-OPTOFF.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-2-OPTOFF.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-2-OPTOFF.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-2-OPTOFF.gld Tue Dec  9 20:40:03 2014
@@ -5,34 +5,34 @@
 # TEZ DAG plan: pig-0_scope-1
 #--------------------------------------------------
 Tez vertex scope-317	->	Tez vertex scope-319,Tez vertex scope-330,Tez vertex scope-341,
-Tez vertex scope-341	->	Tez vertex scope-344,Tez vertex scope-384,
-Tez vertex scope-384	->	Tez vertex group scope-412,
 Tez vertex scope-319	->	Tez vertex scope-322,Tez vertex scope-335,Tez vertex scope-346,
-Tez vertex scope-346	->	Tez vertex scope-349,Tez vertex scope-375,
-Tez vertex scope-375	->	Tez vertex scope-378,
-Tez vertex scope-378	->	Tez vertex scope-382,Tez vertex scope-386,
-Tez vertex scope-386	->	Tez vertex group scope-412,
-Tez vertex group scope-412
-Tez vertex scope-382
-Tez vertex scope-330	->	Tez vertex scope-333,Tez vertex scope-338,
-Tez vertex scope-338	->	Tez vertex scope-340,
-Tez vertex scope-340
 Tez vertex scope-322	->	Tez vertex scope-324,
 Tez vertex scope-324	->	Tez vertex scope-326,Tez vertex scope-328,
 Tez vertex scope-326
-Tez vertex scope-344
 Tez vertex scope-328
-Tez vertex scope-349	->	Tez vertex scope-369,Tez vertex scope-359,
-Tez vertex scope-359	->	Tez vertex scope-369,
-Tez vertex scope-369	->	Tez vertex scope-371,
-Tez vertex scope-371
+Tez vertex scope-330	->	Tez vertex scope-333,Tez vertex scope-338,
 Tez vertex scope-333	->	Tez vertex scope-337,
 Tez vertex scope-335	->	Tez vertex scope-337,
 Tez vertex scope-337
+Tez vertex scope-338	->	Tez vertex scope-340,
+Tez vertex scope-340
+Tez vertex scope-341	->	Tez vertex scope-344,Tez vertex scope-384,
+Tez vertex scope-344
+Tez vertex scope-346	->	Tez vertex scope-349,Tez vertex scope-375,
+Tez vertex scope-349	->	Tez vertex scope-359,Tez vertex scope-369,
+Tez vertex scope-359	->	Tez vertex scope-369,
+Tez vertex scope-369	->	Tez vertex scope-371,
+Tez vertex scope-371
+Tez vertex scope-375	->	Tez vertex scope-378,
+Tez vertex scope-378	->	Tez vertex scope-382,Tez vertex scope-386,
+Tez vertex scope-382
+Tez vertex scope-384	->	Tez vertex group scope-412,
+Tez vertex scope-386	->	Tez vertex group scope-412,
+Tez vertex group scope-412
 
 Tez vertex scope-317
 # Plan on vertex
-POValueOutputTez - scope-318	->	 [scope-341, scope-319, scope-330]
+POValueOutputTez - scope-318	->	 [scope-319, scope-330, scope-341]
 |
 |---a: New For Each(false,false)[bag] - scope-218
     |   |
@@ -45,35 +45,9 @@ POValueOutputTez - scope-318	->	 [scope-
     |   |---Project[bytearray][1] - scope-215
     |
     |---a: Load(file:///tmp/input:org.apache.pig.builtin.PigStorage) - scope-211
-Tez vertex scope-341
-# Plan on vertex
-POValueOutputTez - scope-343	->	 [scope-384, scope-344]
-|
-|---d1: Filter[bag] - scope-284
-    |   |
-    |   Equal To[boolean] - scope-287
-    |   |
-    |   |---Project[int][0] - scope-285
-    |   |
-    |   |---Constant(5) - scope-286
-    |
-    |---d: Filter[bag] - scope-280
-        |   |
-        |   Greater Than[boolean] - scope-283
-        |   |
-        |   |---Project[int][0] - scope-281
-        |   |
-        |   |---Constant(10) - scope-282
-        |
-        |---POValueInputTez - scope-342	<-	 scope-317
-Tez vertex scope-384
-# Plan on vertex
-f2: Store(file:///tmp/output/f2:org.apache.pig.builtin.PigStorage) - scope-413
-|
-|---POValueInputTez - scope-385	<-	 scope-341
 Tez vertex scope-319
 # Plan on vertex
-POValueOutputTez - scope-321	->	 [scope-335, scope-322, scope-346]
+POValueOutputTez - scope-321	->	 [scope-322, scope-335, scope-346]
 |
 |---b: Filter[bag] - scope-220
     |   |
@@ -84,48 +58,41 @@ POValueOutputTez - scope-321	->	 [scope-
     |   |---Constant(5) - scope-222
     |
     |---POValueInputTez - scope-320	<-	 scope-317
-Tez vertex scope-346
-# Plan on vertex
-POValueOutputTez - scope-348	->	 [scope-375, scope-349]
-|
-|---POValueInputTez - scope-347	<-	 scope-319
-Tez vertex scope-375
+Tez vertex scope-322
 # Plan on vertex
-POValueOutputTez - scope-377	->	 [scope-378]
+b1: Local Rearrange[tuple]{int}(false) - scope-229	->	 scope-324
+|   |
+|   Project[int][0] - scope-230
 |
-|---f1: Limit - scope-306
-    |
-    |---f: Filter[bag] - scope-302
-        |   |
-        |   Greater Than or Equal[boolean] - scope-305
-        |   |
-        |   |---Project[int][0] - scope-303
-        |   |
-        |   |---Constant(3) - scope-304
-        |
-        |---POValueInputTez - scope-376	<-	 scope-346
-Tez vertex scope-378
+|---POValueInputTez - scope-323	<-	 scope-319
+Tez vertex scope-324
 # Plan on vertex
-POValueOutputTez - scope-381	->	 [scope-386, scope-382]
+POValueOutputTez - scope-325	->	 [scope-326, scope-328]
 |
-|---f1: Limit - scope-380
-    |
-    |---POValueInputTez - scope-379	<-	 scope-375
-Tez vertex scope-386
+|---b1: Package(Packager)[tuple]{int} - scope-228
+Tez vertex scope-326
 # Plan on vertex
-f2: Store(file:///tmp/output/f2:org.apache.pig.builtin.PigStorage) - scope-414
+b1: Store(file:///tmp/output/b1:org.apache.pig.builtin.PigStorage) - scope-234
 |
-|---POValueInputTez - scope-387	<-	 scope-378
-Tez vertex group scope-412	<-	 [scope-384, scope-386]	->	 null
-# No plan on vertex group
-Tez vertex scope-382
+|---POValueInputTez - scope-327	<-	 scope-324
+Tez vertex scope-328
 # Plan on vertex
-f1: Store(file:///tmp/output/f1:org.apache.pig.builtin.PigStorage) - scope-310
+b2: Store(file:///tmp/output/b2:org.apache.pig.builtin.PigStorage) - scope-244
 |
-|---POValueInputTez - scope-383	<-	 scope-378
+|---b2: New For Each(false,false)[bag] - scope-243
+    |   |
+    |   Project[int][0] - scope-237
+    |   |
+    |   POUserFunc(org.apache.pig.builtin.LongSum)[long] - scope-241
+    |   |
+    |   |---Project[bag][0] - scope-240
+    |       |
+    |       |---Project[bag][1] - scope-239
+    |
+    |---POValueInputTez - scope-329	<-	 scope-324
 Tez vertex scope-330
 # Plan on vertex
-POValueOutputTez - scope-332	->	 [scope-338, scope-333]
+POValueOutputTez - scope-332	->	 [scope-333, scope-338]
 |
 |---c: Filter[bag] - scope-245
     |   |
@@ -136,6 +103,31 @@ POValueOutputTez - scope-332	->	 [scope-
     |   |---Constant(10) - scope-247
     |
     |---POValueInputTez - scope-331	<-	 scope-317
+Tez vertex scope-333
+# Plan on vertex
+c1: Local Rearrange[tuple]{int}(false) - scope-258	->	 scope-337
+|   |
+|   Project[int][0] - scope-259
+|
+|---POValueInputTez - scope-334	<-	 scope-330
+Tez vertex scope-335
+# Plan on vertex
+c1: Local Rearrange[tuple]{int}(false) - scope-260	->	 scope-337
+|   |
+|   Project[int][0] - scope-261
+|
+|---POValueInputTez - scope-336	<-	 scope-319
+Tez vertex scope-337
+# Plan on vertex
+c1: Store(file:///tmp/output/c1:org.apache.pig.builtin.PigStorage) - scope-265
+|
+|---c1: New For Each(true,true)[tuple] - scope-264
+    |   |
+    |   Project[bag][1] - scope-262
+    |   |
+    |   Project[bag][2] - scope-263
+    |
+    |---c1: Package(Packager)[tuple]{int} - scope-257
 Tez vertex scope-338
 # Plan on vertex
 c2: Local Rearrange[tuple]{int}(false) - scope-404	->	 scope-340
@@ -182,43 +174,37 @@ c3: Store(file:///tmp/output/c1:org.apac
     |   |---Project[bag][1] - scope-401
     |
     |---c2: Package(CombinerPackager)[tuple]{int} - scope-269
-Tez vertex scope-322
-# Plan on vertex
-b1: Local Rearrange[tuple]{int}(false) - scope-229	->	 scope-324
-|   |
-|   Project[int][0] - scope-230
-|
-|---POValueInputTez - scope-323	<-	 scope-319
-Tez vertex scope-324
-# Plan on vertex
-POValueOutputTez - scope-325	->	 [scope-328, scope-326]
-|
-|---b1: Package(Packager)[tuple]{int} - scope-228
-Tez vertex scope-326
+Tez vertex scope-341
 # Plan on vertex
-b1: Store(file:///tmp/output/b1:org.apache.pig.builtin.PigStorage) - scope-234
+POValueOutputTez - scope-343	->	 [scope-344, scope-384]
 |
-|---POValueInputTez - scope-327	<-	 scope-324
+|---d1: Filter[bag] - scope-284
+    |   |
+    |   Equal To[boolean] - scope-287
+    |   |
+    |   |---Project[int][0] - scope-285
+    |   |
+    |   |---Constant(5) - scope-286
+    |
+    |---d: Filter[bag] - scope-280
+        |   |
+        |   Greater Than[boolean] - scope-283
+        |   |
+        |   |---Project[int][0] - scope-281
+        |   |
+        |   |---Constant(10) - scope-282
+        |
+        |---POValueInputTez - scope-342	<-	 scope-317
 Tez vertex scope-344
 # Plan on vertex
 d1: Store(file:///tmp/output/d1:org.apache.pig.builtin.PigStorage) - scope-291
 |
 |---POValueInputTez - scope-345	<-	 scope-341
-Tez vertex scope-328
+Tez vertex scope-346
 # Plan on vertex
-b2: Store(file:///tmp/output/b2:org.apache.pig.builtin.PigStorage) - scope-244
+POValueOutputTez - scope-348	->	 [scope-349, scope-375]
 |
-|---b2: New For Each(false,false)[bag] - scope-243
-    |   |
-    |   Project[int][0] - scope-237
-    |   |
-    |   POUserFunc(org.apache.pig.builtin.LongSum)[long] - scope-241
-    |   |
-    |   |---Project[bag][0] - scope-240
-    |       |
-    |       |---Project[bag][1] - scope-239
-    |
-    |---POValueInputTez - scope-329	<-	 scope-324
+|---POValueInputTez - scope-347	<-	 scope-319
 Tez vertex scope-349
 # Plan on vertex
 Local Rearrange[tuple]{tuple}(false) - scope-353	->	 scope-359
@@ -279,28 +265,42 @@ e1: Store(file:///tmp/output/e1:org.apac
     |   Project[bag][1] - scope-373
     |
     |---Package(LitePackager)[tuple]{int} - scope-372
-Tez vertex scope-333
+Tez vertex scope-375
 # Plan on vertex
-c1: Local Rearrange[tuple]{int}(false) - scope-258	->	 scope-337
-|   |
-|   Project[int][0] - scope-259
+POValueOutputTez - scope-377	->	 [scope-378]
 |
-|---POValueInputTez - scope-334	<-	 scope-330
-Tez vertex scope-335
+|---f1: Limit - scope-306
+    |
+    |---f: Filter[bag] - scope-302
+        |   |
+        |   Greater Than or Equal[boolean] - scope-305
+        |   |
+        |   |---Project[int][0] - scope-303
+        |   |
+        |   |---Constant(3) - scope-304
+        |
+        |---POValueInputTez - scope-376	<-	 scope-346
+Tez vertex scope-378
 # Plan on vertex
-c1: Local Rearrange[tuple]{int}(false) - scope-260	->	 scope-337
-|   |
-|   Project[int][0] - scope-261
+POValueOutputTez - scope-381	->	 [scope-382, scope-386]
 |
-|---POValueInputTez - scope-336	<-	 scope-319
-Tez vertex scope-337
+|---f1: Limit - scope-380
+    |
+    |---POValueInputTez - scope-379	<-	 scope-375
+Tez vertex scope-382
 # Plan on vertex
-c1: Store(file:///tmp/output/c1:org.apache.pig.builtin.PigStorage) - scope-265
+f1: Store(file:///tmp/output/f1:org.apache.pig.builtin.PigStorage) - scope-310
 |
-|---c1: New For Each(true,true)[tuple] - scope-264
-    |   |
-    |   Project[bag][1] - scope-262
-    |   |
-    |   Project[bag][2] - scope-263
-    |
-    |---c1: Package(Packager)[tuple]{int} - scope-257
+|---POValueInputTez - scope-383	<-	 scope-378
+Tez vertex scope-384
+# Plan on vertex
+f2: Store(file:///tmp/output/f2:org.apache.pig.builtin.PigStorage) - scope-413
+|
+|---POValueInputTez - scope-385	<-	 scope-341
+Tez vertex scope-386
+# Plan on vertex
+f2: Store(file:///tmp/output/f2:org.apache.pig.builtin.PigStorage) - scope-414
+|
+|---POValueInputTez - scope-387	<-	 scope-378
+Tez vertex group scope-412	<-	 [scope-384, scope-386]	->	 null
+# No plan on vertex group

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-2.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-2.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-2.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-2.gld Tue Dec  9 20:40:03 2014
@@ -4,16 +4,16 @@
 #--------------------------------------------------
 # TEZ DAG plan: pig-0_scope-0
 #--------------------------------------------------
-Tez vertex scope-106	->	Tez vertex scope-119,Tez vertex scope-113,Tez vertex scope-126,Tez vertex scope-158,Tez vertex scope-148,Tez vertex scope-167,Tez vertex group scope-208,
-Tez vertex scope-167	->	Tez vertex group scope-208,
-Tez vertex group scope-208
+Tez vertex scope-106	->	Tez vertex scope-113,Tez vertex scope-119,Tez vertex scope-126,Tez vertex scope-148,Tez vertex scope-158,Tez vertex scope-167,Tez vertex group scope-208,
 Tez vertex scope-113
-Tez vertex scope-148	->	Tez vertex scope-158,
-Tez vertex scope-158	->	Tez vertex scope-160,
-Tez vertex scope-160
 Tez vertex scope-119	->	Tez vertex scope-126,Tez vertex scope-129,
 Tez vertex scope-126
 Tez vertex scope-129
+Tez vertex scope-148	->	Tez vertex scope-158,
+Tez vertex scope-158	->	Tez vertex scope-160,
+Tez vertex scope-160
+Tez vertex scope-167	->	Tez vertex group scope-208,
+Tez vertex group scope-208
 
 Tez vertex scope-106
 # Plan on vertex
@@ -112,19 +112,6 @@ Tez vertex scope-106
     |   |---Project[bytearray][1] - scope-4
     |
     |---a: Load(file:///tmp/input:org.apache.pig.builtin.PigStorage) - scope-0
-Tez vertex scope-167
-# Plan on vertex
-f1: Split - scope-202
-|   |
-|   f1: Store(file:///tmp/output/f1:org.apache.pig.builtin.PigStorage) - scope-99
-|   |
-|   f2: Store(file:///tmp/output/f2:org.apache.pig.builtin.PigStorage) - scope-210
-|
-|---f1: Limit - scope-169
-    |
-    |---POValueInputTez - scope-168	<-	 scope-106
-Tez vertex group scope-208	<-	 [scope-106, scope-167]	->	 null
-# No plan on vertex group
 Tez vertex scope-113
 # Plan on vertex
 b1: Split - scope-201
@@ -144,37 +131,6 @@ b1: Split - scope-201
 |       |       |---Project[bag][1] - scope-28
 |
 |---b1: Package(Packager)[tuple]{int} - scope-17
-Tez vertex scope-148
-# Plan on vertex
-POValueOutputTez - scope-157	->	 [scope-158]
-|
-|---New For Each(false)[tuple] - scope-156
-    |   |
-    |   POUserFunc(org.apache.pig.backend.hadoop.executionengine.tez.plan.udf.FindQuantilesTez)[tuple] - scope-155
-    |   |
-    |   |---Project[tuple][*] - scope-154
-    |
-    |---New For Each(false,false)[tuple] - scope-153
-        |   |
-        |   Constant(-1) - scope-152
-        |   |
-        |   Project[bag][1] - scope-150
-        |
-        |---Package(Packager)[tuple]{bytearray} - scope-149
-Tez vertex scope-158
-# Plan on vertex
-POIdentityInOutTez - scope-159	<-	 scope-106	->	 scope-160
-|   |
-|   Project[int][0] - scope-88
-Tez vertex scope-160
-# Plan on vertex
-e1: Store(file:///tmp/output/e1:org.apache.pig.builtin.PigStorage) - scope-90
-|
-|---New For Each(true)[tuple] - scope-163
-    |   |
-    |   Project[bag][1] - scope-162
-    |
-    |---Package(LitePackager)[tuple]{int} - scope-161
 Tez vertex scope-119
 # Plan on vertex
 c: Split - scope-205
@@ -246,3 +202,47 @@ c3: Store(file:///tmp/output/c1:org.apac
     |   |---Project[bag][1] - scope-190
     |
     |---c2: Package(CombinerPackager)[tuple]{int} - scope-58
+Tez vertex scope-148
+# Plan on vertex
+POValueOutputTez - scope-157	->	 [scope-158]
+|
+|---New For Each(false)[tuple] - scope-156
+    |   |
+    |   POUserFunc(org.apache.pig.backend.hadoop.executionengine.tez.plan.udf.FindQuantilesTez)[tuple] - scope-155
+    |   |
+    |   |---Project[tuple][*] - scope-154
+    |
+    |---New For Each(false,false)[tuple] - scope-153
+        |   |
+        |   Constant(-1) - scope-152
+        |   |
+        |   Project[bag][1] - scope-150
+        |
+        |---Package(Packager)[tuple]{bytearray} - scope-149
+Tez vertex scope-158
+# Plan on vertex
+POIdentityInOutTez - scope-159	<-	 scope-106	->	 scope-160
+|   |
+|   Project[int][0] - scope-88
+Tez vertex scope-160
+# Plan on vertex
+e1: Store(file:///tmp/output/e1:org.apache.pig.builtin.PigStorage) - scope-90
+|
+|---New For Each(true)[tuple] - scope-163
+    |   |
+    |   Project[bag][1] - scope-162
+    |
+    |---Package(LitePackager)[tuple]{int} - scope-161
+Tez vertex scope-167
+# Plan on vertex
+f1: Split - scope-202
+|   |
+|   f1: Store(file:///tmp/output/f1:org.apache.pig.builtin.PigStorage) - scope-99
+|   |
+|   f2: Store(file:///tmp/output/f2:org.apache.pig.builtin.PigStorage) - scope-210
+|
+|---f1: Limit - scope-169
+    |
+    |---POValueInputTez - scope-168	<-	 scope-106
+Tez vertex group scope-208	<-	 [scope-106, scope-167]	->	 null
+# No plan on vertex group

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-3.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-3.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-3.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-3.gld Tue Dec  9 20:40:03 2014
@@ -5,8 +5,8 @@
 # TEZ DAG plan: pig-0_scope-0
 #--------------------------------------------------
 Tez vertex scope-38	->	Tez vertex scope-42,Tez vertex scope-45,
-Tez vertex scope-45
 Tez vertex scope-42
+Tez vertex scope-45
 
 Tez vertex scope-38
 # Plan on vertex
@@ -55,33 +55,6 @@ a: Split - scope-86
     |   |---Project[bytearray][1] - scope-4
     |
     |---a: Load(file:///tmp/input:org.apache.pig.builtin.PigStorage) - scope-0
-Tez vertex scope-45
-# Combine plan on edge <scope-38>
-c: Local Rearrange[tuple]{tuple}(false) - scope-82	->	 scope-45
-|   |
-|   Project[tuple][0] - scope-85
-|
-|---c: New For Each(false,false)[bag] - scope-70
-    |   |
-    |   Project[tuple][0] - scope-71
-    |   |
-    |   POUserFunc(org.apache.pig.builtin.COUNT$Intermediate)[tuple] - scope-72
-    |   |
-    |   |---Project[bag][1] - scope-73
-    |
-    |---c: Package(CombinerPackager)[tuple]{tuple} - scope-76
-# Plan on vertex
-c: Store(file:///tmp/output/c:org.apache.pig.builtin.PigStorage) - scope-37
-|
-|---c: New For Each(false,false)[bag] - scope-36
-    |   |
-    |   Project[tuple][0] - scope-30
-    |   |
-    |   POUserFunc(org.apache.pig.builtin.COUNT$Final)[long] - scope-34
-    |   |
-    |   |---Project[bag][1] - scope-74
-    |
-    |---c: Package(CombinerPackager)[tuple]{tuple} - scope-26
 Tez vertex scope-42
 # Combine plan on edge <scope-38>
 b: Local Rearrange[tuple]{int}(false) - scope-62	->	 scope-42
@@ -109,3 +82,30 @@ b: Store(file:///tmp/output/b:org.apache
     |   |---Project[bag][1] - scope-55
     |
     |---b: Package(CombinerPackager)[tuple]{int} - scope-12
+Tez vertex scope-45
+# Combine plan on edge <scope-38>
+c: Local Rearrange[tuple]{tuple}(false) - scope-82	->	 scope-45
+|   |
+|   Project[tuple][0] - scope-85
+|
+|---c: New For Each(false,false)[bag] - scope-70
+    |   |
+    |   Project[tuple][0] - scope-71
+    |   |
+    |   POUserFunc(org.apache.pig.builtin.COUNT$Intermediate)[tuple] - scope-72
+    |   |
+    |   |---Project[bag][1] - scope-73
+    |
+    |---c: Package(CombinerPackager)[tuple]{tuple} - scope-76
+# Plan on vertex
+c: Store(file:///tmp/output/c:org.apache.pig.builtin.PigStorage) - scope-37
+|
+|---c: New For Each(false,false)[bag] - scope-36
+    |   |
+    |   Project[tuple][0] - scope-30
+    |   |
+    |   POUserFunc(org.apache.pig.builtin.COUNT$Final)[long] - scope-34
+    |   |
+    |   |---Project[bag][1] - scope-74
+    |
+    |---c: Package(CombinerPackager)[tuple]{tuple} - scope-26

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-4-OPTOFF.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-4-OPTOFF.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-4-OPTOFF.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-4-OPTOFF.gld Tue Dec  9 20:40:03 2014
@@ -7,9 +7,9 @@
 Tez vertex scope-117	->	Tez vertex scope-119,
 Tez vertex scope-118	->	Tez vertex scope-119,
 Tez vertex scope-119	->	Tez vertex scope-121,Tez vertex scope-123,Tez vertex scope-125,
+Tez vertex scope-121
 Tez vertex scope-123
 Tez vertex scope-125
-Tez vertex scope-121
 
 Tez vertex scope-117
 # Plan on vertex
@@ -47,7 +47,7 @@ c: Local Rearrange[tuple]{int}(false) -
     |---b: Load(file:///tmp/input2:org.apache.pig.builtin.PigStorage) - scope-72
 Tez vertex scope-119
 # Plan on vertex
-POValueOutputTez - scope-120	->	 [scope-123, scope-125, scope-121]
+POValueOutputTez - scope-120	->	 [scope-121, scope-123, scope-125]
 |
 |---c: New For Each(true,true)[tuple] - scope-90
     |   |
@@ -56,6 +56,11 @@ POValueOutputTez - scope-120	->	 [scope-
     |   Project[bag][2] - scope-89
     |
     |---c: Package(Packager)[tuple]{int} - scope-83
+Tez vertex scope-121
+# Plan on vertex
+c: Store(file:///tmp/output/c:org.apache.pig.builtin.PigStorage) - scope-94
+|
+|---POValueInputTez - scope-122	<-	 scope-119
 Tez vertex scope-123
 # Plan on vertex
 d: Store(file:///tmp/output/d:org.apache.pig.builtin.PigStorage) - scope-104
@@ -84,8 +89,3 @@ e: Store(file:///tmp/output/e:org.apache
     |   Project[int][3] - scope-113
     |
     |---POValueInputTez - scope-126	<-	 scope-119
-Tez vertex scope-121
-# Plan on vertex
-c: Store(file:///tmp/output/c:org.apache.pig.builtin.PigStorage) - scope-94
-|
-|---POValueInputTez - scope-122	<-	 scope-119

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-5-OPTOFF.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-5-OPTOFF.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-5-OPTOFF.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-MQ-5-OPTOFF.gld Tue Dec  9 20:40:03 2014
@@ -6,11 +6,11 @@
 #--------------------------------------------------
 Tez vertex scope-102	->	Tez vertex scope-103,
 Tez vertex scope-103	->	Tez vertex scope-105,Tez vertex scope-107,Tez vertex scope-109,
+Tez vertex scope-105
+Tez vertex scope-107
 Tez vertex scope-109	->	Tez vertex scope-112,Tez vertex scope-114,
-Tez vertex scope-114
 Tez vertex scope-112
-Tez vertex scope-107
-Tez vertex scope-105
+Tez vertex scope-114
 
 Tez vertex scope-102
 # Plan on vertex
@@ -31,34 +31,14 @@ b: Local Rearrange[tuple]{int}(false) -
     |---a: Load(file:///tmp/input:org.apache.pig.builtin.PigStorage) - scope-59
 Tez vertex scope-103
 # Plan on vertex
-POValueOutputTez - scope-104	->	 [scope-107, scope-109, scope-105]
+POValueOutputTez - scope-104	->	 [scope-105, scope-107, scope-109]
 |
 |---b: Package(Packager)[tuple]{int} - scope-68
-Tez vertex scope-109
-# Plan on vertex
-POValueOutputTez - scope-111	->	 [scope-114, scope-112]
-|
-|---d: New For Each(true)[bag] - scope-89
-    |   |
-    |   Project[bag][1] - scope-87
-    |
-    |---POValueInputTez - scope-110	<-	 scope-103
-Tez vertex scope-114
-# Plan on vertex
-e: Store(file:///tmp/output/e:org.apache.pig.builtin.PigStorage) - scope-101
-|
-|---e: New For Each(false,false)[bag] - scope-100
-    |   |
-    |   Project[int][0] - scope-96
-    |   |
-    |   Project[int][1] - scope-98
-    |
-    |---POValueInputTez - scope-115	<-	 scope-109
-Tez vertex scope-112
+Tez vertex scope-105
 # Plan on vertex
-d: Store(file:///tmp/output/d:org.apache.pig.builtin.PigStorage) - scope-93
+b: Store(file:///tmp/output/b:org.apache.pig.builtin.PigStorage) - scope-74
 |
-|---POValueInputTez - scope-113	<-	 scope-109
+|---POValueInputTez - scope-106	<-	 scope-103
 Tez vertex scope-107
 # Plan on vertex
 c: Store(file:///tmp/output/c:org.apache.pig.builtin.PigStorage) - scope-84
@@ -74,8 +54,28 @@ c: Store(file:///tmp/output/c:org.apache
     |   |---Project[bag][1] - scope-80
     |
     |---POValueInputTez - scope-108	<-	 scope-103
-Tez vertex scope-105
+Tez vertex scope-109
 # Plan on vertex
-b: Store(file:///tmp/output/b:org.apache.pig.builtin.PigStorage) - scope-74
+POValueOutputTez - scope-111	->	 [scope-112, scope-114]
 |
-|---POValueInputTez - scope-106	<-	 scope-103
+|---d: New For Each(true)[bag] - scope-89
+    |   |
+    |   Project[bag][1] - scope-87
+    |
+    |---POValueInputTez - scope-110	<-	 scope-103
+Tez vertex scope-112
+# Plan on vertex
+d: Store(file:///tmp/output/d:org.apache.pig.builtin.PigStorage) - scope-93
+|
+|---POValueInputTez - scope-113	<-	 scope-109
+Tez vertex scope-114
+# Plan on vertex
+e: Store(file:///tmp/output/e:org.apache.pig.builtin.PigStorage) - scope-101
+|
+|---e: New For Each(false,false)[bag] - scope-100
+    |   |
+    |   Project[int][0] - scope-96
+    |   |
+    |   Project[int][1] - scope-98
+    |
+    |---POValueInputTez - scope-115	<-	 scope-109

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Order-1.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Order-1.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Order-1.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Order-1.gld Tue Dec  9 20:40:03 2014
@@ -4,7 +4,7 @@
 #--------------------------------------------------
 # TEZ DAG plan: pig-0_scope-0
 #--------------------------------------------------
-Tez vertex scope-11	->	Tez vertex scope-30,Tez vertex scope-20,
+Tez vertex scope-11	->	Tez vertex scope-20,Tez vertex scope-30,
 Tez vertex scope-20	->	Tez vertex scope-30,
 Tez vertex scope-30	->	Tez vertex scope-32,
 Tez vertex scope-32

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Rank-1.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Rank-1.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Rank-1.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Rank-1.gld Tue Dec  9 20:40:03 2014
@@ -4,7 +4,7 @@
 #--------------------------------------------------
 # TEZ DAG plan: pig-0_scope-0
 #--------------------------------------------------
-Tez vertex scope-11	->	Tez vertex scope-14,Tez vertex scope-12,
+Tez vertex scope-11	->	Tez vertex scope-12,Tez vertex scope-14,
 Tez vertex scope-12	->	Tez vertex scope-14,
 Tez vertex scope-14
 

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Rank-2.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Rank-2.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Rank-2.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Rank-2.gld Tue Dec  9 20:40:03 2014
@@ -5,10 +5,10 @@
 # TEZ DAG plan: pig-0_scope-0
 #--------------------------------------------------
 Tez vertex scope-23	->	Tez vertex scope-24,
-Tez vertex scope-24	->	Tez vertex scope-43,Tez vertex scope-33,
+Tez vertex scope-24	->	Tez vertex scope-33,Tez vertex scope-43,
 Tez vertex scope-33	->	Tez vertex scope-43,
 Tez vertex scope-43	->	Tez vertex scope-45,
-Tez vertex scope-45	->	Tez vertex scope-51,Tez vertex scope-49,
+Tez vertex scope-45	->	Tez vertex scope-49,Tez vertex scope-51,
 Tez vertex scope-49	->	Tez vertex scope-51,
 Tez vertex scope-51
 

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-SkewJoin-1.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-SkewJoin-1.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-SkewJoin-1.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-SkewJoin-1.gld Tue Dec  9 20:40:03 2014
@@ -4,8 +4,8 @@
 #--------------------------------------------------
 # TEZ DAG plan: pig-0_scope-0
 #--------------------------------------------------
-Tez vertex scope-27	->	Tez vertex scope-46,Tez vertex scope-36,
-Tez vertex scope-36	->	Tez vertex scope-46,Tez vertex scope-28,
+Tez vertex scope-27	->	Tez vertex scope-36,Tez vertex scope-46,
+Tez vertex scope-36	->	Tez vertex scope-28,Tez vertex scope-46,
 Tez vertex scope-46	->	Tez vertex scope-50,
 Tez vertex scope-28	->	Tez vertex scope-50,
 Tez vertex scope-50
@@ -43,7 +43,7 @@ Local Rearrange[tuple]{tuple}(false) - s
                 |---a: Load(file:///tmp/input1:org.apache.pig.builtin.PigStorage) - scope-0
 Tez vertex scope-36
 # Plan on vertex
-POValueOutputTez - scope-45	->	 [scope-46, scope-28]
+POValueOutputTez - scope-45	->	 [scope-28, scope-46]
 |
 |---New For Each(false)[tuple] - scope-44
     |   |

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-1-OPTOFF.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-1-OPTOFF.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-1-OPTOFF.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-1-OPTOFF.gld Tue Dec  9 20:40:03 2014
@@ -42,4 +42,4 @@ Tez vertex scope-47
 # Plan on vertex
 c: Store(file:///tmp/output:org.apache.pig.builtin.PigStorage) - scope-44
 |
-|---POShuffledValueInputTez - scope-48	<-	 [scope-46, scope-45]
+|---POShuffledValueInputTez - scope-48	<-	 [scope-45, scope-46]

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-10-OPTOFF.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-10-OPTOFF.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-10-OPTOFF.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-10-OPTOFF.gld Tue Dec  9 20:40:03 2014
@@ -67,7 +67,7 @@ f: Local Rearrange[tuple]{int}(false) -
 |   |
 |   Project[int][0] - scope-29
 |
-|---POShuffledValueInputTez - scope-39	<-	 [scope-37, scope-33]
+|---POShuffledValueInputTez - scope-39	<-	 [scope-33, scope-37]
 Tez vertex scope-42
 # Plan on vertex
 f: Store(file:///tmp/output:org.apache.pig.builtin.PigStorage) - scope-30

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-2-OPTOFF.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-2-OPTOFF.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-2-OPTOFF.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-2-OPTOFF.gld Tue Dec  9 20:40:03 2014
@@ -57,7 +57,7 @@ d: Local Rearrange[tuple]{int}(false) -
     |
     |---Pre Combiner Local Rearrange[tuple]{Unknown} - scope-123
         |
-        |---POShuffledValueInputTez - scope-104	<-	 [scope-102, scope-101]
+        |---POShuffledValueInputTez - scope-104	<-	 [scope-101, scope-102]
 Tez vertex scope-107
 # Combine plan on edge <scope-103>
 d: Local Rearrange[tuple]{int}(false) - scope-124	->	 scope-107

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-2.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-2.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-2.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-2.gld Tue Dec  9 20:40:03 2014
@@ -11,19 +11,19 @@ Tez vertex scope-35
 
 Tez vertex scope-29
 # Plan on vertex
-d: Local Rearrange[tuple]{int}(false) - scope-56	->	 scope-35
+d: Local Rearrange[tuple]{int}(false) - scope-61	->	 scope-35
 |   |
-|   Project[int][0] - scope-57
+|   Project[int][0] - scope-62
 |
-|---e: New For Each(false,false)[bag] - scope-62
+|---e: New For Each(false,false)[bag] - scope-60
     |   |
-    |   Project[int][0] - scope-58
+    |   Project[int][0] - scope-56
     |   |
-    |   POUserFunc(org.apache.pig.builtin.AlgebraicMathBase$Initial)[tuple] - scope-59
+    |   POUserFunc(org.apache.pig.builtin.AlgebraicMathBase$Initial)[tuple] - scope-57
     |   |
-    |   |---Project[bag][1] - scope-60
+    |   |---Project[bag][1] - scope-58
     |       |
-    |       |---Project[bag][1] - scope-61
+    |       |---Project[bag][1] - scope-59
     |
     |---Pre Combiner Local Rearrange[tuple]{Unknown} - scope-63
         |
@@ -40,19 +40,19 @@ d: Local Rearrange[tuple]{int}(false) -
             |---a: Load(file:///tmp/input:org.apache.pig.builtin.PigStorage) - scope-0
 Tez vertex scope-30
 # Plan on vertex
-d: Local Rearrange[tuple]{int}(false) - scope-64	->	 scope-35
+d: Local Rearrange[tuple]{int}(false) - scope-69	->	 scope-35
 |   |
-|   Project[int][0] - scope-65
+|   Project[int][0] - scope-70
 |
-|---e: New For Each(false,false)[bag] - scope-70
+|---e: New For Each(false,false)[bag] - scope-68
     |   |
-    |   Project[int][0] - scope-66
+    |   Project[int][0] - scope-64
     |   |
-    |   POUserFunc(org.apache.pig.builtin.AlgebraicMathBase$Initial)[tuple] - scope-67
+    |   POUserFunc(org.apache.pig.builtin.AlgebraicMathBase$Initial)[tuple] - scope-65
     |   |
-    |   |---Project[bag][1] - scope-68
+    |   |---Project[bag][1] - scope-66
     |       |
-    |       |---Project[bag][1] - scope-69
+    |       |---Project[bag][1] - scope-67
     |
     |---Pre Combiner Local Rearrange[tuple]{Unknown} - scope-71
         |
@@ -70,7 +70,7 @@ d: Local Rearrange[tuple]{int}(false) -
 Tez vertex group scope-55	<-	 [scope-29, scope-30]	->	 scope-35
 # No plan on vertex group
 Tez vertex scope-35
-# Combine plan on edge <scope-30>
+# Combine plan on edge <scope-29>
 d: Local Rearrange[tuple]{int}(false) - scope-52	->	 scope-35
 |   |
 |   Project[int][0] - scope-54
@@ -84,7 +84,7 @@ d: Local Rearrange[tuple]{int}(false) -
     |   |---Project[bag][1] - scope-44
     |
     |---d: Package(CombinerPackager)[tuple]{int} - scope-47
-# Combine plan on edge <scope-29>
+# Combine plan on edge <scope-30>
 d: Local Rearrange[tuple]{int}(false) - scope-52	->	 scope-35
 |   |
 |   Project[int][0] - scope-54

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-4-OPTOFF.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-4-OPTOFF.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-4-OPTOFF.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-4-OPTOFF.gld Tue Dec  9 20:40:03 2014
@@ -66,4 +66,4 @@ e: Store(file:///tmp/output:org.apache.p
     |   |
     |   Project[int][0] - scope-67
     |
-    |---POShuffledValueInputTez - scope-78	<-	 [scope-76, scope-75]
+    |---POShuffledValueInputTez - scope-78	<-	 [scope-75, scope-76]

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-6-OPTOFF.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-6-OPTOFF.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-6-OPTOFF.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-6-OPTOFF.gld Tue Dec  9 20:40:03 2014
@@ -6,8 +6,8 @@
 #--------------------------------------------------
 Tez vertex scope-114	->	Tez vertex scope-116,
 Tez vertex scope-115	->	Tez vertex scope-116,
-Tez vertex scope-116	->	Tez vertex scope-138,Tez vertex scope-128,
-Tez vertex scope-128	->	Tez vertex scope-138,Tez vertex scope-120,
+Tez vertex scope-116	->	Tez vertex scope-128,Tez vertex scope-138,
+Tez vertex scope-128	->	Tez vertex scope-120,Tez vertex scope-138,
 Tez vertex scope-138	->	Tez vertex scope-142,
 Tez vertex scope-120	->	Tez vertex scope-142,
 Tez vertex scope-142
@@ -65,7 +65,7 @@ Local Rearrange[tuple]{tuple}(false) - s
             |---POShuffledValueInputTez - scope-117	<-	 [scope-114, scope-115]
 Tez vertex scope-128
 # Plan on vertex
-POValueOutputTez - scope-137	->	 [scope-138, scope-120]
+POValueOutputTez - scope-137	->	 [scope-120, scope-138]
 |
 |---New For Each(false)[tuple] - scope-136
     |   |

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-6.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-6.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-6.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-6.gld Tue Dec  9 20:40:03 2014
@@ -7,7 +7,7 @@
 Tez vertex scope-29	->	Tez vertex group scope-63,Tez vertex group scope-64,
 Tez vertex scope-30	->	Tez vertex group scope-63,Tez vertex group scope-64,
 Tez vertex group scope-64	->	Tez vertex scope-43,
-Tez vertex scope-43	->	Tez vertex scope-53,Tez vertex scope-35,
+Tez vertex scope-43	->	Tez vertex scope-35,Tez vertex scope-53,
 Tez vertex group scope-63	->	Tez vertex scope-53,
 Tez vertex scope-53	->	Tez vertex scope-57,
 Tez vertex scope-35	->	Tez vertex scope-57,
@@ -23,9 +23,9 @@ Local Rearrange[tuple]{tuple}(false) - s
     |   |
     |   Project[int][0] - scope-71
     |   |
-    |   POUserFunc(org.apache.pig.impl.builtin.GetMemNumRows)[tuple] - scope-72
+    |   POUserFunc(org.apache.pig.impl.builtin.GetMemNumRows)[tuple] - scope-73
     |   |
-    |   |---Project[tuple][*] - scope-73
+    |   |---Project[tuple][*] - scope-72
     |
     |---PoissonSample - scope-70
         |
@@ -54,9 +54,9 @@ Local Rearrange[tuple]{tuple}(false) - s
     |   |
     |   Project[int][0] - scope-81
     |   |
-    |   POUserFunc(org.apache.pig.impl.builtin.GetMemNumRows)[tuple] - scope-82
+    |   POUserFunc(org.apache.pig.impl.builtin.GetMemNumRows)[tuple] - scope-83
     |   |
-    |   |---Project[tuple][*] - scope-83
+    |   |---Project[tuple][*] - scope-82
     |
     |---PoissonSample - scope-80
         |
@@ -79,7 +79,7 @@ Tez vertex group scope-64	<-	 [scope-29,
 # No plan on vertex group
 Tez vertex scope-43
 # Plan on vertex
-POValueOutputTez - scope-52	->	 [scope-53, scope-35]
+POValueOutputTez - scope-52	->	 [scope-35, scope-53]
 |
 |---New For Each(false)[tuple] - scope-51
     |   |

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-7-OPTOFF.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-7-OPTOFF.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-7-OPTOFF.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-7-OPTOFF.gld Tue Dec  9 20:40:03 2014
@@ -6,7 +6,7 @@
 #--------------------------------------------------
 Tez vertex scope-93	->	Tez vertex scope-95,
 Tez vertex scope-94	->	Tez vertex scope-95,
-Tez vertex scope-95	->	Tez vertex scope-117,Tez vertex scope-107,
+Tez vertex scope-95	->	Tez vertex scope-107,Tez vertex scope-117,
 Tez vertex scope-107	->	Tez vertex scope-117,
 Tez vertex scope-117	->	Tez vertex scope-119,
 Tez vertex scope-119

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-7.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-7.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-7.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-7.gld Tue Dec  9 20:40:03 2014
@@ -14,9 +14,9 @@ Tez vertex scope-46
 
 Tez vertex scope-20
 # Plan on vertex
-Local Rearrange[tuple]{tuple}(false) - scope-54	->	 scope-34
+Local Rearrange[tuple]{tuple}(false) - scope-55	->	 scope-34
 |   |
-|   Constant(DummyVal) - scope-55
+|   Constant(DummyVal) - scope-56
 |
 |---New For Each(false,true)[tuple] - scope-62
     |   |
@@ -26,11 +26,11 @@ Local Rearrange[tuple]{tuple}(false) - s
     |   |
     |   |---Project[tuple][*] - scope-60
     |
-    |---ReservoirSample - scope-53
+    |---ReservoirSample - scope-58
         |
-        |---d: Local Rearrange[tuple]{int}(false) - scope-57	->	 scope-44
+        |---d: Local Rearrange[tuple]{int}(false) - scope-53	->	 scope-44
             |   |
-            |   Project[int][0] - scope-58
+            |   Project[int][0] - scope-54
             |
             |---a: New For Each(false,false)[bag] - scope-7
                 |   |
@@ -45,9 +45,9 @@ Local Rearrange[tuple]{tuple}(false) - s
                 |---a: Load(file:///tmp/input:org.apache.pig.builtin.PigStorage) - scope-0
 Tez vertex scope-21
 # Plan on vertex
-Local Rearrange[tuple]{tuple}(false) - scope-64	->	 scope-34
+Local Rearrange[tuple]{tuple}(false) - scope-65	->	 scope-34
 |   |
-|   Constant(DummyVal) - scope-65
+|   Constant(DummyVal) - scope-66
 |
 |---New For Each(false,true)[tuple] - scope-72
     |   |
@@ -57,11 +57,11 @@ Local Rearrange[tuple]{tuple}(false) - s
     |   |
     |   |---Project[tuple][*] - scope-70
     |
-    |---ReservoirSample - scope-63
+    |---ReservoirSample - scope-68
         |
-        |---d: Local Rearrange[tuple]{int}(false) - scope-67	->	 scope-44
+        |---d: Local Rearrange[tuple]{int}(false) - scope-63	->	 scope-44
             |   |
-            |   Project[int][0] - scope-68
+            |   Project[int][0] - scope-64
             |
             |---c: New For Each(false,false)[bag] - scope-15
                 |   |

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-8-OPTOFF.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-8-OPTOFF.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-8-OPTOFF.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-8-OPTOFF.gld Tue Dec  9 20:40:03 2014
@@ -68,4 +68,4 @@ d: Store(file:///tmp/output:org.apache.p
 |
 |---d: Limit - scope-58
     |
-    |---POShuffledValueInputTez - scope-71	<-	 [scope-67, scope-62]
+    |---POShuffledValueInputTez - scope-71	<-	 [scope-62, scope-67]

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-8.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-8.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-8.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-8.gld Tue Dec  9 20:40:03 2014
@@ -68,4 +68,4 @@ d: Store(file:///tmp/output:org.apache.p
 |
 |---d: Limit - scope-21
     |
-    |---POShuffledValueInputTez - scope-34	<-	 [scope-30, scope-25]
+    |---POShuffledValueInputTez - scope-34	<-	 [scope-25, scope-30]

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-9-OPTOFF.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-9-OPTOFF.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-9-OPTOFF.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-9-OPTOFF.gld Tue Dec  9 20:40:03 2014
@@ -86,4 +86,4 @@ Tez vertex scope-109
 |       |       |
 |       |       |---Constant(500) - scope-98
 |
-|---POShuffledValueInputTez - scope-110	<-	 [scope-108, scope-102]
+|---POShuffledValueInputTez - scope-110	<-	 [scope-102, scope-108]

Modified: pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-9.gld
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-9.gld?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-9.gld (original)
+++ pig/trunk/test/org/apache/pig/test/data/GoldenFiles/tez/TEZC-Union-9.gld Tue Dec  9 20:40:03 2014
@@ -6,8 +6,8 @@
 #--------------------------------------------------
 Tez vertex scope-40	->	Tez vertex group scope-58,Tez vertex group scope-59,
 Tez vertex scope-46	->	Tez vertex group scope-58,Tez vertex group scope-59,
-Tez vertex group scope-59
 Tez vertex group scope-58
+Tez vertex group scope-59
 
 Tez vertex scope-40
 # Plan on vertex
@@ -105,7 +105,7 @@ Tez vertex scope-46
     |   |---Project[bytearray][0] - scope-23
     |
     |---b: Load(file:///tmp/input:org.apache.pig.builtin.PigStorage) - scope-19
-Tez vertex group scope-59	<-	 [scope-40, scope-46]	->	 null
-# No plan on vertex group
 Tez vertex group scope-58	<-	 [scope-40, scope-46]	->	 null
 # No plan on vertex group
+Tez vertex group scope-59	<-	 [scope-40, scope-46]	->	 null
+# No plan on vertex group

Modified: pig/trunk/test/org/apache/pig/tez/TestTezJobControlCompiler.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/tez/TestTezJobControlCompiler.java?rev=1644173&r1=1644172&r2=1644173&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/tez/TestTezJobControlCompiler.java (original)
+++ pig/trunk/test/org/apache/pig/tez/TestTezJobControlCompiler.java Tue Dec  9 20:40:03 2014
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTru
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Properties;
@@ -267,7 +268,9 @@ public class TestTezJobControlCompiler {
                 + "d = group c by name;"
                 + "store d into 'o2';";
         Pair<TezOperPlan, DAG> compiledPlan = compile(query);
-        TezOperator leafOper = compiledPlan.first.getLeaves().get(0);
+        List<TezOperator> leaves = compiledPlan.first.getLeaves();
+        Collections.sort(leaves);
+        TezOperator leafOper = leaves.get(1);
         Vertex leafVertex = compiledPlan.second.getVertex(leafOper.getOperatorKey().toString());
         assertEquals(leafVertex.getParallelism(), 7);
     }