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/08 22:50:06 UTC

svn commit: r1593413 [1/2] - in /pig/branches/tez: src/org/apache/pig/backend/hadoop/executionengine/tez/ test/ test/org/apache/pig/test/

Author: rohini
Date: Thu May  8 20:50:06 2014
New Revision: 1593413

URL: http://svn.apache.org/r1593413
Log:
PIG-3924: Port minicluster unit tests for Tez mode (rohini)

Modified:
    pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/tez/PigProcessor.java
    pig/branches/tez/test/org/apache/pig/test/MiniGenericCluster.java
    pig/branches/tez/test/org/apache/pig/test/TestBinaryExpressionOps.java
    pig/branches/tez/test/org/apache/pig/test/TestCombiner.java
    pig/branches/tez/test/org/apache/pig/test/TestFilterUDF.java
    pig/branches/tez/test/org/apache/pig/test/TestFinish.java
    pig/branches/tez/test/org/apache/pig/test/TestForEachNestedPlan.java
    pig/branches/tez/test/org/apache/pig/test/TestLoad.java
    pig/branches/tez/test/org/apache/pig/test/TestLocalRearrange.java
    pig/branches/tez/test/org/apache/pig/test/TestMapReduce.java
    pig/branches/tez/test/org/apache/pig/test/TestMapReduce2.java
    pig/branches/tez/test/org/apache/pig/test/TestPigServer.java
    pig/branches/tez/test/org/apache/pig/test/TestPigStorage.java
    pig/branches/tez/test/org/apache/pig/test/TestSample.java
    pig/branches/tez/test/org/apache/pig/test/TestSchema.java
    pig/branches/tez/test/org/apache/pig/test/TestScriptLanguageJavaScript.java
    pig/branches/tez/test/org/apache/pig/test/TestScriptUDF.java
    pig/branches/tez/test/org/apache/pig/test/TestStoreOld.java
    pig/branches/tez/test/org/apache/pig/test/TestStreaming.java
    pig/branches/tez/test/org/apache/pig/test/TestUDF.java
    pig/branches/tez/test/org/apache/pig/test/TestUDFContext.java
    pig/branches/tez/test/org/apache/pig/test/Util.java
    pig/branches/tez/test/tez-tests

Modified: pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/tez/PigProcessor.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/tez/PigProcessor.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/tez/PigProcessor.java (original)
+++ pig/branches/tez/src/org/apache/pig/backend/hadoop/executionengine/tez/PigProcessor.java Thu May  8 20:50:06 2014
@@ -32,6 +32,7 @@ import org.apache.pig.PigException;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigHadoopLogger;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.UDFFinishVisitor;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PigProgressable;
@@ -43,6 +44,8 @@ import org.apache.pig.backend.hadoop.exe
 import org.apache.pig.data.SchemaTupleBackend;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.plan.DependencyOrderWalker;
+import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.util.ObjectSerializer;
 import org.apache.pig.impl.util.UDFContext;
 import org.apache.pig.tools.pigstats.PigStatusReporter;
@@ -163,6 +166,18 @@ public class PigProcessor implements Log
 
             runPipeline(leaf);
 
+            // Calling EvalFunc.finish()
+            UDFFinishVisitor finisher = new UDFFinishVisitor(execPlan,
+                    new DependencyOrderWalker<PhysicalOperator, PhysicalPlan>(
+                            execPlan));
+            try {
+                finisher.visit();
+            } catch (VisitorException e) {
+                int errCode = 2121;
+                String msg = "Error while calling finish method on UDFs.";
+                throw new VisitorException(msg, errCode, PigException.BUG, e);
+            }
+
             for (MROutput fileOutput : fileOutputs){
                 fileOutput.commit();
             }

Modified: pig/branches/tez/test/org/apache/pig/test/MiniGenericCluster.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/MiniGenericCluster.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/MiniGenericCluster.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/MiniGenericCluster.java Thu May  8 20:50:06 2014
@@ -41,6 +41,7 @@ abstract public class MiniGenericCluster
     protected MiniDFSCluster m_dfs = null;
     protected FileSystem m_fileSys = null;
     protected Configuration m_conf = null;
+    protected Properties m_props = null;
 
     protected static MiniGenericCluster INSTANCE = null;
     protected static boolean isSetup = false;
@@ -100,6 +101,8 @@ abstract public class MiniGenericCluster
         isSetup = false;
         shutdownMiniDfsClusters();
         shutdownMiniMrClusters();
+        m_conf = null;
+        m_props = null;
     }
 
     protected void shutdownMiniDfsClusters() {
@@ -117,7 +120,10 @@ abstract public class MiniGenericCluster
 
     public Properties getProperties() {
         errorIfNotSetup();
-        return ConfigurationUtil.toProperties(m_conf);
+        if (m_props == null) {
+            m_props = ConfigurationUtil.toProperties(m_conf);
+        }
+        return m_props;
     }
 
     public Configuration getConfiguration() {

Modified: pig/branches/tez/test/org/apache/pig/test/TestBinaryExpressionOps.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestBinaryExpressionOps.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestBinaryExpressionOps.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestBinaryExpressionOps.java Thu May  8 20:50:06 2014
@@ -19,9 +19,8 @@
 package org.apache.pig.test;
 
 
-import static org.junit.Assert.assertEquals;
-
 import java.util.Iterator;
+import java.util.List;
 import java.util.Properties;
 
 import org.apache.pig.PigServer;
@@ -71,13 +70,12 @@ public class TestBinaryExpressionOps {
                 + "(SUM(A.val) * SUM(B.val)), (SUM(A.val) / SUM(B.val)), "
                 + "(SUM(A.val) % SUM(B.val)), (SUM(A.val) < 0 ? SUM(A.val) : SUM(B.val));");
 
-        String[] expectedResults = new String[] {"(id1,2,,,,,,,)", "(id2,2,10,8,12,20,5,0,2)"};
+        List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
+                new String[] {
+                        "('id1',2L,null,null,null,null,null,null,null)",
+                        "('id2',2L,10L,8L,12L,20L,5L,0L,2L)" });
         Iterator<Tuple> iter = pig.openIterator("D");
-        int counter = 0;
-        while (iter.hasNext()) { 
-            assertEquals(expectedResults[counter++], iter.next().toString());
-        }
-        assertEquals(expectedResults.length, counter);
+        Util.checkQueryOutputsAfterSort(iter, expectedResults);
     }
 
 }

Modified: pig/branches/tez/test/org/apache/pig/test/TestCombiner.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestCombiner.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestCombiner.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestCombiner.java Thu May  8 20:50:06 2014
@@ -38,8 +38,6 @@ import org.apache.pig.data.DefaultDataBa
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileLocalizer;
-import org.apache.pig.tools.pigstats.ScriptState;
-import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -62,21 +60,7 @@ public class TestCombiner {
 
     @Before
     public void setUp() throws Exception {
-        // cause a re initialization of FileLocalizer's
-        // internal state before each test run
-        // A previous test might have been in a different
-        // mode than the test which is about to run. To
-        // ensure each test runs correctly in it's exectype
-        // mode, let's re initialize.
-        FileLocalizer.setInitialized(false);
-    }
-
-    @After
-    public void tearDown() throws Exception{
-        // TODO: once we have Tez local mode, we can get rid of this. For now,
-        // if we run this test suite in Tez mode, we need to set ScriptState to
-        // null to force ScriptState gets initialized every time.
-        ScriptState.start(null);
+        Util.resetStateForExecModeSwitch();
     }
 
     @Test

Modified: pig/branches/tez/test/org/apache/pig/test/TestFilterUDF.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestFilterUDF.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestFilterUDF.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestFilterUDF.java Thu May  8 20:50:06 2014
@@ -17,7 +17,6 @@ package org.apache.pig.test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -27,7 +26,6 @@ import java.io.PrintWriter;
 import java.util.Iterator;
 
 import org.apache.pig.EvalFunc;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.Tuple;
@@ -40,7 +38,7 @@ import org.junit.Test;
 
 public class TestFilterUDF {
     private PigServer pigServer;
-    private static MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private File tmpFile;
 
     TupleFactory tf = TupleFactory.getInstance();
@@ -57,7 +55,7 @@ public class TestFilterUDF {
 
     @Before
     public void setUp() throws Exception {
-        pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
         int LOOP_SIZE = 20;
         tmpFile = File.createTempFile("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));

Modified: pig/branches/tez/test/org/apache/pig/test/TestFinish.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestFinish.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestFinish.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestFinish.java Thu May  8 20:50:06 2014
@@ -34,7 +34,6 @@ import org.apache.pig.builtin.PigStorage
 import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.data.TupleFactory;
-import org.apache.pig.impl.io.FileLocalizer;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
@@ -46,7 +45,7 @@ public class TestFinish {
     BagFactory mBf = BagFactory.getInstance();
     File f1;
 
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
 
     static public class MyEvalFunction extends EvalFunc<Tuple> {
         String execType;
@@ -78,10 +77,8 @@ public class TestFinish {
 
     @Before
     public void setUp() throws Exception {
-        // re initialize FileLocalizer so that each test runs correctly without
-        // any side effect of other tests - this is needed here since some
-        // tests are in mapred and some in local mode
-        FileLocalizer.setInitialized(false);
+        // Reset state since some tests are in mapred and some in local mode
+        Util.resetStateForExecModeSwitch();
     }
 
     @AfterClass
@@ -102,7 +99,7 @@ public class TestFinish {
             }
             ps.close();
         } else {
-            pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+            pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
             f1 = File.createTempFile("test", "txt");
             f1.delete();
             inputFileName = Util.removeColon(f1.getAbsolutePath());
@@ -118,7 +115,7 @@ public class TestFinish {
 
     private void checkAndCleanup(ExecType execType, String expectedFileName,
             String inputFileName) throws IOException {
-        if (execType == ExecType.MAPREDUCE) {
+        if (execType == cluster.getExecType()) {
             FileSystem fs = FileSystem.get(ConfigurationUtil.toConfiguration(
                     cluster.getProperties()));
             assertTrue(fs.exists(new Path(expectedFileName)));
@@ -136,7 +133,7 @@ public class TestFinish {
 
     @Test
     public void testFinishInMapMR() throws Exception {
-        String inputFileName = setUp(ExecType.MAPREDUCE);
+        String inputFileName = setUp(cluster.getExecType());
         // this file will be created on the cluster if finish() is called
         String expectedFileName = "testFinishInMapMR-finish.txt";
         pigServer.registerQuery("define MYUDF " + MyEvalFunction.class.getName() + "('MAPREDUCE','"
@@ -149,13 +146,13 @@ public class TestFinish {
             iter.next();
         }
 
-        checkAndCleanup(ExecType.MAPREDUCE, expectedFileName, inputFileName);
+        checkAndCleanup(cluster.getExecType(), expectedFileName, inputFileName);
 
     }
 
     @Test
     public void testFinishInReduceMR() throws Exception {
-        String inputFileName = setUp(ExecType.MAPREDUCE);
+        String inputFileName = setUp(cluster.getExecType());
         // this file will be created on the cluster if finish() is called
         String expectedFileName = "testFinishInReduceMR-finish.txt";
         pigServer.registerQuery("define MYUDF " + MyEvalFunction.class.getName() + "('MAPREDUCE','"
@@ -169,7 +166,7 @@ public class TestFinish {
             iter.next();
         }
 
-        checkAndCleanup(ExecType.MAPREDUCE, expectedFileName, inputFileName);
+        checkAndCleanup(cluster.getExecType(), expectedFileName, inputFileName);
     }
 
     @Test

Modified: pig/branches/tez/test/org/apache/pig/test/TestForEachNestedPlan.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestForEachNestedPlan.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestForEachNestedPlan.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestForEachNestedPlan.java Thu May  8 20:50:06 2014
@@ -18,7 +18,7 @@
 
 package org.apache.pig.test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 import java.io.File;
 import java.io.FileWriter;
@@ -29,9 +29,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 
-import junit.framework.Assert;
-
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.DataBag;
@@ -39,32 +36,33 @@ import org.apache.pig.data.Tuple;
 import org.apache.pig.parser.ParserException;
 import org.apache.pig.test.utils.TestHelper;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Test;
 
 public class TestForEachNestedPlan {
 
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
 
     private PigServer pig ;
 
     public TestForEachNestedPlan() throws Throwable {
-        pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()) ;
+        pig = new PigServer(cluster.getExecType(), cluster.getProperties()) ;
     }
 
     Boolean[] nullFlags = new Boolean[]{ false, true };
-    
+
     @AfterClass
     public static void oneTimeTearDown() throws Exception {
         cluster.shutDown();
     }
-    
+
     @Test
     public void testInnerOrderBy() throws Exception {
         for (int i = 0; i < nullFlags.length; i++) {
             System.err.println("Running testInnerOrderBy with nullFlags set to :"
                             + nullFlags[i]);
             File tmpFile = genDataSetFile1(nullFlags[i]);
-            pig.registerQuery("a = load '" 
+            pig.registerQuery("a = load '"
                     + Util.generateURI(tmpFile.toString(), pig.getPigContext()) + "'; ");
             pig.registerQuery("b = group a by $0; ");
             pig.registerQuery("c = foreach b { " + "     c1 = order $1 by *; "
@@ -82,9 +80,9 @@ public class TestForEachNestedPlan {
     }
 
     @Test
-    public void testInnerOrderByStarWithSchema() throws Exception {        
+    public void testInnerOrderByStarWithSchema() throws Exception {
         File tmpFile = genDataSetFile1(false);
-        pig.registerQuery("a = load '" + Util.generateURI(tmpFile.toString(), 
+        pig.registerQuery("a = load '" + Util.generateURI(tmpFile.toString(),
                 pig.getPigContext()) + "' as (a0, a1);");
         pig.registerQuery("b = group a by a0; ");
         pig.registerQuery("c = foreach b { d = order a by *; "
@@ -99,23 +97,23 @@ public class TestForEachNestedPlan {
         }
         Assert.assertEquals(count, 10);
     }
-   
+
     @Test
-    public void testMultiColInAlias() throws Exception {    
+    public void testMultiColInAlias() throws Exception {
     	pig.getPigContext().getProperties().setProperty("pig.exec.nosecondarykey", "true");
     	String INPUT_FILE = "test-multi-alias.txt";
         PrintWriter w = new PrintWriter(new FileWriter(INPUT_FILE));
         w.println("10\tnrai01\t01");
         w.println("20\tnrai02\t02");
         w.close();
-        
+
         try {
-          
+
             Util.copyFromLocalToCluster(cluster, INPUT_FILE, INPUT_FILE);
             pig.registerQuery("A = load '" + INPUT_FILE + "' "
                     + "as (a:int, b:chararray, c:int);");
             pig.registerQuery("B = GROUP A BY (a, b);") ;
-           
+
             DataBag dbfrj = BagFactory.getInstance().newDefaultBag(), dbshj = BagFactory.getInstance().newDefaultBag();
             {
                 pig.registerQuery("C = FOREACH B { bg = A.($1,$2); GENERATE group, bg; } ;") ;
@@ -132,7 +130,7 @@ public class TestForEachNestedPlan {
                 }
             }
             Assert.assertEquals(dbfrj.size(), dbshj.size());
-            Assert.assertEquals(true, TestHelper.compareBags(dbfrj, dbshj)); 
+            Assert.assertEquals(true, TestHelper.compareBags(dbfrj, dbshj));
 
         } finally{
             new File(INPUT_FILE).delete();
@@ -144,9 +142,9 @@ public class TestForEachNestedPlan {
             }
         }
     }
-   
+
     @Test
-    public void testAlgebricFuncWithoutGroupBy() 
+    public void testAlgebricFuncWithoutGroupBy()
     throws IOException, ParserException {
         String INPUT_FILE = "test-sum.txt";
 
@@ -194,7 +192,7 @@ public class TestForEachNestedPlan {
     }
 
     @Test
-    public void testInnerDistinct() 
+    public void testInnerDistinct()
     throws IOException, ParserException {
         String INPUT_FILE = "test-distinct.txt";
 
@@ -208,13 +206,13 @@ public class TestForEachNestedPlan {
 
         try {
             Util.copyFromLocalToCluster(cluster, INPUT_FILE, INPUT_FILE);
-        
+
             pig.registerQuery("A = load '" + INPUT_FILE
                     + "' as (age:int, gpa:int);");
             pig.registerQuery("B = group A by age;");
             pig.registerQuery("C = foreach B { D = A.gpa; E = distinct D; " +
             		"generate group, MIN(E); };");
-    
+
             Iterator<Tuple> iter = pig.openIterator("C");
 
             List<Tuple> expectedResults =
@@ -224,9 +222,9 @@ public class TestForEachNestedPlan {
             int counter = 0;
             while (iter.hasNext()) {
                assertEquals(expectedResults.get(counter++).toString(),
-                        iter.next().toString());                
+                        iter.next().toString());
             }
-    
+
             assertEquals(expectedResults.size(), counter);
         } finally{
             new File(INPUT_FILE).delete();
@@ -240,7 +238,7 @@ public class TestForEachNestedPlan {
     }
 
     @Test
-    public void testInnerOrderByAliasReuse() 
+    public void testInnerOrderByAliasReuse()
     throws IOException, ParserException {
         String INPUT_FILE = "test-innerorderbyaliasreuse.txt";
 
@@ -253,13 +251,13 @@ public class TestForEachNestedPlan {
 
         try {
             Util.copyFromLocalToCluster(cluster, INPUT_FILE, INPUT_FILE);
-        
+
             pig.registerQuery("A = load '" + INPUT_FILE
                     + "' as (v1:int, v2:int);");
             pig.registerQuery("B = group A by v1;");
             pig.registerQuery("C = foreach B { X = A; X = order X by v2 asc; " +
             		"generate flatten(X); };");
-    
+
             Iterator<Tuple> iter = pig.openIterator("C");
 
             List<Tuple> expectedResults =
@@ -269,9 +267,9 @@ public class TestForEachNestedPlan {
             int counter = 0;
             while (iter.hasNext()) {
                 assertEquals(expectedResults.get(counter++).toString(),
-                        iter.next().toString());                
+                        iter.next().toString());
             }
-    
+
             assertEquals(expectedResults.size(), counter);
         } finally{
             new File(INPUT_FILE).delete();
@@ -283,8 +281,8 @@ public class TestForEachNestedPlan {
             }
         }
     }
-    
-    
+
+
     /***
      * For generating a sample dataset
      */
@@ -296,7 +294,7 @@ public class TestForEachNestedPlan {
         DecimalFormat formatter = new DecimalFormat("0000000");
 
         Random r = new Random();
-        
+
         for (int i = 0; i < dataLength; i++) {
             data[i] = new String[2] ;
             // inject nulls randomly

Modified: pig/branches/tez/test/org/apache/pig/test/TestLoad.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestLoad.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestLoad.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestLoad.java Thu May  8 20:50:06 2014
@@ -66,13 +66,13 @@ public class TestLoad {
     PigContext pc;
     PigServer[] servers;
 
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
 
     @Before
     public void setUp() throws Exception {
         FileLocalizer.deleteTempFiles();
         servers = new PigServer[] {
-                    new PigServer(ExecType.MAPREDUCE, cluster.getProperties()),
+                    new PigServer(cluster.getExecType(), cluster.getProperties()),
                     new PigServer(ExecType.LOCAL, new Properties())
         };
     }
@@ -327,7 +327,7 @@ public class TestLoad {
             if(noConversionExpected) {
                 assertEquals(expected, p);
             } else  {
-                String protocol = pc.getExecType() == ExecType.MAPREDUCE ? "hdfs" : "file";
+                String protocol = pc.getExecType() == cluster.getExecType() ? "hdfs" : "file";
                 // regex : A word character, i.e. [a-zA-Z_0-9] or '-' followed by ':' then any characters
                 String regex = "[\\-\\w:\\.]";
                 assertTrue(p.matches(".*" + protocol + "://" + regex + "*.*"));

Modified: pig/branches/tez/test/org/apache/pig/test/TestLocalRearrange.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestLocalRearrange.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestLocalRearrange.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestLocalRearrange.java Thu May  8 20:50:06 2014
@@ -17,7 +17,8 @@
  */
 package org.apache.pig.test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.io.FileWriter;
@@ -28,9 +29,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 
-import junit.framework.Assert;
-
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
@@ -50,33 +48,34 @@ import org.apache.pig.test.utils.GenRand
 import org.apache.pig.test.utils.TestHelper;
 import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 /**
  * Tests localrearrange db for
- * group db by $0 
+ * group db by $0
  *
  */
 public class TestLocalRearrange  {
-    
+
     POLocalRearrange lr;
     Tuple t;
     DataBag db;
-    private static final MiniCluster cluster = MiniCluster.buildCluster();
+    private static final MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
+
 
-    
     @Before
     public void setUp() throws Exception {
         Random r = new Random();
         db = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
     }
-    
+
     @AfterClass
     public static void oneTimeTearDown() throws Exception {
         cluster.shutDown();
     }
-    
+
     private void setUp1() throws PlanException, ExecException{
         lr = GenPhyOp.topLocalRearrangeOPWithPlanPlain(0,0,db.iterator().next());
         POProject proj = GenPhyOp.exprProject();
@@ -106,32 +105,32 @@ public class TestLocalRearrange  {
             // The input data has 2 columns of which the first
             // is the key
             // With the optimized LocalRearrange, the part
-            // of the "value" present in the "key" is 
+            // of the "value" present in the "key" is
             // excluded from the "value". So to reconstruct
             // the true "value", create a tuple with "key" in
             // first position and the "value" (val) we currently
             // have in the second position
             assertEquals(1, val.size());
-            
+
             Tuple actualVal = new DefaultTuple();
             actualVal.append(key);
             actualVal.append(val.get(0));
             //Check if the index is same as input index
             assertEquals((byte)0, (byte)(Byte)t.get(0));
-            
+
             //Check if the input bag contains the value tuple
             assertTrue(TestHelper.bagContains(db, actualVal));
-            
+
             //Check if the input key and the output key are same
             String inpKey = (String)actualVal.get(0);
             assertEquals(0, inpKey.compareTo((String)t.get(1)));
             ++size;
         }
-        
+
         //check if all the tuples in the input are generated
         assertEquals(db.size(), size);
     }
-    
+
     private void setUp2() throws PlanException, ExecException{
         lr = GenPhyOp.topLocalRearrangeOPWithPlanPlain(0,0,db.iterator().next());
         List<PhysicalPlan> plans = lr.getPlans();
@@ -139,7 +138,7 @@ public class TestLocalRearrange  {
         List<PhysicalPlan> plansT = lrT.getPlans();
         plans.add(plansT.get(0));
         lr.setPlans(plans);
-        
+
         POProject proj = GenPhyOp.exprProject();
         proj.setColumn(0);
         proj.setResultType(DataType.TUPLE);
@@ -151,7 +150,7 @@ public class TestLocalRearrange  {
         inputs.add(proj);
         lr.setInputs(inputs);
     }
-    
+
     @Test
     public void testGetNextTuple2() throws ExecException, PlanException {
         setUp2();
@@ -160,32 +159,32 @@ public class TestLocalRearrange  {
             Tuple t = (Tuple)res.result;
             Tuple key = (Tuple)t.get(1);
             Tuple val = (Tuple)t.get(2);
-            
+
             // The input data has 2 columns of which both
             // are the key.
             // With the optimized LocalRearrange, the part
-            // of the "value" present in the "key" is 
-            // excluded from the "value". So in this case, 
+            // of the "value" present in the "key" is
+            // excluded from the "value". So in this case,
             // the "value" coming out of the LocalRearrange
             // would be an empty tuple
             assertEquals(0, val.size());
-            
+
             //Check if the index is same as input index
             assertEquals((byte)0, (byte)(Byte)t.get(0));
-            
+
             // reconstruct value from tuple
             val = key;
             //Check if the input baf contains the value tuple
             assertTrue(TestHelper.bagContains(db, val));
-            
+
             //Check if the input key and the output key are same
-            Tuple inpKey = TupleFactory.getInstance().newTuple(2); 
+            Tuple inpKey = TupleFactory.getInstance().newTuple(2);
             inpKey.set(0, val.get(0));
             inpKey.set(1, val.get(1));
-            assertEquals(0, inpKey.compareTo((Tuple)t.get(1)));
+            assertEquals(inpKey, (Tuple)t.get(1));
             ++size;
         }
-        
+
         //check if all the tuples in the input are generated
         assertEquals(db.size(), size);
     }
@@ -193,11 +192,11 @@ public class TestLocalRearrange  {
     @Test
     public void testMultiQueryJiraPig1194() {
 
-        // test case: POLocalRearrange doesn't handle nulls returned by POBinCond 
-        
+        // test case: POLocalRearrange doesn't handle nulls returned by POBinCond
+
         String INPUT_FILE = "data.txt";
-        
-        
+
+
         try {
             PrintWriter w = new PrintWriter(new FileWriter(INPUT_FILE));
             w.println("10\t2\t3");
@@ -210,25 +209,21 @@ public class TestLocalRearrange  {
             w.close();
             Util.copyFromLocalToCluster(cluster, INPUT_FILE, INPUT_FILE);
 
-            PigServer myPig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+            PigServer myPig = new PigServer(cluster.getExecType(), cluster.getProperties());
 
             myPig.registerQuery("data = load '" + INPUT_FILE + "' as (a0, a1, a2);");
             myPig.registerQuery("grp = GROUP data BY (((double) a2)/((double) a1) > .001 OR a0 < 11 ? a0 : 0);");
             myPig.registerQuery("res = FOREACH grp GENERATE group, SUM(data.a1), SUM(data.a2);");
-            
+
             List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
-                    new String[] {   
+                    new String[] {
                             "(0,7000.0,5.0)",
-                            "(10,6.0,8.0)",                            
+                            "(10,6.0,8.0)",
                             "(null,12.0,null)"
                     });
-            
+
             Iterator<Tuple> iter = myPig.openIterator("res");
-            int counter = 0;
-            while (iter.hasNext()) {
-                assertEquals(expectedResults.get(counter++).toString(), iter.next().toString());      
-            }
-            assertEquals(expectedResults.size(), counter);
+            Util.checkQueryOutputsAfterSort(iter, expectedResults);
 
         } catch (Exception e) {
             e.printStackTrace();
@@ -243,5 +238,5 @@ public class TestLocalRearrange  {
             }
         }
     }
-    
+
 }

Modified: pig/branches/tez/test/org/apache/pig/test/TestMapReduce.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestMapReduce.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestMapReduce.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestMapReduce.java Thu May  8 20:50:06 2014
@@ -29,13 +29,12 @@ import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.PrintStream;
 import java.util.Iterator;
-import java.util.Properties;
 import java.util.Map.Entry;
+import java.util.Properties;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.pig.EvalFunc;
-import org.apache.pig.ExecType;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.datastorage.ElementDescriptor;
@@ -56,21 +55,21 @@ import org.junit.Test;
 public class TestMapReduce {
 
     private Log log = LogFactory.getLog(getClass());
-    
-    static MiniCluster cluster = MiniCluster.buildCluster();
+
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
 
     private PigServer pig;
-    
+
     @Before
     public void setUp() throws Exception {
-        pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        pig = new PigServer(cluster.getExecType(), cluster.getProperties());
     }
-    
+
     @AfterClass
     public static void oneTimeTearDown() throws Exception {
         cluster.shutDown();
     }
-    
+
 
     @Test
     public void testBigGroupAll() throws Throwable {
@@ -112,19 +111,19 @@ public class TestMapReduce {
 
     /**
      * This test checks records that align perfectly on
-     * bzip block boundaries and hdfs block boundaries 
+     * bzip block boundaries and hdfs block boundaries
      */
     @Test
     public void testBZip2Aligned() throws Throwable {
         int offsets[] = { 219642, 219643, 219644, 552019, 552020 };
         for(int i = 1; i < offsets.length; i ++) {
-            
+
             Properties props = new Properties();
             for (Entry<Object, Object> entry : cluster.getProperties().entrySet()) {
                 props.put(entry.getKey(), entry.getValue());
             }
             props.setProperty("mapred.max.split.size", Integer.toString(offsets[i]));
-            PigContext pigContext = new PigContext(ExecType.MAPREDUCE, props);
+            PigContext pigContext = new PigContext(cluster.getExecType(), props);
             PigServer pig = new PigServer(pigContext);
             pig.registerQuery("a = load '"
                     + Util.generateURI(
@@ -143,7 +142,7 @@ public class TestMapReduce {
             //assertEquals("1000000", it.next().getField(0));
         }
     }
-    
+
     public Double bigGroupAll( File tmpFile ) throws Throwable {
 
         String query = "foreach (group (load '"
@@ -157,7 +156,7 @@ public class TestMapReduce {
         return  DataType.toDouble(t.get(0));
     }
 
-    
+
     static public class MyApply extends EvalFunc<DataBag> {
         String field0 = "Got";
         public MyApply() {}
@@ -210,14 +209,14 @@ public class TestMapReduce {
         public void setNulls(boolean hasNulls ) { this.hasNulls=hasNulls; }
 
         /**
-         * 
+         *
          */
         public MyStorage() {
             // initialize delimiter to be "-" for output
             // since that is the delimiter in the tests below
             super("-");
         }
-        
+
         @Override
         public Tuple getNext() throws IOException {
             if (count < COUNT) {
@@ -289,7 +288,7 @@ public class TestMapReduce {
         String[][] data = genDataSetFile1( 10, true );
         storeFunction( data);
     }
-   
+
     public void storeFunction(String[][] data) throws Throwable {
 
         File tmpFile=TestHelper.createTempFile(data) ;
@@ -362,7 +361,7 @@ public class TestMapReduce {
         assertEquals( MyStorage.COUNT, count );
         tmpFile.delete();
     }
-    
+
     @Test
     public void testQualifiedFunctionsWithNulls() throws Throwable {
 
@@ -401,7 +400,7 @@ public class TestMapReduce {
         assertEquals( MyStorage.COUNT, count );
         tmpFile.delete();
     }
-    
+
 
     @Test
     public void testDefinedFunctions() throws Throwable {
@@ -489,7 +488,7 @@ public class TestMapReduce {
     @Test
     public void testPigServer() throws Throwable {
         log.debug("creating pig server");
-        PigContext pigContext = new PigContext(ExecType.MAPREDUCE, cluster.getProperties());
+        PigContext pigContext = new PigContext(cluster.getExecType(), cluster.getProperties());
         PigServer pig = new PigServer(pigContext);
         System.out.println("testing capacity");
         long capacity = pig.capacity();
@@ -523,11 +522,11 @@ public class TestMapReduce {
      *           3  3
      *           4  4
      *           5  5
-     *           6   
+     *           6
      *           7  7
-     *               
+     *
      *           9  9
-     *           
+     *
      */
     private String[][] genDataSetFile1( int dataLength, boolean hasNulls ) throws IOException {
 
@@ -544,7 +543,7 @@ public class TestMapReduce {
             		data[i][1] = new Integer(i).toString();
 
 		     } else if ( i == 6 ) {
-                   
+
             		data[i][0] = new Integer(i).toString();
             		data[i][1] = "";
 

Modified: pig/branches/tez/test/org/apache/pig/test/TestMapReduce2.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestMapReduce2.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestMapReduce2.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestMapReduce2.java Thu May  8 20:50:06 2014
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.text.DecimalFormat;
 import java.util.Iterator;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.test.utils.TestHelper;
@@ -34,12 +33,12 @@ import org.junit.Test;
 
 public class TestMapReduce2 {
 
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
 
     private PigServer pig ;
 
     public TestMapReduce2() throws Throwable {
-        pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()) ;
+        pig = new PigServer(cluster.getExecType(), cluster.getProperties()) ;
     }
 
     @AfterClass

Modified: pig/branches/tez/test/org/apache/pig/test/TestPigServer.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestPigServer.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestPigServer.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestPigServer.java Thu May  8 20:50:06 2014
@@ -72,8 +72,6 @@ import org.apache.pig.impl.util.Properti
 import org.apache.pig.impl.util.Utils;
 import org.apache.pig.tools.grunt.Grunt;
 import org.apache.pig.tools.grunt.GruntParser;
-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;
@@ -93,7 +91,7 @@ public class TestPigServer {
 
     @Before
     public void setUp() throws Exception{
-        FileLocalizer.setInitialized(false);
+        Util.resetStateForExecModeSwitch();
         tempDir = Files.createTempDir();
         tempDir.deleteOnExit();
         registerNewResource(tempDir.getAbsolutePath());
@@ -102,10 +100,6 @@ public class TestPigServer {
     @After
     public void tearDown() throws Exception{
         tempDir.delete();
-        // TODO: once we have Tez local mode, we can get rid of this. For now,
-        // if we run this test suite in Tez mode, we need to set ScriptState to
-        // null to force ScriptState gets initialized every time.
-        ScriptState.start(null);
     }
 
     @BeforeClass

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=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestPigStorage.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestPigStorage.java Thu May  8 20:50:06 2014
@@ -19,7 +19,11 @@
 package org.apache.pig.test;
 
 import static org.apache.pig.builtin.mock.Storage.tuple;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.FileWriter;
@@ -48,8 +52,6 @@ import org.apache.pig.impl.logicalLayer.
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.util.Utils;
 import org.apache.pig.test.utils.TypeCheckingTestUtil;
-import org.apache.pig.tools.pigstats.PigStats;
-import org.apache.pig.tools.pigstats.ScriptState;
 import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.map.JsonMappingException;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -70,9 +72,9 @@ public class TestPigStorage  {
     @Before
     public void setup() throws IOException {
         // some tests are in map-reduce mode and some in local - so before
-        // each test, we will de-initialize FileLocalizer so that temp files
+        // each test, we will re-initialize FileLocalizer so that temp files
         // are created correctly depending on the ExecType in the test.
-        FileLocalizer.setInitialized(false);
+        Util.resetStateForExecModeSwitch();
 
         // If needed, a test can change that. Most tests are local so we save a bit
         // of typing here.
@@ -268,12 +270,12 @@ public class TestPigStorage  {
 
         // Verify that loaded data has the correct data type after the prune
         pig.registerQuery("b = LOAD '" + datadir + "aout' using PigStorage('\\t'); c = FOREACH b GENERATE f2;");
-        
+
         Iterator<Tuple> it = pig.openIterator("c");
         Assert.assertTrue("results were produced", it.hasNext());
-        
+
         Tuple t = it.next();
-        
+
         Assert.assertTrue("data is correct type", t.get(0) instanceof Integer);
     }
 
@@ -357,7 +359,7 @@ public class TestPigStorage  {
     @Test
     public void testSchemaDataNotMatchWITHCast() throws Exception {
         pig.registerQuery("A = LOAD '" + datadir + "originput' using PigStorage(',') as (x:chararray);");
-        
+
         List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
                 new String[] {
                         "('A')",
@@ -378,7 +380,7 @@ public class TestPigStorage  {
     @Test
     public void testSchemaDataNotMatchNOCast() throws Exception {
         pig.registerQuery("A = LOAD '" + datadir + "originput' using PigStorage(',') as (x:bytearray);");
-        
+
         List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
                 new String[] {
                         "('A')",
@@ -399,7 +401,7 @@ public class TestPigStorage  {
     @Test
     public void testSchemaDataNotMatchAsEXTRACoumns() throws Exception {
         pig.registerQuery("A = LOAD '" + datadir + "originput' using PigStorage(',') as (x,y,z);");
-        
+
         List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
                 new String[] {
                         "('A',1,NULL)",
@@ -507,6 +509,7 @@ public class TestPigStorage  {
         new ObjectMapper().writeValue(new File(schemaFilename), testSchema);
     }
 
+    @SuppressWarnings("deprecation")
     @Test
     public void testPigStorageSchemaSearch() throws Exception {
         String globtestdir = "build/test/tmpglobbingdata/";
@@ -664,7 +667,7 @@ public class TestPigStorage  {
         assertTrue(it.hasNext());
         assertEquals(tuple(1,null,null), it.next());
         assertFalse(it.hasNext());
-        
+
         // Now, test with prune
         pig.registerQuery("a = load '"+Util.encodeEscape(inputDir.getAbsolutePath())+"'; b = foreach a generate y, z;");
         it = pig.openIterator("b");
@@ -713,5 +716,5 @@ public class TestPigStorage  {
         pig.store("a", datadir + "aout", "PigStorage(',')");
         pig.store("a", datadir + "aout", "PigStorage(',')");
     }
-    
+
 }

Modified: pig/branches/tez/test/org/apache/pig/test/TestSample.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestSample.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestSample.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestSample.java Thu May  8 20:50:06 2014
@@ -23,7 +23,6 @@ import java.io.File;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.FrontendException;
@@ -38,13 +37,13 @@ public class TestSample {
     private String tmpfilepath;
 
     private int DATALEN = 1024;
-    static MiniCluster cluster = MiniCluster.buildCluster();
-    
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
+
     @Before
     public void setUp()
     throws Exception
     {
-        pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        pig = new PigServer(cluster.getExecType(), cluster.getProperties());
 
         tmpFile = File.createTempFile( this.getClass().getName(), ".txt");
         tmpFile.delete(); // we don't want the file, just the temp path
@@ -65,7 +64,7 @@ public class TestSample {
     {
         Util.deleteFile(cluster, tmpfilepath);
     }
-    
+
     @AfterClass
     public static void oneTimeTearDown() throws Exception {
         cluster.shutDown();
@@ -109,7 +108,7 @@ public class TestSample {
     {
        verify("myid = sample (load '"+ Util.encodeEscape(tmpfilepath) + "') 0.5;", DATALEN/3, DATALEN*2/3);
     }
-    
+
     @Test
     public void testSample_VariableNone() throws Exception {
         verify("a = LOAD '" + Util.encodeEscape(tmpfilepath) + "'; " +
@@ -117,7 +116,7 @@ public class TestSample {
                 "c = FOREACH b GENERATE COUNT(a) AS count;" +
         		"myid = SAMPLE a (c.count - c.count);", 0, 0);
 }
-    
+
     @Test
     public void testSample_VariableAll() throws Exception {
         verify("a = LOAD '" + Util.encodeEscape(tmpfilepath) + "'; " +
@@ -125,7 +124,7 @@ public class TestSample {
                 "c = FOREACH b GENERATE COUNT(a) AS count;" +
                 "myid = SAMPLE a 1.0 * (c.count / c.count) PARALLEL 2;", DATALEN, DATALEN); // test for PIG-2156
     }
-    
+
     @Test
     public void testSample_VariableSome() throws Exception {
         verify("a = LOAD '" + Util.encodeEscape(tmpfilepath) + "'; " +
@@ -133,11 +132,11 @@ public class TestSample {
                 "c = FOREACH b GENERATE COUNT(a) AS count;" +
                 "myid = SAMPLE a (c.count / (2.0 * c.count) );", DATALEN/3, DATALEN*2/3);
     }
-    
+
     @Test(expected=FrontendException.class)
     public void testSampleScalarException() throws IOException {
-        String query = 
-            "a = load '" + Util.encodeEscape(tmpfilepath) + "';" + 
+        String query =
+            "a = load '" + Util.encodeEscape(tmpfilepath) + "';" +
             "b = sample a $0;" // reference to non scalar context is not allowed
             ;
 

Modified: pig/branches/tez/test/org/apache/pig/test/TestSchema.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestSchema.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestSchema.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestSchema.java Thu May  8 20:50:06 2014
@@ -29,7 +29,6 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.ResourceSchema;
 import org.apache.pig.data.DataType;
@@ -613,6 +612,7 @@ public class TestSchema {
         assertFalse(Schema.equals(bagSchema1, bagSchema2, false, false));
     }
 
+    @Test
     public void testCharArray2Numeric(){
     	byte[] numbericTypes=new byte[]{DataType.DOUBLE,DataType.FLOAT,DataType.LONG,DataType.INTEGER};
     	Schema.FieldSchema inputFieldSchema=new Schema.FieldSchema("",DataType.CHARARRAY);
@@ -622,9 +622,10 @@ public class TestSchema {
     	}
     }
 
+    @Test
     public void testSchemaSerialization() throws IOException {
-        MiniCluster cluster = MiniCluster.buildCluster();
-        PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
+        PigServer pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
         String inputFileName = "testSchemaSerialization-input.txt";
         String[] inputData = new String[] { "foo\t1", "hello\t2" };
         Util.createInputFile(cluster, inputFileName, inputData);

Modified: pig/branches/tez/test/org/apache/pig/test/TestScriptLanguageJavaScript.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestScriptLanguageJavaScript.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestScriptLanguageJavaScript.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestScriptLanguageJavaScript.java Thu May  8 20:50:06 2014
@@ -26,7 +26,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.scripting.ScriptEngine;
 import org.apache.pig.tools.pigstats.PigStats;
@@ -38,7 +37,7 @@ import org.junit.Test;
 
 public class TestScriptLanguageJavaScript {
 
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private PigServer pigServer;
 
     @BeforeClass
@@ -53,8 +52,7 @@ public class TestScriptLanguageJavaScrip
 
     @Before
     public void setUp() throws Exception {
-        pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
-//        pigServer = new PigServer(ExecType.LOCAL);
+        pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
     }
 
     @After

Modified: pig/branches/tez/test/org/apache/pig/test/TestScriptUDF.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestScriptUDF.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestScriptUDF.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestScriptUDF.java Thu May  8 20:50:06 2014
@@ -20,35 +20,33 @@ package org.apache.pig.test;
 import java.io.File;
 import java.util.Iterator;
 
-import junit.framework.Assert;
-
 import org.apache.hadoop.util.Shell;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.data.TupleFactory;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 public class TestScriptUDF{
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private PigServer pigServer;
 
     TupleFactory mTf = TupleFactory.getInstance();
     BagFactory mBf = BagFactory.getInstance();
-    
+
     @Before
     public void setUp() throws Exception{
-        pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
     }
-    
+
     @AfterClass
     public static void oneTimeTearDown() throws Exception {
         cluster.shutDown();
     }
-    
+
     // See PIG-928
     @Test
     public void testJavascriptExampleScript() throws Exception{
@@ -132,7 +130,7 @@ public class TestScriptUDF{
         t = iter.next();
 
         Assert.assertTrue(t.toString().equals("(9)"));
-        
+
         Assert.assertFalse(iter.hasNext());
     }
 
@@ -143,7 +141,7 @@ public class TestScriptUDF{
      * to use a jython install, the Lib dir must be in the jython search path
      * via env variable JYTHON_HOME=jy_home or JYTHONPATH=jy_home/Lib:... or
      * jython-standalone.jar should be in the classpath
-     * 
+     *
      * Left in for now as we don't have paths to include other scripts in a
      * script in the e2e harness.
      *
@@ -156,7 +154,7 @@ public class TestScriptUDF{
 
     @Test
     public void testPythonNestedImportClassPath() throws Exception {
-        // Use different names for the script as PythonInterpreter is static in JythonScriptEngine 
+        // Use different names for the script as PythonInterpreter is static in JythonScriptEngine
         testPythonNestedImport("build/classes", "scriptC.py", "scriptD.py");
     }
 

Modified: pig/branches/tez/test/org/apache/pig/test/TestStoreOld.java
URL: http://svn.apache.org/viewvc/pig/branches/tez/test/org/apache/pig/test/TestStoreOld.java?rev=1593413&r1=1593412&r2=1593413&view=diff
==============================================================================
--- pig/branches/tez/test/org/apache/pig/test/TestStoreOld.java (original)
+++ pig/branches/tez/test/org/apache/pig/test/TestStoreOld.java Thu May  8 20:50:06 2014
@@ -23,7 +23,6 @@ import java.io.File;
 import java.io.PrintWriter;
 import java.util.Iterator;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
@@ -34,7 +33,7 @@ import org.junit.Test;
 
 public class TestStoreOld {
 
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private int LOOP_COUNT = 1024;
 
     String fileName;
@@ -114,7 +113,7 @@ public class TestStoreOld {
             pw.println(i + "\t" + i);
         }
         pw.close();
-        pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        pig = new PigServer(cluster.getExecType(), cluster.getProperties());
         fileName = "'" + FileLocalizer.hadoopify(f.toString(), pig.getPigContext()) + "'";
         tmpFile1 = "'" + FileLocalizer.getTemporaryPath(pig.getPigContext()).toString() + "'";
         tmpFile2 = "'" + FileLocalizer.getTemporaryPath(pig.getPigContext()).toString() + "'";