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/05/16 18:46:57 UTC

svn commit: r1595262 [2/2] - in /pig/branches/tez: ./ shims/test/hadoop23/org/apache/pig/test/ src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/ src/org/apache/pig/backend/hadoop/executionengine/tez/ src/org/apache/pig/backend/hadoop/ex...

Modified: pig/branches/tez/test/org/apache/pig/test/TestGrunt.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestGrunt.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestGrunt.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestGrunt.java Fri May 16 16:46:56 2014
@@ -54,12 +54,13 @@ import org.apache.pig.test.Util.ProcessR
 import org.apache.pig.tools.grunt.Grunt;
 import org.apache.pig.tools.pigscript.parser.ParseException;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestGrunt {
 
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private String basedir = "test/org/apache/pig/test/data";
 
     @BeforeClass
@@ -72,10 +73,15 @@ public class TestGrunt {
         cluster.shutDown();
     }
 
+    @Before
+    public void setup() {
+        Util.resetStateForExecModeSwitch();
+    }
+
 
     @Test
     public void testCopyFromLocal() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "copyFromLocal README.txt sh_copy ;";
@@ -91,7 +97,7 @@ public class TestGrunt {
 
     @Test
     public void testDefine() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "define myudf org.apache.pig.builtin.AVG();\n";
@@ -111,7 +117,7 @@ public class TestGrunt {
 
     @Test
     public void testBagSchema() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'input1' as (b: bag{t:(i: int, c:chararray, f: float)});\n";
@@ -126,7 +132,7 @@ public class TestGrunt {
 
     @Test
     public void testBagSchemaFail() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'input1'as (b: bag{t:(i: int, c:chararray, f: float)});\n";
@@ -147,7 +153,7 @@ public class TestGrunt {
 
     @Test
     public void testBagConstant() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'input1'; b = foreach a generate {(1, '1', 0.4f),(2, '2', 0.45)};\n";
@@ -162,7 +168,7 @@ public class TestGrunt {
 
     @Test
     public void testBagConstantWithSchema() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'input1'; b = foreach a generate "
@@ -179,7 +185,7 @@ public class TestGrunt {
 
     @Test
     public void testBagConstantInForeachBlock() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'input1'; "
@@ -195,7 +201,7 @@ public class TestGrunt {
 
     @Test
     public void testBagConstantWithSchemaInForeachBlock() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'input1'; "
@@ -212,7 +218,7 @@ public class TestGrunt {
 
     @Test
     public void testParsingAsInForeachBlock() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast); "
@@ -229,7 +235,7 @@ public class TestGrunt {
 
     @Test
     public void testParsingAsInForeachWithOutBlock() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast); "
@@ -245,7 +251,7 @@ public class TestGrunt {
 
     @Test
     public void testParsingWordWithAsInForeachBlock() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast); "
@@ -261,7 +267,7 @@ public class TestGrunt {
 
     @Test
     public void testParsingWordWithAsInForeachWithOutBlock() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast); "
@@ -277,7 +283,7 @@ public class TestGrunt {
 
     @Test
     public void testParsingWordWithAsInForeachWithOutBlock2() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "cash = load 'foo' as (foo, fast); "
@@ -294,7 +300,7 @@ public class TestGrunt {
 
     @Test
     public void testParsingGenerateInForeachBlock() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast, regenerate); "
@@ -310,7 +316,7 @@ public class TestGrunt {
 
     @Test
     public void testParsingGenerateInForeachWithOutBlock() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast, regenerate); "
@@ -326,7 +332,7 @@ public class TestGrunt {
 
     @Test
     public void testParsingAsGenerateInForeachBlock() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast, regenerate); "
@@ -345,7 +351,7 @@ public class TestGrunt {
 
     @Test
     public void testParsingAsGenerateInForeachWithOutBlock() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast, regenerate); "
@@ -364,7 +370,7 @@ public class TestGrunt {
 
     @Test
     public void testRunStatment() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast, regenerate);"
@@ -381,7 +387,7 @@ public class TestGrunt {
 
     @Test
     public void testExecStatment() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
         boolean caught = false;
 
@@ -405,7 +411,7 @@ public class TestGrunt {
 
     @Test
     public void testRunStatmentNested() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast, regenerate); run "
@@ -421,7 +427,7 @@ public class TestGrunt {
 
     @Test
     public void testExecStatmentNested() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
         boolean caught = false;
 
@@ -444,7 +450,7 @@ public class TestGrunt {
 
     @Test
     public void testErrorLineNumber() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "A = load 'x' as ( u:int, v:chararray );\n" +
@@ -469,7 +475,7 @@ public class TestGrunt {
 
     @Test
     public void testExplainEmpty() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast, regenerate); run "
@@ -485,7 +491,7 @@ public class TestGrunt {
 
     @Test
     public void testExplainScript() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast, regenerate); explain -script "
@@ -510,7 +516,7 @@ public class TestGrunt {
      */
     @Test
     public void testExplainScriptIsEachStatementValidated() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast, regenerate);" +
@@ -648,7 +654,7 @@ public class TestGrunt {
     @Test
     public void testExplainScript2() throws Throwable {
 
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "explain -script "
@@ -693,7 +699,7 @@ public class TestGrunt {
 
     @Test
     public void testExplainBrief() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast, regenerate); explain -brief -script "
@@ -709,7 +715,7 @@ public class TestGrunt {
 
     @Test
     public void testExplainDot() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast, regenerate); explain -dot -script "
@@ -725,7 +731,7 @@ public class TestGrunt {
 
     @Test
     public void testExplainOut() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "a = load 'foo' as (foo, fast, regenerate); explain -out /tmp -script "
@@ -741,7 +747,7 @@ public class TestGrunt {
 
     @Test
     public void testPartialExecution() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
         FileLocalizer.setInitialized(false);
 
@@ -762,7 +768,7 @@ public class TestGrunt {
 
     @Test
     public void testFileCmds() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd =
@@ -793,7 +799,7 @@ public class TestGrunt {
 
     @Test
     public void testCD() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd =
@@ -818,7 +824,7 @@ public class TestGrunt {
 
     @Test
     public void testDump() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd =
@@ -842,7 +848,8 @@ public class TestGrunt {
 
     @Test
     public void testIllustrate() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        if (!Util.isMapredExecType(cluster.getExecType())) return;
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd =
@@ -866,7 +873,8 @@ public class TestGrunt {
 
     @Test
     public void testKeepGoing() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        if (!Util.isMapredExecType(cluster.getExecType())) return;
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
 
         PigContext context = server.getPigContext();
 
@@ -896,7 +904,7 @@ public class TestGrunt {
 
     @Test
     public void testKeepGoigFailed() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
         Util.copyFromLocalToCluster(cluster, "test/org/apache/pig/test/data/passwd", "passwd");
         String strCmd =
@@ -953,7 +961,8 @@ public class TestGrunt {
 
     @Test
     public void testStopOnFailure() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        if (!Util.isMapredExecType(cluster.getExecType())) return;
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
         context.getProperties().setProperty("stop.on.failure", ""+true);
 
@@ -990,7 +999,7 @@ public class TestGrunt {
     @Test
     public void testFsCommand() throws Throwable {
 
-        PigServer server = new PigServer(ExecType.MAPREDUCE,cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(),cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd =
@@ -1018,7 +1027,7 @@ public class TestGrunt {
     public void testShellCommand(){
 
         try {
-            PigServer server = new PigServer(ExecType.MAPREDUCE,cluster.getProperties());
+            PigServer server = new PigServer(cluster.getExecType(),cluster.getProperties());
             PigContext context = server.getPigContext();
 
             String strRemoveFile = "rm";
@@ -1143,7 +1152,7 @@ public class TestGrunt {
 
     @Test
     public void testSetPriority() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "set job.priority high\n";
@@ -1159,7 +1168,7 @@ public class TestGrunt {
 
     @Test
     public void testSetWithQuotes() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "set job.priority 'high'\n";
@@ -1175,7 +1184,7 @@ public class TestGrunt {
 
     @Test
     public void testRegisterWithQuotes() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "register 'pig-withouthadoop.jar'\n";
@@ -1192,7 +1201,7 @@ public class TestGrunt {
 
     @Test
     public void testRegisterWithoutQuotes() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "register pig-withouthadoop.jar\n";
@@ -1218,7 +1227,7 @@ public class TestGrunt {
 
         Util.createLocalInputFile( "testRegisterScripts.py", script);
 
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String strCmd = "register testRegisterScripts.py using jython as pig\n";
@@ -1256,7 +1265,7 @@ public class TestGrunt {
     // than an unrelated EOF error message
     @Test
     public void testBlockErrMessage() throws Throwable {
-        PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
 
         String script = "A = load 'inputdata' using PigStorage() as ( curr_searchQuery );\n" +

Modified: pig/branches/tez/test/org/apache/pig/test/TestImplicitSplit.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestImplicitSplit.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestImplicitSplit.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestImplicitSplit.java Fri May 16 16:46:56 2014
@@ -19,13 +19,13 @@ package org.apache.pig.test;
 
 
 import static org.apache.pig.test.utils.TypeCheckingTestUtil.printTypeGraph;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.util.HashMap;
 import java.util.Iterator;
 
-
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.newplan.logical.relational.LogicalRelationalOperator;
@@ -37,22 +37,22 @@ import org.junit.Test;
 
 public class TestImplicitSplit {
     private PigServer pigServer;
-    static MiniCluster cluster = MiniCluster.buildCluster();
-    
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
+
     @Before
     public void setUp() throws Exception {
-        pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
     }
 
     @After
     public void tearDown() throws Exception {
     }
-    
+
     @AfterClass
     public static void oneTimeTearDown() throws Exception {
         cluster.shutDown();
     }
-    
+
     @Test
     public void testImplicitSplit() throws Exception{
         int LOOP_SIZE = 20;
@@ -76,20 +76,20 @@ public class TestImplicitSplit {
         assertEquals(20, cnt);
         Util.deleteFile(cluster, inputFileName);
     }
-    
+
     @Test
     public void testImplicitSplitInCoGroup() throws Exception {
         // this query is similar to the one reported in JIRA - PIG-537
         // Create input file
         String input1 = "testImplicitSplitInCoGroup-input1.txt";
         String input2 = "testImplicitSplitInCoGroup-input2.txt";
-        Util.createInputFile(cluster, input1, 
+        Util.createInputFile(cluster, input1,
                 new String[] {"a:1", "b:2", "b:20", "c:3", "c:30"});
-        Util.createInputFile(cluster, input2, 
+        Util.createInputFile(cluster, input2,
                 new String[] {"a:first", "b:second", "c:third"});
-        pigServer.registerQuery("a = load '" + input1 + 
+        pigServer.registerQuery("a = load '" + input1 +
                 "' using PigStorage(':') as (name:chararray, marks:int);");
-        pigServer.registerQuery("b = load '" + input2 + 
+        pigServer.registerQuery("b = load '" + input2 +
                 "' using PigStorage(':') as (name:chararray, rank:chararray);");
         pigServer.registerQuery("c = cogroup a by name, b by name;");
         pigServer.registerQuery("d = foreach c generate group, FLATTEN(a.marks) as newmarks;");
@@ -101,7 +101,7 @@ public class TestImplicitSplit {
         results.put(3, new Object[] { "c", 3, "c", 3 });
         results.put(20, new Object[] { "b", 20, "b", 20 });
         results.put(30, new Object[] { "c", 30, "c", 30 });
-        
+
         Iterator<Tuple> it = pigServer.openIterator("f");
         while(it.hasNext()) {
             Tuple t = it.next();
@@ -109,13 +109,13 @@ public class TestImplicitSplit {
             Integer group = (Integer)t.get(0);
             Object[] groupValues = results.get(group);
             for(int i = 0; i < 4; i++) {
-                assertEquals(groupValues[i], t.get(i+1));    
+                assertEquals(groupValues[i], t.get(i+1));
             }
         }
         Util.deleteFile(cluster, input1);
         Util.deleteFile(cluster, input2);
     }
-    
+
     @Test
     public void testImplicitSplitInCoGroup2() throws Exception {
         // this query is similar to the one reported in JIRA - PIG-537
@@ -127,12 +127,12 @@ public class TestImplicitSplit {
                        "f = foreach e generate group, flatten(a), flatten(d);" +
                        "store f into 'output';";
         org.apache.pig.newplan.logical.relational.LogicalPlan plan = Util.buildLp(pigServer, query);
-        
+
         printTypeGraph(plan) ;
-        
+
         // this will run ImplicitSplitInserter
         Util.optimizeNewLP(plan);
-        
+
         // get Schema of leaf and compare:
         String schema = "group:int,a::name:chararray,a::marks:int,d::group:chararray,d::newmarks:int";
         LogicalSchema sch = ((LogicalRelationalOperator)plan.getSinks().get(0)).getSchema();

Modified: pig/branches/tez/test/org/apache/pig/test/TestInputOutputMiniClusterFileValidator.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestInputOutputMiniClusterFileValidator.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestInputOutputMiniClusterFileValidator.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestInputOutputMiniClusterFileValidator.java Fri May 16 16:46:56 2014
@@ -30,7 +30,6 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.PigException;
 import org.apache.pig.PigServer;
@@ -55,15 +54,15 @@ import org.junit.Before;
 import org.junit.Test;
 
 public class TestInputOutputMiniClusterFileValidator {
-    private static MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private PigServer pig;
     private PigContext ctx;
 
     @Before
     public void setUp() throws Exception {
-        ctx = new PigContext(ExecType.MAPREDUCE, cluster.getProperties());
+        ctx = new PigContext(cluster.getExecType(), cluster.getProperties());
         ctx.connect() ;
-        pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        pig = new PigServer(ctx);
     }
 
     @AfterClass
@@ -165,7 +164,6 @@ public class TestInputOutputMiniClusterF
     @Test
     public void testValidationNeg() throws Throwable{
 
-        PigServer pig = new PigServer(ExecType.MAPREDUCE,cluster.getProperties());
         try{
             pig.setBatchOn();
             pig.registerQuery("A = load 'inputfile' using PigStorage () as (a:int);");

Modified: pig/branches/tez/test/org/apache/pig/test/TestJoin.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestJoin.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestJoin.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestJoin.java Fri May 16 16:46:56 2014
@@ -40,7 +40,6 @@ import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.data.TupleFactory;
-import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.util.LogUtils;
 import org.apache.pig.impl.util.Utils;
@@ -60,12 +59,12 @@ import com.google.common.collect.Sets;
 
 public class TestJoin {
 
-    static MiniCluster cluster;
+    private static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private PigServer pigServer;
 
     TupleFactory mTf = TupleFactory.getInstance();
     BagFactory mBf = BagFactory.getInstance();
-    ExecType[] execTypes = new ExecType[] {ExecType.LOCAL, ExecType.MAPREDUCE};
+    private static ExecType[] execTypes = new ExecType[] {ExecType.LOCAL, cluster.getExecType()};
 
     @AfterClass
     public static void oneTimeTearDown() throws Exception {
@@ -73,12 +72,9 @@ public class TestJoin {
     }
 
     private void setUp(ExecType execType) throws ExecException {
-        // cause a reinitialization of FileLocalizer's
-        // internal state
-        FileLocalizer.setInitialized(false);
-        if(execType == ExecType.MAPREDUCE) {
-            cluster =  MiniCluster.buildCluster();
-            pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        Util.resetStateForExecModeSwitch();
+        if(execType == cluster.getExecType()) {
+            pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
         } else if(execType == ExecType.LOCAL) {
             pigServer = new PigServer(ExecType.LOCAL);
         }
@@ -86,7 +82,7 @@ public class TestJoin {
 
     private String createInputFile(ExecType execType, String fileNameHint, String[] data) throws IOException {
         String fileName = "";
-        if(execType == ExecType.MAPREDUCE) {
+        if(execType == cluster.getExecType()) {
             Util.createInputFile(cluster, fileNameHint, data);
             fileName = fileNameHint;
         } else if (execType == ExecType.LOCAL) {
@@ -97,7 +93,7 @@ public class TestJoin {
     }
 
     private void deleteInputFile(ExecType execType, String fileName) throws IOException {
-        if(execType == ExecType.MAPREDUCE) {
+        if(execType == cluster.getExecType()) {
             Util.deleteFile(cluster, fileName);
         } else if(execType == ExecType.LOCAL){
             fileName = fileName.replace("file://", "");
@@ -107,7 +103,7 @@ public class TestJoin {
 
     @Test
     public void testJoinWithMissingFieldsInTuples() throws IOException{
-        setUp(ExecType.MAPREDUCE);
+        setUp(cluster.getExecType());
         String[] input1 = {
                 "ff ff ff",
                 "",
@@ -126,16 +122,16 @@ public class TestJoin {
                 ""
                 };
 
-        String firstInput = createInputFile(ExecType.MAPREDUCE, "a.txt", input1);
-        String secondInput = createInputFile(ExecType.MAPREDUCE, "b.txt", input2);
+        String firstInput = createInputFile(cluster.getExecType(), "a.txt", input1);
+        String secondInput = createInputFile(cluster.getExecType(), "b.txt", input2);
         String script = "a = load 'a.txt'  using PigStorage(' ');" +
         "b = load 'b.txt'  using PigStorage('\u0001');" +
         "c = join a by $0, b by $0;";
         Util.registerMultiLineQuery(pigServer, script);
         Iterator<Tuple> it = pigServer.openIterator("c");
         assertFalse(it.hasNext());
-        deleteInputFile(ExecType.MAPREDUCE, firstInput);
-        deleteInputFile(ExecType.MAPREDUCE, secondInput);
+        deleteInputFile(cluster.getExecType(), firstInput);
+        deleteInputFile(cluster.getExecType(), secondInput);
     }
 
     @Test
@@ -582,13 +578,12 @@ public class TestJoin {
             Util.registerMultiLineQuery(pigServer, script);
             Iterator<Tuple> it = pigServer.openIterator("c");
 
-            assertTrue(it.hasNext());
-            Tuple t = it.next();
-            assertEquals("((1,a),(1,b))", t.toString());
-
-            assertTrue(it.hasNext());
-            t = it.next();
-            assertEquals("((2,aa),(2,bb))", t.toString());
+            List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
+                    new String[] {
+                            "((1,'a'),(1,'b'))",
+                            "((2,'aa'),(2,'bb'))"
+                            });
+            Util.checkQueryOutputsAfterSort(it, expectedResults);
 
             deleteInputFile(execType, firstInput);
             deleteInputFile(execType, secondInput);
@@ -617,11 +612,9 @@ public class TestJoin {
             Util.registerMultiLineQuery(pigServer, script);
             Iterator<Tuple> it = pigServer.openIterator("c");
 
-            assertTrue(it.hasNext());
-            Tuple t = it.next();
-            assertEquals("(2,aa,2,aa)", t.toString());
-
-            assertFalse(it.hasNext());
+            List<Tuple> expectedResults = Util
+                    .getTuplesFromConstantTupleStrings(new String[] { "(2,'aa',2,'aa')" });
+            Util.checkQueryOutputs(it, expectedResults);
 
             deleteInputFile(execType, firstInput);
             deleteInputFile(execType, secondInput);

Modified: pig/branches/tez/test/org/apache/pig/test/TestJoinSmoke.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestJoinSmoke.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestJoinSmoke.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestJoinSmoke.java Fri May 16 16:46:56 2014
@@ -23,7 +23,6 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.Iterator;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.BagFactory;
@@ -43,11 +42,10 @@ public class TestJoinSmoke {
     private static final String SKEW_INPUT_FILE5 = "SkewedJoinInput5.txt";
 
     private PigServer pigServer;
-    private static MiniCluster cluster = MiniCluster.buildCluster();
-    private File tmpFile;
+    private static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
 
     public TestJoinSmoke() throws ExecException, IOException {
-        pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
     }
 
     @Before

Modified: pig/branches/tez/test/org/apache/pig/test/TestKeyTypeDiscoveryVisitor.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestKeyTypeDiscoveryVisitor.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestKeyTypeDiscoveryVisitor.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestKeyTypeDiscoveryVisitor.java Fri May 16 16:46:56 2014
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertFal
 
 import java.util.Iterator;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.Tuple;
@@ -38,7 +37,7 @@ import org.junit.Test;
  */
 public class TestKeyTypeDiscoveryVisitor {
 
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private PigServer pigServer;
 
     TupleFactory mTf = TupleFactory.getInstance();
@@ -46,7 +45,7 @@ public class TestKeyTypeDiscoveryVisitor
 
     @Before
     public void setUp() throws Exception{
-        pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()); //TODO this doesn't need to be M/R mode
+        pigServer = new PigServer(cluster.getExecType(), cluster.getProperties()); //TODO this doesn't need to be M/R mode
     }
 
     @AfterClass

Modified: pig/branches/tez/test/org/apache/pig/test/TestNewPlanImplicitSplit.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestNewPlanImplicitSplit.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestNewPlanImplicitSplit.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestNewPlanImplicitSplit.java Fri May 16 16:46:56 2014
@@ -17,12 +17,12 @@
  */
 package org.apache.pig.test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 import java.util.HashMap;
 import java.util.Iterator;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.newplan.logical.optimizer.LogicalPlanOptimizer;
@@ -34,23 +34,23 @@ import org.junit.Test;
 
 public class TestNewPlanImplicitSplit {
     private PigServer pigServer;
-    static MiniCluster cluster = MiniCluster.buildCluster();
-    
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
+
     @Before
     public void setUp() throws Exception {
-        pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
         pigServer.getPigContext().getProperties().setProperty("pig.usenewlogicalplan", "true");
     }
 
     @After
     public void tearDown() throws Exception {
     }
-    
+
     @AfterClass
     public static void oneTimeTearDown() throws Exception {
         cluster.shutDown();
     }
-    
+
     @Test
     public void testImplicitSplit() throws Exception{
         int LOOP_SIZE = 20;
@@ -74,20 +74,20 @@ public class TestNewPlanImplicitSplit {
         assertEquals(20, cnt);
         Util.deleteFile(cluster, inputFileName);
     }
-    
+
     @Test
     public void testImplicitSplitInCoGroup() throws Exception {
         // this query is similar to the one reported in JIRA - PIG-537
         // Create input file
         String input1 = "testImplicitSplitInCoGroup-input1.txt";
         String input2 = "testImplicitSplitInCoGroup-input2.txt";
-        Util.createInputFile(cluster, input1, 
+        Util.createInputFile(cluster, input1,
                 new String[] {"a:1", "b:2", "b:20", "c:3", "c:30"});
-        Util.createInputFile(cluster, input2, 
+        Util.createInputFile(cluster, input2,
                 new String[] {"a:first", "b:second", "c:third"});
-        pigServer.registerQuery("a = load '" + input1 + 
+        pigServer.registerQuery("a = load '" + input1 +
                 "' using PigStorage(':') as (name:chararray, marks:int);");
-        pigServer.registerQuery("b = load '" + input2 + 
+        pigServer.registerQuery("b = load '" + input2 +
                 "' using PigStorage(':') as (name:chararray, rank:chararray);");
         pigServer.registerQuery("c = cogroup a by name, b by name;");
         pigServer.registerQuery("d = foreach c generate group, FLATTEN(a.marks) as newmarks;");
@@ -100,7 +100,7 @@ public class TestNewPlanImplicitSplit {
         results.put(20, new Object[] { "b", 20, "b", 20 });
         results.put(30, new Object[] { "c", 30, "c", 30 });
         pigServer.explain("f", System.out);
-        
+
         Iterator<Tuple> it = pigServer.openIterator("f");
         while(it.hasNext()) {
             Tuple t = it.next();
@@ -108,13 +108,13 @@ public class TestNewPlanImplicitSplit {
             Integer group = (Integer)t.get(0);
             Object[] groupValues = results.get(group);
             for(int i = 0; i < 4; i++) {
-                assertEquals(groupValues[i], t.get(i+1));    
+                assertEquals(groupValues[i], t.get(i+1));
             }
         }
         Util.deleteFile(cluster, input1);
         Util.deleteFile(cluster, input2);
     }
-    
+
     @Test
     public void testImplicitSplitInCoGroup2() throws Exception {
         // this query is similar to the one reported in JIRA - PIG-537
@@ -125,13 +125,13 @@ public class TestNewPlanImplicitSplit {
         "e = cogroup a by marks, d by newmarks;"+
         "f = foreach e generate group, flatten(a), flatten(d);" +
         "store f into 'out';");
-        
+
         org.apache.pig.newplan.logical.relational.LogicalPlan plan = Util.buildLp(pigServer, query);
-        
+
 
         // this will run ImplicitSplitInserter
         new LogicalPlanOptimizer( plan, 5, null ).optimize();
-        
+
         // get Schema of leaf and compare:
         String expectedSchema = "group:int,a::name:chararray,a::marks:int,d::group:chararray,d::newmarks:int";
         assertEquals(expectedSchema, ((LogicalRelationalOperator)plan.getSinks().get(0)).getSchema().toString(false));

Modified: pig/branches/tez/test/org/apache/pig/test/TestPigContext.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestPigContext.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestPigContext.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestPigContext.java Fri May 16 16:46:56 2014
@@ -34,9 +34,6 @@ import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.util.JavaCompilerHelper;
-import org.apache.pig.tools.pigstats.PigStats;
-import org.apache.pig.tools.pigstats.ScriptState;
-import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -61,6 +58,7 @@ public class TestPigContext {
 
     @Before
     public void setUp() throws Exception {
+        Util.resetStateForExecModeSwitch();
         pigContext = new PigContext(ExecType.LOCAL, getProperties());
         input = File.createTempFile("PigContextTest-", ".txt");
     }
@@ -152,6 +150,7 @@ public class TestPigContext {
         int status = Util.executeJavaCommand("jar -cf " + jarFile +
                 " -C " + tmpDir.getAbsolutePath() + " " + "com");
         assertEquals(0, status);
+        Util.resetStateForExecModeSwitch();
         PigContext localPigContext = new PigContext(cluster.getExecType(), properties);
 
         // register jar using properties

Modified: pig/branches/tez/test/org/apache/pig/test/TestPigServerWithMacros.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestPigServerWithMacros.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestPigServerWithMacros.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestPigServerWithMacros.java Fri May 16 16:46:56 2014
@@ -39,10 +39,16 @@ import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage;
 import org.apache.pig.data.Tuple;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.Test;
 
 public class TestPigServerWithMacros {
-    private static MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
+
+    @Before
+    public void setup() throws Exception {
+        Util.resetStateForExecModeSwitch();
+    }
 
     @AfterClass
     public static void tearDown() throws Exception {
@@ -51,7 +57,7 @@ public class TestPigServerWithMacros {
 
     @Test
     public void testRegisterRemoteMacro() throws Throwable {
-        PigServer pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer pig = new PigServer(cluster.getExecType(), cluster.getProperties());
 
         String macroName = "util.pig";
         File macroFile = File.createTempFile("tmp", "");
@@ -94,42 +100,42 @@ public class TestPigServerWithMacros {
 
         pig.shutdown();
     }
-    
+
     @Test
     public void testRegisterResourceMacro() throws Throwable {
         PigServer pig = new PigServer(ExecType.LOCAL);
 
         String macrosFile = "test/pig/macros.pig";
         File macrosJarFile = File.createTempFile("macros", ".jar");
-        
+
         System.out.println("Creating macros jar " + macrosJarFile);
-        
+
         Manifest manifest = new Manifest();
         manifest.getMainAttributes().put(Attributes.Name.MANIFEST_VERSION, "1.0");
-        
+
         JarOutputStream jarStream = new JarOutputStream(new FileOutputStream(macrosJarFile), manifest);
-        
+
         JarEntry jarEntry = new JarEntry(macrosFile);
         jarEntry.setTime(System.currentTimeMillis());
-        jarStream.putNextEntry(jarEntry);        
-        
+        jarStream.putNextEntry(jarEntry);
+
         PrintWriter pw = new PrintWriter(jarStream);
         pw.println("DEFINE row_count_in_jar(X) RETURNS Z { Y = group $X all; $Z = foreach Y generate COUNT($X); };");
-        pw.close();        
-        
+        pw.close();
+
         jarStream.close();
-        
+
         Storage.Data data = resetData(pig);
         data.set("some_path", "(l:int)", tuple(tuple("1")), tuple(tuple("2")), tuple(tuple("3")), tuple(tuple("10")), tuple(tuple("11")));
-                
+
         System.out.println("Registering macros jar " + macrosJarFile);
         pig.registerJar(macrosJarFile.toString());
-        
+
         pig.registerQuery("import '" + macrosFile + "';");
         pig.registerQuery("a = load 'some_path' USING mock.Storage();");
         pig.registerQuery("b = row_count_in_jar(a);");
         Iterator<Tuple> iter = pig.openIterator("b");
-        
+
         assertTrue(((Long)iter.next().get(0))==5);
 
         pig.shutdown();

Modified: pig/branches/tez/test/org/apache/pig/test/TestPigStorage.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestPigStorage.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestPigStorage.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestPigStorage.java Fri May 16 16:46:56 2014
@@ -128,6 +128,7 @@ public class TestPigStorage  {
             props.put(entry.getKey(), entry.getValue());
         }
         props.setProperty("mapred.max.split.size", "20");
+        Util.resetStateForExecModeSwitch();
         PigServer pigServer = new PigServer(cluster.getExecType(), props);
         String[] inputs = {
                 "abcdefgh1", "abcdefgh2", "abcdefgh3",

Modified: pig/branches/tez/test/org/apache/pig/test/TestPoissonSampleLoader.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestPoissonSampleLoader.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestPoissonSampleLoader.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestPoissonSampleLoader.java Fri May 16 16:46:56 2014
@@ -29,7 +29,6 @@ import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.Tuple;
 import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -37,7 +36,6 @@ public class TestPoissonSampleLoader {
     private static final String INPUT_FILE1 = "SkewedJoinInput1.txt";
 
     private PigServer pigServer;
-    private static MiniCluster cluster = MiniCluster.buildCluster();
 
     public TestPoissonSampleLoader() throws ExecException, IOException {
         pigServer = new PigServer(ExecType.LOCAL);
@@ -55,11 +53,6 @@ public class TestPoissonSampleLoader {
         createFiles();
     }
 
-    @AfterClass
-    public static void oneTimeTearDown() throws Exception {
-        cluster.shutDown();
-    }
-
     private void createFiles() throws IOException {
         PrintWriter w = new PrintWriter(new FileWriter(INPUT_FILE1));
 
@@ -75,30 +68,11 @@ public class TestPoissonSampleLoader {
 
         w.close();
 
-        Util.copyFromLocalToCluster(cluster, INPUT_FILE1, INPUT_FILE1);
     }
 
     @After
     public void tearDown() throws Exception {
         new File(INPUT_FILE1).delete();
-
-        Util.deleteFile(cluster, INPUT_FILE1);
-    }
-
-    private int testNumSamples(String memUsage, String sampleRate) throws IOException {
-        pigServer.getPigContext().getProperties()
-                .setProperty("pig.skewedjoin.reduce.memusage", memUsage);
-        pigServer.getPigContext().getProperties()
-                .setProperty("pig.sksampler.samplerate", sampleRate);
-        pigServer.registerQuery("A = Load '" + INPUT_FILE1
-                + "' Using PoissonSampleLoader('PigStorage()', '100');");
-        Iterator<Tuple> iter = pigServer.openIterator("A");
-        int count = 0;
-        while (iter.hasNext()) {
-            count++;
-            iter.next();
-        }
-        return count;
     }
 
     /*

Modified: pig/branches/tez/test/org/apache/pig/test/TestRegisteredJarVisibility.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestRegisteredJarVisibility.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestRegisteredJarVisibility.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestRegisteredJarVisibility.java Fri May 16 16:46:56 2014
@@ -44,6 +44,7 @@ import org.apache.pig.PigServer;
 import org.apache.pig.impl.util.JarManager;
 import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -62,7 +63,7 @@ public class TestRegisteredJarVisibility
     // Actual data is not important. Reusing an existing input file.
     private static final File INPUT_FILE = new File("test/data/pigunit/top_queries_input_data.txt");
 
-    private static MiniCluster cluster;
+    private static MiniGenericCluster cluster;
     private static File jarFile;
 
     @BeforeClass()
@@ -95,7 +96,7 @@ public class TestRegisteredJarVisibility
 
         jar(filesToJar);
 
-        cluster = MiniCluster.buildCluster();
+        cluster = MiniGenericCluster.buildCluster();
         Util.copyFromLocalToCluster(cluster, INPUT_FILE.getPath(), INPUT_FILE.getName());
     }
 
@@ -104,6 +105,11 @@ public class TestRegisteredJarVisibility
         cluster.shutDown();
     }
 
+    @Before
+    public void setup() {
+        Util.resetStateForExecModeSwitch();
+    }
+
     @Test()
     public void testRegisteredJarVisibilitySchemaNotOnClasspath() {
         boolean exceptionThrown = false;
@@ -114,10 +120,10 @@ public class TestRegisteredJarVisibility
         }
         Assert.assertTrue(exceptionThrown);
     }
-    
+
     @Test
     public void testRegisterJarVisibilityMR() throws IOException {
-        PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
         testRegisteredJarVisibility(pigServer, INPUT_FILE.getName());
     }
 
@@ -126,7 +132,7 @@ public class TestRegisteredJarVisibility
         PigServer pigServer = new PigServer(ExecType.LOCAL, new Properties());
         testRegisteredJarVisibility(pigServer, INPUT_FILE.getAbsolutePath());
     }
-    
+
     public void testRegisteredJarVisibility(PigServer pigServer, String inputPath) throws IOException {
         String query = "register " + jarFile.getAbsolutePath() + ";\n"
                 + "a = load '" + inputPath

Modified: pig/branches/tez/test/org/apache/pig/test/TestStoreInstances.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestStoreInstances.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestStoreInstances.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestStoreInstances.java Fri May 16 16:46:56 2014
@@ -18,7 +18,6 @@
 package org.apache.pig.test;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
@@ -53,10 +52,10 @@ import org.junit.Test;
  * Test to ensure that same instance of store func is used for multiple
  *  backend tasks. This enables sharing of information between putNext and
  *  output committer
- * 
+ *
  */
 public class TestStoreInstances  {
-    static MiniCluster cluster ;
+    static MiniGenericCluster cluster ;
     private static final String INP_FILE_2NUMS = "TestStoreInstances";
 
     @Before
@@ -71,7 +70,7 @@ public class TestStoreInstances  {
 
     @BeforeClass
     public static void oneTimeSetup() throws IOException, Exception {
-        cluster = MiniCluster.buildCluster();
+        cluster = MiniGenericCluster.buildCluster();
 
         String[] input = {
                 "1\t5",
@@ -99,29 +98,30 @@ public class TestStoreInstances  {
      */
     @Test
     public void testBackendStoreCommunication() throws IOException, ParserException {
-        ExecType[] execTypes = { ExecType.MAPREDUCE, ExecType.LOCAL};
+        ExecType[] execTypes = { cluster.getExecType(), ExecType.LOCAL};
         PigServer pig = null;
         for(ExecType execType : execTypes){
+            Util.resetStateForExecModeSwitch();
             System.err.println("Starting test mode " + execType);
-            if(execType == ExecType.MAPREDUCE) {
-                pig = new PigServer(ExecType.MAPREDUCE, 
+            if (execType == cluster.getExecType()) {
+                pig = new PigServer(cluster.getExecType(),
                         cluster.getProperties());
-            }else{
-                pig = new PigServer(ExecType.LOCAL);
+            } else {
+                pig = new PigServer(execType);
             }
             final String outFile = "TestStoreInst1";
             Util.deleteFile(pig.getPigContext(), outFile);
             pig.setBatchOn();
             String query =
                 "  l1 = load '" + INP_FILE_2NUMS + "' as (i : int, j : int);" +
-                " store l1 into '" + outFile + "' using " + CHECK_INSTANCE_STORE_FUNC + 
+                " store l1 into '" + outFile + "' using " + CHECK_INSTANCE_STORE_FUNC +
                 ";";
             Util.registerMultiLineQuery(pig, query);
             List<ExecJob> execJobs = pig.executeBatch();
             assertEquals("num jobs", 1, execJobs.size());
             assertEquals("status ", JOB_STATUS.COMPLETED, execJobs.get(0).getStatus());
         }
-            
+
     }
 
 
@@ -160,7 +160,7 @@ public class TestStoreInstances  {
                 throws IOException {
             Configuration conf = job.getConfiguration();
             conf.set("mapred.output.dir", location);
-            
+
         }
 
 
@@ -200,11 +200,11 @@ public class TestStoreInstances  {
             this.outRows = outRows;
         }
 
-   
+
         @Override
         public void commitTask(TaskAttemptContext arg0) {
             System.err.println("OutputCommitterTestInstances commitTask called");
-            assertTrue("Number of output rows > 0 ",outRows.size() > 0);
+            assertEquals("Number of output rows should be 3", 3, outRows.size());
         }
 
         @Override

Modified: pig/branches/tez/test/org/apache/pig/test/TestToolsPigServer.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestToolsPigServer.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestToolsPigServer.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestToolsPigServer.java Fri May 16 16:46:56 2014
@@ -27,7 +27,6 @@ import java.io.File;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.backend.executionengine.ExecJob;
 import org.apache.pig.backend.executionengine.ExecJob.JOB_STATUS;
 import org.apache.pig.builtin.PigStorage;
@@ -40,12 +39,12 @@ import org.junit.Test;
 
 public class TestToolsPigServer {
     private ToolsPigServer pig = null;
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private File stdOutRedirectedFile;
 
     @Before
     public void setUp() throws Exception{
-        pig = new ToolsPigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        pig = new ToolsPigServer(cluster.getExecType(), cluster.getProperties());
         stdOutRedirectedFile = new File("stdout.redirected");
         // Create file if it does not exist
         if(!stdOutRedirectedFile.createNewFile())

Modified: pig/branches/tez/test/org/apache/pig/test/Util.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/Util.java?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/Util.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/Util.java Fri May 16 16:46:56 2014
@@ -614,7 +614,7 @@ public class Util {
             localFileName = localFileName.replace('\\','/');
             fileNameOnCluster = fileNameOnCluster.replace('\\','/');
         }
-        PigServer ps = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        PigServer ps = new PigServer(cluster.getExecType(), cluster.getProperties());
         String script = getMkDirCommandForHadoop2_0(fileNameOnCluster) + "fs -put " + localFileName + " " + fileNameOnCluster;
         GruntParser parser = new GruntParser(new StringReader(script), ps);
         parser.setInteractive(false);
@@ -1318,4 +1318,9 @@ public class Util {
         // null to force ScriptState gets initialized every time.
         ScriptState.start(null);
     }
+
+    public static boolean isMapredExecType(ExecType execType) {
+        return execType == ExecType.MAPREDUCE;
+    }
+
 }

Modified: pig/branches/tez/test/tez-tests
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/tez-tests?rev=1595262&r1=1595261&r2=1595262&view=diff
==============================================================================
--- pig/branches/tez/test/tez-tests (original)
+++ pig/branches/tez/test/tez-tests Fri May 16 16:46:56 2014
@@ -12,22 +12,35 @@
 **/TestFilterUDF.java
 **/TestFinish.java
 **/TestForEachNestedPlan.java
+**/TestFRJoin.java
+**/TestFRJoinNullValue.java
+**/TestGrunt.java
+**/TestImplicitSplit.java
+**/TestInputOutputMiniClusterFileValidator.java
+**/TestJoin.java
+**/TestJoinSmoke.java
+**/TestKeyTypeDiscoveryVisitor.java
 **/TestLoad.java
 **/TestLocalRearrange.java
 **/TestMapReduce.java
 **/TestMapReduce2.java
 **/TestNestedForeach.java
+**/TestNewPlanImplicitSplit.java
 **/TestPigContext.java
 **/TestPigServer.java
+**/TestPigServerWithMacros.java
 **/TestPigStorage.java
+**/TestRegisteredJarVisibility.java
 **/TestSample.java
 **/TestSchema.java
 **/TestScriptLanguageJavaScript.java
 **/TestScriptUDF.java
 **/TestSkewedJoin.java
 **/TestSplitStore.java
+**/TestStoreInstances.java
 **/TestStoreOld.java
 **/TestStreaming.java
+**/TestToolsPigServer.java
 **/TestUDF.java
 **/TestUDFContext.java
 **/TestSecondarySortTez.java