You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by da...@apache.org on 2014/12/08 23:29:32 UTC

svn commit: r1643944 [1/2] - in /pig/trunk: ./ src/org/apache/pig/builtin/mock/ test/ test/org/apache/pig/builtin/ test/org/apache/pig/builtin/mock/ test/org/apache/pig/impl/builtin/ test/org/apache/pig/newplan/logical/optimizer/ test/org/apache/pig/ne...

Author: daijy
Date: Mon Dec  8 22:29:31 2014
New Revision: 1643944

URL: http://svn.apache.org/r1643944
Log:
PIG-4350: Port local mode tests to Tez - part2

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/builtin/mock/Storage.java
    pig/trunk/test/org/apache/pig/builtin/TestOrcStorage.java
    pig/trunk/test/org/apache/pig/builtin/TestOrcStoragePushdown.java
    pig/trunk/test/org/apache/pig/builtin/TestPluckTuple.java
    pig/trunk/test/org/apache/pig/builtin/mock/TestMockStorage.java
    pig/trunk/test/org/apache/pig/impl/builtin/TestStreamingUDF.java
    pig/trunk/test/org/apache/pig/newplan/logical/optimizer/TestImplicitSplitOnTuple.java
    pig/trunk/test/org/apache/pig/newplan/logical/optimizer/TestSchemaResetter.java
    pig/trunk/test/org/apache/pig/newplan/logical/relational/TestLocationInPhysicalPlan.java
    pig/trunk/test/org/apache/pig/parser/TestLogicalPlanGenerator.java
    pig/trunk/test/org/apache/pig/test/TestAlgebraicEvalLocal.java
    pig/trunk/test/org/apache/pig/test/TestAssert.java
    pig/trunk/test/org/apache/pig/test/TestBatchAliases.java
    pig/trunk/test/org/apache/pig/test/TestBlackAndWhitelistValidator.java
    pig/trunk/test/org/apache/pig/test/TestBuiltInBagToTupleOrString.java
    pig/trunk/test/org/apache/pig/test/TestBuiltin.java
    pig/trunk/test/org/apache/pig/test/TestBuiltinInvoker.java
    pig/trunk/test/org/apache/pig/test/TestCase.java
    pig/trunk/test/org/apache/pig/test/TestCommit.java
    pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java
    pig/trunk/test/org/apache/pig/test/TestFilterOpNumeric.java
    pig/trunk/test/org/apache/pig/test/TestFilterOpString.java
    pig/trunk/test/org/apache/pig/test/TestForEachNestedPlanLocal.java
    pig/trunk/test/org/apache/pig/test/TestForEachStar.java
    pig/trunk/test/org/apache/pig/test/TestIn.java
    pig/trunk/test/org/apache/pig/test/TestInfixArithmetic.java
    pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java
    pig/trunk/test/org/apache/pig/test/TestJsonLoaderStorage.java
    pig/trunk/test/org/apache/pig/test/TestLOLoadDeterminedSchema.java
    pig/trunk/test/org/apache/pig/test/TestLimitSchemaStore.java
    pig/trunk/test/org/apache/pig/test/TestLoad.java
    pig/trunk/test/org/apache/pig/test/TestLocal.java
    pig/trunk/test/org/apache/pig/test/TestLocal2.java
    pig/trunk/test/org/apache/pig/test/TestMacroExpansion.java
    pig/trunk/test/org/apache/pig/test/TestNullConstant.java
    pig/trunk/test/org/apache/pig/test/TestOrderBy3.java
    pig/trunk/test/org/apache/pig/test/TestPONegative.java
    pig/trunk/test/org/apache/pig/test/TestParser.java
    pig/trunk/test/org/apache/pig/test/TestPigContext.java
    pig/trunk/test/org/apache/pig/test/TestPigSplit.java
    pig/trunk/test/org/apache/pig/test/TestPigStorage.java
    pig/trunk/test/org/apache/pig/test/TestPoissonSampleLoader.java
    pig/trunk/test/org/apache/pig/test/TestProject.java
    pig/trunk/test/org/apache/pig/test/TestProjectStarExpander.java
    pig/trunk/test/org/apache/pig/test/TestRank1.java
    pig/trunk/test/org/apache/pig/test/TestRank2.java
    pig/trunk/test/org/apache/pig/test/TestRegisteredJarVisibility.java
    pig/trunk/test/org/apache/pig/test/TestRelationToExprProject.java
    pig/trunk/test/org/apache/pig/test/TestScriptingLanguagePython.java
    pig/trunk/test/org/apache/pig/test/TestSplit.java
    pig/trunk/test/org/apache/pig/test/TestSplitIndex.java
    pig/trunk/test/org/apache/pig/test/TestStoreInstances.java
    pig/trunk/test/org/apache/pig/test/TestStreamingLocal.java
    pig/trunk/test/org/apache/pig/test/TestTypedMap.java
    pig/trunk/test/org/apache/pig/test/TestUDF.java
    pig/trunk/test/org/apache/pig/test/TestUDFContext.java
    pig/trunk/test/org/apache/pig/test/TestUDFGroovy.java
    pig/trunk/test/org/apache/pig/test/TestUDFWithoutParameter.java
    pig/trunk/test/org/apache/pig/test/TestUTF8.java
    pig/trunk/test/org/apache/pig/test/TestUnion.java
    pig/trunk/test/org/apache/pig/test/Util.java
    pig/trunk/test/org/apache/pig/test/pigunit/pig/TestGruntParser.java
    pig/trunk/test/org/apache/pig/tools/grunt/TestGruntParser.java
    pig/trunk/test/tez-local-tests

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Mon Dec  8 22:29:31 2014
@@ -28,6 +28,8 @@ PIG-4333: Split BigData tests into multi
  
 BUG FIXES
 
+PIG-4350: Port local mode tests to Tez - part2 (daijy)
+
 PIG-4326: AvroStorageSchemaConversionUtilities does not properly convert schema for maps of arrays of records (mprim via daijy)
 
 PIG-4345: e2e test "RubyUDFs_13" fails because of the different result of "group a all" in different engines like "spark", "mapreduce" (kellyzly via rohini)

Modified: pig/trunk/src/org/apache/pig/builtin/mock/Storage.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/builtin/mock/Storage.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/builtin/mock/Storage.java (original)
+++ pig/trunk/src/org/apache/pig/builtin/mock/Storage.java Mon Dec  8 22:29:31 2014
@@ -648,6 +648,10 @@ private MockRecordWriter mockRecordWrite
     @Override
     public RecordWriter<Object, Object> getRecordWriter(TaskAttemptContext arg0) throws IOException,
     InterruptedException {
+      if (arg0.getConfiguration().get("mapreduce.output.basename")!=null) {
+          return new MockRecordWriter(arg0.getConfiguration().get("mapreduce.output.basename") + "-" +
+                  arg0.getTaskAttemptID().getTaskID().getId());
+      }
       return new MockRecordWriter(getUniqueFile(arg0, "part", ".mock"));
     }
 

Modified: pig/trunk/test/org/apache/pig/builtin/TestOrcStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/builtin/TestOrcStorage.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/builtin/TestOrcStorage.java (original)
+++ pig/trunk/test/org/apache/pig/builtin/TestOrcStorage.java Mon Dec  8 22:29:31 2014
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertFal
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.util.ArrayList;
@@ -31,8 +32,11 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.ql.io.orc.CompressionKind;
 import org.apache.hadoop.hive.ql.io.orc.OrcFile;
 import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
@@ -51,9 +55,7 @@ import org.apache.hadoop.io.FloatWritabl
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
-import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.data.BinSedesTuple;
 import org.apache.pig.data.DataByteArray;
@@ -96,8 +98,8 @@ public class TestOrcStorage {
     }
 
     @Before
-    public void setup() throws ExecException, IOException {
-        pigServer = new PigServer(ExecType.LOCAL);
+    public void setup() throws Exception {
+        pigServer = new PigServer(Util.getLocalTestMode());
         fs = FileSystem.get(ConfigurationUtil.toConfiguration(pigServer.getPigContext().getProperties()));
         deleteTestFiles();
         pigServer.mkdirs(outbasedir);
@@ -189,8 +191,8 @@ public class TestOrcStorage {
     public void testSimpleStore() throws Exception {
         pigServer.registerQuery("A = load '" + INPUT1 + "' as (a0:int, a1:chararray);");
         pigServer.store("A", OUTPUT1, "OrcStorage");
-        Path outputFilePath = new Path(new Path(OUTPUT1), "part-m-00000");
-        Reader reader = OrcFile.createReader(fs, outputFilePath);
+        
+        Reader reader = OrcFile.createReader(fs, Util.getFirstPartFile(new Path(OUTPUT1)));
         assertEquals(reader.getNumberOfRows(), 2);
 
         RecordReader rows = reader.rows(null);
@@ -242,12 +244,12 @@ public class TestOrcStorage {
         pigServer.registerQuery("store B into '" + OUTPUT3 +"' using OrcStorage('-c SNAPPY');");
         pigServer.executeBatch();
 
-        Path outputFilePath = new Path(new Path(OUTPUT2), "part-r-00000");
+        Path outputFilePath = Util.getFirstPartFile(new Path(OUTPUT2));
         Reader reader = OrcFile.createReader(fs, outputFilePath);
         assertEquals(reader.getNumberOfRows(), 2);
         assertEquals(reader.getCompression(), CompressionKind.ZLIB);
 
-        Path outputFilePath2 = new Path(new Path(OUTPUT3), "part-r-00000");
+        Path outputFilePath2 = Util.getFirstPartFile(new Path(OUTPUT3));
         reader = OrcFile.createReader(fs, outputFilePath2);
         assertEquals(reader.getNumberOfRows(), 2);
         assertEquals(reader.getCompression(), CompressionKind.SNAPPY);

Modified: pig/trunk/test/org/apache/pig/builtin/TestOrcStoragePushdown.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/builtin/TestOrcStoragePushdown.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/builtin/TestOrcStoragePushdown.java (original)
+++ pig/trunk/test/org/apache/pig/builtin/TestOrcStoragePushdown.java Mon Dec  8 22:29:31 2014
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTru
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileOutputStream;
-import java.io.FileWriter;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.math.BigDecimal;
@@ -39,11 +38,9 @@ import org.apache.log4j.FileAppender;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.log4j.SimpleLayout;
-import org.apache.pig.ExecType;
 import org.apache.pig.Expression;
 import org.apache.pig.Expression.OpType;
 import org.apache.pig.PigServer;
-import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.executionengine.ExecJob;
 import org.apache.pig.impl.PigImplConstants;
 import org.apache.pig.impl.util.ObjectSerializer;
@@ -122,7 +119,7 @@ public class TestOrcStoragePushdown {
     }
 
     private static void createInputData() throws Exception {
-        pigServer = new PigServer(ExecType.LOCAL);
+        pigServer = new PigServer(Util.getLocalTestMode());
 
         new File(inpbasedir).mkdirs();
         new File(outbasedir).mkdirs();
@@ -181,9 +178,9 @@ public class TestOrcStoragePushdown {
     }
 
     @Before
-    public void setup() throws ExecException{
+    public void setup() throws Exception{
         Util.resetStateForExecModeSwitch();
-        pigServer = new PigServer(ExecType.LOCAL);
+        pigServer = new PigServer(Util.getLocalTestMode());
         orcStorage = new OrcStorage();
     }
 
@@ -353,9 +350,9 @@ public class TestOrcStoragePushdown {
     }
 
     // For eclipse debugging
-    private void testPredicatePushdownLocal(String filterStmt, int expectedRows) throws IOException {
+    private void testPredicatePushdownLocal(String filterStmt, int expectedRows) throws Exception {
 
-        PigServer pigServer_disabledRule = new PigServer(ExecType.LOCAL);
+        PigServer pigServer_disabledRule = new PigServer(Util.getLocalTestMode());
         // Test with PredicatePushdownOptimizer disabled.
         HashSet<String> disabledOptimizerRules = new HashSet<String>();
         disabledOptimizerRules.add("PredicatePushdownOptimizer");

Modified: pig/trunk/test/org/apache/pig/builtin/TestPluckTuple.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/builtin/TestPluckTuple.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/builtin/TestPluckTuple.java (original)
+++ pig/trunk/test/org/apache/pig/builtin/TestPluckTuple.java Mon Dec  8 22:29:31 2014
@@ -26,12 +26,12 @@ import static org.junit.Assert.assertTru
 
 import java.util.Iterator;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.util.Utils;
+import org.apache.pig.test.Util;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -40,7 +40,7 @@ public class TestPluckTuple {
 
     @Before
     public void setUp() throws Exception {
-        pigServer = new PigServer(ExecType.LOCAL);
+        pigServer = new PigServer(Util.getLocalTestMode());
     }
 
     @Test

Modified: pig/trunk/test/org/apache/pig/builtin/mock/TestMockStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/builtin/mock/TestMockStorage.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/builtin/mock/TestMockStorage.java (original)
+++ pig/trunk/test/org/apache/pig/builtin/mock/TestMockStorage.java Mon Dec  8 22:29:31 2014
@@ -19,6 +19,7 @@ package org.apache.pig.builtin.mock;
 
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertTrue;
+import static junit.framework.Assert.assertFalse;
 import static junit.framework.Assert.fail;
 import static org.apache.pig.builtin.mock.Storage.resetData;
 import static org.apache.pig.builtin.mock.Storage.schema;
@@ -28,19 +29,19 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecJob;
 import org.apache.pig.backend.executionengine.ExecJob.JOB_STATUS;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.Tuple;
+import org.apache.pig.test.Util;
 import org.junit.Test;
 
 public class TestMockStorage {
 
   @Test
   public void testMockStoreAndLoad() throws Exception {
-    PigServer pigServer = new PigServer(ExecType.LOCAL);
+    PigServer pigServer = new PigServer(Util.getLocalTestMode());
     Data data = resetData(pigServer);
 
     data.set("foo",
@@ -60,7 +61,7 @@ public class TestMockStorage {
   
   @Test
   public void testMockSchema() throws Exception {
-    PigServer pigServer = new PigServer(ExecType.LOCAL);
+    PigServer pigServer = new PigServer(Util.getLocalTestMode());
     Data data = resetData(pigServer);
 
     data.set("foo", "blah:chararray",
@@ -83,7 +84,7 @@ public class TestMockStorage {
 
   @Test
   public void testMockStoreUnion() throws Exception {
-    PigServer pigServer = new PigServer(ExecType.LOCAL);
+    PigServer pigServer = new PigServer(Util.getLocalTestMode());
     Data data = resetData(pigServer);
 
     data.set("input1",
@@ -122,7 +123,7 @@ public class TestMockStorage {
   
   @Test
   public void testBadUsage1() throws Exception {
-    PigServer pigServer = new PigServer(ExecType.LOCAL);
+    PigServer pigServer = new PigServer(Util.getLocalTestMode());
     Data data = resetData(pigServer);
 
     data.set("input1",
@@ -145,7 +146,7 @@ public class TestMockStorage {
   
   @Test
   public void testBadUsage2() throws Exception {
-    PigServer pigServer = new PigServer(ExecType.LOCAL);
+    PigServer pigServer = new PigServer(Util.getLocalTestMode());
     Data data = resetData(pigServer);
 
     data.set("input",
@@ -168,6 +169,10 @@ public class TestMockStorage {
             break;
         }
     }
+    if (Util.getLocalTestMode().toString().equals("TEZ_LOCAL")) {
+        assertFalse("job should have success in tez_local mode since we have two vertex, each for a single store", failed);
+        return;
+    }
     assertTrue("job should have failed for storing twice in the same location", failed);
 
   }

Modified: pig/trunk/test/org/apache/pig/impl/builtin/TestStreamingUDF.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/impl/builtin/TestStreamingUDF.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/impl/builtin/TestStreamingUDF.java (original)
+++ pig/trunk/test/org/apache/pig/impl/builtin/TestStreamingUDF.java Mon Dec  8 22:29:31 2014
@@ -26,7 +26,6 @@ import java.math.BigInteger;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.DataBag;
@@ -34,12 +33,26 @@ import org.apache.pig.data.Tuple;
 import org.apache.pig.data.TupleFactory;
 import org.apache.pig.test.MiniGenericCluster;
 import org.apache.pig.test.Util;
+import org.apache.pig.test.junit.OrderedJUnit4Runner;
+import org.apache.pig.test.junit.OrderedJUnit4Runner.TestOrder;
 import org.joda.time.DateTime;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
 
+// Need to run testPythonUDF_onCluster first due to TEZ-1802
+@RunWith(OrderedJUnit4Runner.class)
+@TestOrder({
+    "testPythonUDF_onCluster",
+    "testPythonUDF__allTypes",
+    "testPythonUDF__withBigDecimal",
+    "testPythonUDF",
+    "testPythonUDF__withBigInteger",
+    "testPythonUDF__withDateTime",
+    "testPythonUDF_withNewline"
+})
 public class TestStreamingUDF {
     private static PigServer pigServerLocal = null;
     private static PigServer pigServerMapReduce = null;
@@ -101,7 +114,7 @@ public class TestStreamingUDF {
 
     @Test
     public void testPythonUDF() throws Exception {
-        pigServerLocal = new PigServer(ExecType.LOCAL);
+        pigServerLocal = new PigServer(Util.getLocalTestMode());
 
         String[] pythonScript = {
                 "from pig_util import outputSchema",
@@ -136,7 +149,7 @@ public class TestStreamingUDF {
     
     @Test
     public void testPythonUDF_withNewline() throws Exception {
-        pigServerLocal = new PigServer(ExecType.LOCAL);
+        pigServerLocal = new PigServer(Util.getLocalTestMode());
 
         String[] pythonScript = {
                 "from pig_util import outputSchema",
@@ -171,7 +184,7 @@ public class TestStreamingUDF {
     
     @Test
     public void testPythonUDF__withBigInteger() throws Exception {
-        pigServerLocal = new PigServer(ExecType.LOCAL);
+        pigServerLocal = new PigServer(Util.getLocalTestMode());
 
         String[] pythonScript = {
                 "from pig_util import outputSchema",
@@ -199,7 +212,7 @@ public class TestStreamingUDF {
     
     @Test
     public void testPythonUDF__withBigDecimal() throws Exception {
-        pigServerLocal = new PigServer(ExecType.LOCAL);
+        pigServerLocal = new PigServer(Util.getLocalTestMode());
 
         String[] pythonScript = {
                 "from pig_util import outputSchema",
@@ -229,7 +242,7 @@ public class TestStreamingUDF {
     
     @Test
     public void testPythonUDF__withDateTime() throws Exception {
-        pigServerLocal = new PigServer(ExecType.LOCAL);
+        pigServerLocal = new PigServer(Util.getLocalTestMode());
 
         String[] pythonScript = {
                 "from pig_util import outputSchema",
@@ -257,7 +270,7 @@ public class TestStreamingUDF {
     
     @Test
     public void testPythonUDF__allTypes() throws Exception {
-        pigServerLocal = new PigServer(ExecType.LOCAL);
+        pigServerLocal = new PigServer(Util.getLocalTestMode());
 
         String[] pythonScript = {
             "# -*- coding: utf-8 -*-",

Modified: pig/trunk/test/org/apache/pig/newplan/logical/optimizer/TestImplicitSplitOnTuple.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/newplan/logical/optimizer/TestImplicitSplitOnTuple.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/newplan/logical/optimizer/TestImplicitSplitOnTuple.java (original)
+++ pig/trunk/test/org/apache/pig/newplan/logical/optimizer/TestImplicitSplitOnTuple.java Mon Dec  8 22:29:31 2014
@@ -18,16 +18,11 @@
 
 package org.apache.pig.newplan.logical.optimizer;
 
-import static org.apache.pig.ExecType.LOCAL;
 import static org.apache.pig.builtin.mock.Storage.tuple;
 import static org.junit.Assert.assertEquals;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.pig.PigRunner;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage;
 import org.apache.pig.builtin.mock.Storage.Data;
@@ -39,8 +34,8 @@ import org.junit.Test;
 public class TestImplicitSplitOnTuple {
 
     @Test
-    public void testImplicitSplitterOnTuple() throws IOException {
-        PigServer pigServer = new PigServer(LOCAL);
+    public void testImplicitSplitterOnTuple() throws Exception {
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = Storage.resetData(pigServer);
         data.set("input",
                 tuple("1", "1001", "101"),

Modified: pig/trunk/test/org/apache/pig/newplan/logical/optimizer/TestSchemaResetter.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/newplan/logical/optimizer/TestSchemaResetter.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/newplan/logical/optimizer/TestSchemaResetter.java (original)
+++ pig/trunk/test/org/apache/pig/newplan/logical/optimizer/TestSchemaResetter.java Mon Dec  8 22:29:31 2014
@@ -18,7 +18,6 @@
 
 package org.apache.pig.newplan.logical.optimizer;
 
-import static org.apache.pig.ExecType.LOCAL;
 import static org.apache.pig.builtin.mock.Storage.tuple;
 import static org.junit.Assert.assertEquals;
 
@@ -68,8 +67,8 @@ public class TestSchemaResetter {
     }
 
     @Test
-    public void testSchemaResetterExec() throws IOException {
-        PigServer pigServer = new PigServer(LOCAL);
+    public void testSchemaResetterExec() throws Exception {
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = Storage.resetData(pigServer);
         data.set("input",
                 tuple(tuple("1", "2")),

Modified: pig/trunk/test/org/apache/pig/newplan/logical/relational/TestLocationInPhysicalPlan.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/newplan/logical/relational/TestLocationInPhysicalPlan.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/newplan/logical/relational/TestLocationInPhysicalPlan.java (original)
+++ pig/trunk/test/org/apache/pig/newplan/logical/relational/TestLocationInPhysicalPlan.java Mon Dec  8 22:29:31 2014
@@ -23,7 +23,6 @@ import java.util.List;
 
 import junit.framework.Assert;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecJob;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.OriginalLocation;
@@ -46,7 +45,7 @@ public class TestLocationInPhysicalPlan
             "3,1,1",
             "1,2,1",
         });
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         pigServer.setBatchOn();
         pigServer.registerQuery(
                 "A = LOAD '" + Util.encodeEscape(input.getAbsolutePath()) + "' using PigStorage();\n"
@@ -61,6 +60,10 @@ public class TestLocationInPhysicalPlan
         Assert.assertEquals(0, originalLocation.getOffset());
         Assert.assertEquals("A", originalLocation.getAlias());
         JobStats jStats = (JobStats)job.getStatistics().getJobGraph().getSinks().get(0);
-        Assert.assertEquals("M: A[1,4],A[3,4],B[2,4] C: A[3,4],B[2,4] R: A[3,4]", jStats.getAliasLocation());
+        if (Util.getLocalTestMode().toString().equals("TEZ_LOCAL")) {
+            Assert.assertEquals("A[1,4],A[3,4],B[2,4]", jStats.getAliasLocation());
+        } else {
+            Assert.assertEquals("M: A[1,4],A[3,4],B[2,4] C: A[3,4],B[2,4] R: A[3,4]", jStats.getAliasLocation());
+        }
     }
 }

Modified: pig/trunk/test/org/apache/pig/parser/TestLogicalPlanGenerator.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/parser/TestLogicalPlanGenerator.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/parser/TestLogicalPlanGenerator.java (original)
+++ pig/trunk/test/org/apache/pig/parser/TestLogicalPlanGenerator.java Mon Dec  8 22:29:31 2014
@@ -29,7 +29,6 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.antlr.runtime.MismatchedTokenException;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.Tuple;
@@ -50,7 +49,7 @@ public class TestLogicalPlanGenerator {
 
     @Before
     public void setUp() throws Exception {
-        pigServer = new PigServer(ExecType.LOCAL);
+        pigServer = new PigServer(Util.getLocalTestMode());
     }
 
     @BeforeClass

Modified: pig/trunk/test/org/apache/pig/test/TestAlgebraicEvalLocal.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestAlgebraicEvalLocal.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestAlgebraicEvalLocal.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestAlgebraicEvalLocal.java Mon Dec  8 22:29:31 2014
@@ -26,7 +26,6 @@ import java.util.Iterator;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.DataType;
@@ -42,7 +41,7 @@ public class TestAlgebraicEvalLocal {
 
     @Before
     public void setUp() throws Exception {
-        pig = new PigServer(ExecType.LOCAL, new Properties());
+        pig = new PigServer(Util.getLocalTestMode(), new Properties());
     }
 
     Boolean[] nullFlags = new Boolean[]{ false, true};

Modified: pig/trunk/test/org/apache/pig/test/TestAssert.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestAssert.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestAssert.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestAssert.java Mon Dec  8 22:29:31 2014
@@ -24,12 +24,9 @@ import static org.apache.pig.builtin.moc
 import java.io.ByteArrayInputStream;
 import java.io.InputStream;
 import java.util.List;
-import java.util.Properties;
 
 import junit.framework.Assert;
 
-import org.apache.pig.ExecType;
-import org.apache.pig.PigConfiguration;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.Tuple;
@@ -44,7 +41,7 @@ public class TestAssert {
    */
   @Test
   public void testPositive() throws Exception {
-      PigServer pigServer = new PigServer(ExecType.LOCAL);
+      PigServer pigServer = new PigServer(Util.getLocalTestMode());
       Data data = resetData(pigServer);
 
       data.set("foo",
@@ -74,7 +71,7 @@ public class TestAssert {
    */
   @Test
   public void testInScript() throws Exception {
-      PigServer pigServer = new PigServer(ExecType.LOCAL);
+      PigServer pigServer = new PigServer(Util.getLocalTestMode());
       Data data = resetData(pigServer);
 
       data.set("foo",
@@ -104,7 +101,7 @@ public class TestAssert {
    */
   @Test
   public void testNegative() throws Exception {
-      PigServer pigServer = new PigServer(ExecType.LOCAL);
+      PigServer pigServer = new PigServer(Util.getLocalTestMode());
       Data data = resetData(pigServer);
 
       data.set("foo",
@@ -130,7 +127,7 @@ public class TestAssert {
    */
   @Test
   public void testNegativeWithoutFetch() throws Exception {
-      PigServer pigServer = new PigServer(ExecType.LOCAL);
+      PigServer pigServer = new PigServer(Util.getLocalTestMode());
       Data data = resetData(pigServer);
 
       data.set("foo",
@@ -156,7 +153,7 @@ public class TestAssert {
    */
   @Test(expected=FrontendException.class)
   public void testNegativeWithAlias() throws Exception {
-      PigServer pigServer = new PigServer(ExecType.LOCAL);
+      PigServer pigServer = new PigServer(Util.getLocalTestMode());
       Data data = resetData(pigServer);
 
       data.set("foo",

Modified: pig/trunk/test/org/apache/pig/test/TestBatchAliases.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestBatchAliases.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestBatchAliases.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestBatchAliases.java Mon Dec  8 22:29:31 2014
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTru
 import java.io.IOException;
 import java.util.Properties;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigConfiguration;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecJob;
@@ -39,7 +38,7 @@ public class TestBatchAliases {
     @Before
     public void setUp() throws Exception {
         System.setProperty(PigConfiguration.PIG_OPT_MULTIQUERY, ""+true);
-        myPig = new PigServer(ExecType.LOCAL, new Properties());
+        myPig = new PigServer(Util.getLocalTestMode(), new Properties());
         deleteOutputFiles();
     }
 

Modified: pig/trunk/test/org/apache/pig/test/TestBlackAndWhitelistValidator.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestBlackAndWhitelistValidator.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestBlackAndWhitelistValidator.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestBlackAndWhitelistValidator.java Mon Dec  8 22:29:31 2014
@@ -26,7 +26,6 @@ import java.io.File;
 import java.util.Properties;
 
 import org.apache.commons.io.IOUtils;
-import org.apache.pig.ExecType;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.PigConfiguration;
 import org.apache.pig.PigServer;
@@ -61,7 +60,7 @@ public class TestBlackAndWhitelistValida
 
     @Before
     public void setUp() throws Exception {
-        ctx = new PigContext(ExecType.LOCAL, new Properties());
+        ctx = new PigContext(Util.getLocalTestMode(), new Properties());
         ctx.connect();
     }
 

Modified: pig/trunk/test/org/apache/pig/test/TestBuiltInBagToTupleOrString.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestBuiltInBagToTupleOrString.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestBuiltInBagToTupleOrString.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestBuiltInBagToTupleOrString.java Mon Dec  8 22:29:31 2014
@@ -22,7 +22,6 @@ import static junit.framework.Assert.*;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.builtin.BagToString;
@@ -407,7 +406,7 @@ public class TestBuiltInBagToTupleOrStri
 
 	@Test
 	public void testPigScriptForBagToTupleUDF() throws Exception {
-		PigServer pigServer = new PigServer(ExecType.LOCAL);
+		PigServer pigServer = new PigServer(Util.getLocalTestMode());
 		Data data = resetData(pigServer);
 
 		// bag of chararray
@@ -436,7 +435,7 @@ public class TestBuiltInBagToTupleOrStri
 
 	@Test
 	public void testPigScriptMultipleElmementsPerTupleForBagTupleUDF() throws Exception {
-		PigServer pigServer = new PigServer(ExecType.LOCAL);
+		PigServer pigServer = new PigServer(Util.getLocalTestMode());
 		Data data = resetData(pigServer);
 
 		data.set("foo", "myBag:bag{t:(l:chararray)}",
@@ -451,7 +450,7 @@ public class TestBuiltInBagToTupleOrStri
 
 	@Test
 	public void testPigScriptNestedTupleForBagToTupleDF() throws Exception {
-		PigServer pigServer = new PigServer(ExecType.LOCAL);
+		PigServer pigServer = new PigServer(Util.getLocalTestMode());
 		Data data = resetData(pigServer);
 
 	    Tuple nestedTuple = tuple(bag(tuple("c"), tuple("d")));
@@ -469,7 +468,7 @@ public class TestBuiltInBagToTupleOrStri
 
 	@Test
 	public void testPigScriptEmptyBagForBagToTupleUDF() throws Exception {
-		PigServer pigServer = new PigServer(ExecType.LOCAL);
+		PigServer pigServer = new PigServer(Util.getLocalTestMode());
 		Data data = resetData(pigServer);
 
 	    data.set("foo", "myBag:bag{t:(l:chararray)}",
@@ -487,7 +486,7 @@ public class TestBuiltInBagToTupleOrStri
 
 	@Test
 	public void testPigScriptrForBagToStringUDF() throws Exception {
-		PigServer pigServer = new PigServer(ExecType.LOCAL);
+		PigServer pigServer = new PigServer(Util.getLocalTestMode());
 		Data data = resetData(pigServer);
 
 		data.set("foo", "myBag:bag{t:(l:chararray)}",
@@ -509,7 +508,7 @@ public class TestBuiltInBagToTupleOrStri
 
 	@Test
 	public void testPigScriptMultipleElmementsPerTupleForBagToStringUDF() throws Exception {
-		PigServer pigServer = new PigServer(ExecType.LOCAL);
+		PigServer pigServer = new PigServer(Util.getLocalTestMode());
 		Data data = resetData(pigServer);
 
 		data.set("foo", "myBag:bag{t:(l:chararray)}",
@@ -530,7 +529,7 @@ public class TestBuiltInBagToTupleOrStri
 
 	@Test
 	public void testPigScriptNestedTupleForBagToStringUDF() throws Exception {
-		PigServer pigServer = new PigServer(ExecType.LOCAL);
+		PigServer pigServer = new PigServer(Util.getLocalTestMode());
 		Data data = resetData(pigServer);
 
 	    Tuple nestedTuple = tuple(bag(tuple("c"), tuple("d")));
@@ -548,7 +547,7 @@ public class TestBuiltInBagToTupleOrStri
 
 	@Test
 	public void testPigScriptEmptyBagForBagToStringUDF() throws Exception {
-		PigServer pigServer = new PigServer(ExecType.LOCAL);
+		PigServer pigServer = new PigServer(Util.getLocalTestMode());
 		Data data = resetData(pigServer);
 
 	    data.set("foo", "myBag:bag{t:(l:chararray)}",

Modified: pig/trunk/test/org/apache/pig/test/TestBuiltin.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestBuiltin.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestBuiltin.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestBuiltin.java Mon Dec  8 22:29:31 2014
@@ -42,7 +42,6 @@ import java.math.BigInteger;
 
 import org.apache.pig.Algebraic;
 import org.apache.pig.EvalFunc;
-import org.apache.pig.ExecType;
 import org.apache.pig.LoadFunc;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
@@ -204,7 +203,7 @@ public class TestBuiltin {
     public void setUp() throws Exception {
         Util.resetStateForExecModeSwitch();
 
-        pigServer = new PigServer(ExecType.LOCAL, new Properties());
+        pigServer = new PigServer(Util.getLocalTestMode(), new Properties());
         pigServer.setValidateEachStatement(true);
         // First set up data structs for "base" SUM, MIN and MAX and AVG.
         // The allowed input and expected output data structs for
@@ -2411,7 +2410,7 @@ public class TestBuiltin {
     public void testComplexMultiCONCAT() throws Exception {
         String input = "vararg_concat_test_jira_3444.txt";
         Util.createLocalInputFile(input, new String[]{"dummy"});
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         pigServer.registerQuery("A = LOAD '"+input+"' as (x:chararray);");
         
         pigServer.registerQuery("B = foreach A generate CONCAT('a', CONCAT('b',CONCAT('c','d')));");
@@ -2483,7 +2482,7 @@ public class TestBuiltin {
         //
         String input = "vararg_sprintf_test_jira_3939.txt";
         Util.createLocalInputFile(input, new String[]{"dummy"});
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         pigServer.registerQuery("A = LOAD '"+input+"' as (x:chararray);");
         //
         pigServer.registerQuery("B = foreach A generate SPRINTF('%6s|%-8s|%2$,+12d %2$8x', 'yay', 665568);");

Modified: pig/trunk/test/org/apache/pig/test/TestBuiltinInvoker.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestBuiltinInvoker.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestBuiltinInvoker.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestBuiltinInvoker.java Mon Dec  8 22:29:31 2014
@@ -27,7 +27,6 @@ import java.util.List;
 import java.util.Random;
 import java.util.Set;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.Tuple;
@@ -50,7 +49,7 @@ public class TestBuiltinInvoker {
 
     @Before
     public void setUp() throws Exception {
-        pigServer = new PigServer(ExecType.LOCAL);
+        pigServer = new PigServer(Util.getLocalTestMode());
 
         data = resetData(pigServer);
 

Modified: pig/trunk/test/org/apache/pig/test/TestCase.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestCase.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestCase.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestCase.java Mon Dec  8 22:29:31 2014
@@ -25,7 +25,6 @@ import static org.junit.Assert.fail;
 
 import java.util.List;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.Tuple;
@@ -40,7 +39,7 @@ public class TestCase {
      */
     @Test
     public void testNoElseBranch() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -81,7 +80,7 @@ public class TestCase {
      */
     @Test
     public void testWithElseBranch() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -122,7 +121,7 @@ public class TestCase {
      */
     @Test
     public void testConditionalWithNoElse() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -163,7 +162,7 @@ public class TestCase {
      */
     @Test
     public void testConditionalWithElse() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -204,7 +203,7 @@ public class TestCase {
      */
     @Test
     public void testOrderOfConditions() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -244,7 +243,7 @@ public class TestCase {
      */
     @Test
     public void testWithDereferenceOperator() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -281,7 +280,7 @@ public class TestCase {
      */
     @Test(expected = FrontendException.class)
     public void testMissingCaseExpression() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -310,7 +309,7 @@ public class TestCase {
      */
     @Test(expected = FrontendException.class)
     public void testMissingWhenExpression() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -339,7 +338,7 @@ public class TestCase {
      */
     @Test(expected = FrontendException.class)
     public void testMissingThenExpression() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -368,7 +367,7 @@ public class TestCase {
      */
     @Test(expected = FrontendException.class)
     public void testMissingElseExpression() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",

Modified: pig/trunk/test/org/apache/pig/test/TestCommit.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestCommit.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestCommit.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestCommit.java Mon Dec  8 22:29:31 2014
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTru
 import java.io.File;
 import java.util.Iterator;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.Tuple;
@@ -39,7 +38,7 @@ public class TestCommit {
 
     @Before
     public void setUp() throws Exception{
-        pigServer = new PigServer(ExecType.LOCAL);
+        pigServer = new PigServer(Util.getLocalTestMode());
     }
 
     @Test

Modified: pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java Mon Dec  8 22:29:31 2014
@@ -29,7 +29,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.builtin.BinStorage;
@@ -52,7 +51,7 @@ public class TestDataBagAccess {
 
     @Before
     public void setUp() throws Exception{
-        pigServer = new PigServer(ExecType.LOCAL, new Properties());
+        pigServer = new PigServer(Util.getLocalTestMode(), new Properties());
     }
 
     @Test

Modified: pig/trunk/test/org/apache/pig/test/TestFilterOpNumeric.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestFilterOpNumeric.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestFilterOpNumeric.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestFilterOpNumeric.java Mon Dec  8 22:29:31 2014
@@ -28,7 +28,6 @@ import java.util.Iterator;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.Tuple;
@@ -44,7 +43,7 @@ public class TestFilterOpNumeric {
 
     @Before
     public void setUp() throws Exception {
-        pig = new PigServer(ExecType.LOCAL);
+        pig = new PigServer(Util.getLocalTestMode());
     }
 
     @Test

Modified: pig/trunk/test/org/apache/pig/test/TestFilterOpString.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestFilterOpString.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestFilterOpString.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestFilterOpString.java Mon Dec  8 22:29:31 2014
@@ -28,7 +28,6 @@ import java.util.Iterator;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.Tuple;
@@ -46,7 +45,7 @@ public class TestFilterOpString {
     @Before
     public void setUp() throws Exception {
         FileLocalizer.deleteTempFiles();
-        pig = new PigServer(ExecType.LOCAL);
+        pig = new PigServer(Util.getLocalTestMode());
     }
 
     @Test

Modified: pig/trunk/test/org/apache/pig/test/TestForEachNestedPlanLocal.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestForEachNestedPlanLocal.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestForEachNestedPlanLocal.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestForEachNestedPlanLocal.java Mon Dec  8 22:29:31 2014
@@ -19,7 +19,6 @@
 package org.apache.pig.test;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -30,7 +29,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.test.utils.TestHelper;
@@ -41,7 +39,7 @@ public class TestForEachNestedPlanLocal
     private PigServer pig ;
 
     public TestForEachNestedPlanLocal() throws Throwable {
-        pig = new PigServer(ExecType.LOCAL) ;
+        pig = new PigServer(Util.getLocalTestMode()) ;
     }
 
     Boolean[] nullFlags = new Boolean[]{ false, true };
@@ -79,19 +77,9 @@ public class TestForEachNestedPlanLocal
         pig.registerQuery("c = foreach b { " + "     c1 = limit $1 5; "
                 + "    generate COUNT(c1); " + "};");
         Iterator<Tuple> it = pig.openIterator("c");
-        Tuple t = null;
-        long count[] = new long[3];
-        for (int i = 0; i < 3 && it.hasNext(); i++) {
-            t = it.next();
-            count[i] = (Long)t.get(0);
-        }
-
-        assertFalse(it.hasNext());
+        List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(new String[] {"(5L)", "(5L)", "(3L)" });
 
-        // Pig's previous local mode was screwed up correcting that
-        assertEquals(5L, count[0]);
-        assertEquals(5L, count[1]);
-        assertEquals(3L, count[2]);
+        Util.checkQueryOutputsAfterSort(it, expected);
     }
 
     @Test

Modified: pig/trunk/test/org/apache/pig/test/TestForEachStar.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestForEachStar.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestForEachStar.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestForEachStar.java Mon Dec  8 22:29:31 2014
@@ -20,17 +20,12 @@ package org.apache.pig.test;
 import static org.junit.Assert.*;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.Iterator;
-import java.util.List;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
-import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.data.TupleFactory;
-import org.apache.pig.parser.ParserException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -65,8 +60,8 @@ public class TestForEachStar {
     }
 
     @Test
-    public void testForeachStarSchemaUnkown() throws IOException, ParserException{
-        PigServer pig = new PigServer(ExecType.LOCAL);
+    public void testForeachStarSchemaUnkown() throws Exception {
+        PigServer pig = new PigServer(Util.getLocalTestMode());
         String query =
             "  l1 = load '" + INPUT_FILE + "' ;"
             + "f1 = foreach l1 generate * ;"

Modified: pig/trunk/test/org/apache/pig/test/TestIn.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestIn.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestIn.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestIn.java Mon Dec  8 22:29:31 2014
@@ -25,7 +25,6 @@ import static org.junit.Assert.fail;
 
 import java.util.List;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.Tuple;
@@ -40,7 +39,7 @@ public class TestIn {
      */
     @Test
     public void testWithFilter() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -68,7 +67,7 @@ public class TestIn {
      */
     @Test
     public void testWithBincond() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -98,7 +97,7 @@ public class TestIn {
      */
     @Test
     public void testWithSplit() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -132,7 +131,7 @@ public class TestIn {
      */
     @Test
     public void testWithDereferenceOperator() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",
@@ -161,7 +160,7 @@ public class TestIn {
      */
     @Test(expected = FrontendException.class)
     public void testMissingRhsOperand() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL);
+        PigServer pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         data.set("foo",

Modified: pig/trunk/test/org/apache/pig/test/TestInfixArithmetic.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestInfixArithmetic.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestInfixArithmetic.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestInfixArithmetic.java Mon Dec  8 22:29:31 2014
@@ -28,7 +28,6 @@ import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.DataType;
@@ -46,7 +45,7 @@ public class TestInfixArithmetic {
 
     @Before
     public void setUp() throws Exception {
-        pig = new PigServer(ExecType.LOCAL);
+        pig = new PigServer(Util.getLocalTestMode());
     }
 
     Boolean[] nullFlags = new Boolean[] { false, true };

Modified: pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java Mon Dec  8 22:29:31 2014
@@ -28,7 +28,6 @@ import java.io.IOException;
 import java.util.Iterator;
 import java.util.Properties;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.PigException;
 import org.apache.pig.PigServer;
@@ -57,9 +56,9 @@ public class TestInputOutputFileValidato
 
     @Before
     public void setUp() throws Exception {
-        ctx = new PigContext(ExecType.LOCAL, new Properties());
+        ctx = new PigContext(Util.getLocalTestMode(), new Properties());
         ctx.connect();
-        pig = new PigServer(ExecType.LOCAL);
+        pig = new PigServer(Util.getLocalTestMode());
     }
 
     @Test

Modified: pig/trunk/test/org/apache/pig/test/TestJsonLoaderStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestJsonLoaderStorage.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestJsonLoaderStorage.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestJsonLoaderStorage.java Mon Dec  8 22:29:31 2014
@@ -26,12 +26,16 @@ import java.io.File;
 import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.util.Iterator;
 import java.util.Map;
 
-import org.apache.pig.ExecType;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.builtin.mock.Storage;
@@ -138,8 +142,8 @@ public class TestJsonLoaderStorage {
   private PigServer pigServer;
 
   @Before
-  public void setup() throws ExecException {
-    pigServer = new PigServer(ExecType.LOCAL);
+  public void setup() throws Exception {
+    pigServer = new PigServer(Util.getLocalTestMode());
   }
 
   private String getTempOutputPath() throws IOException {
@@ -176,7 +180,7 @@ public class TestJsonLoaderStorage {
     return pigServer.openIterator("data");
   }
 
-  private BufferedReader storeJson(String input) throws IOException {
+  private BufferedReader storeJson(String input) throws Exception {
     String pathInputFile = createInput(input);
     String pathJsonFile = getTempOutputPath();
     pigServer.registerQuery("data = load '" + pathInputFile
@@ -184,7 +188,10 @@ public class TestJsonLoaderStorage {
     pigServer.registerQuery("store data into '" + pathJsonFile
         + "' using JsonStorage();");
 
-    FileReader r = new FileReader(pathJsonFile + "/part-m-00000");
+    Path p = new Path(pathJsonFile);
+    FileSystem fs = FileSystem.get(p.toUri(), new Configuration());
+    Reader r = new InputStreamReader(fs.open(Util.getFirstPartFile(p)));
+
     BufferedReader br = new BufferedReader(r);
 
     return br;
@@ -307,7 +314,7 @@ public class TestJsonLoaderStorage {
   }
 
   @Test
-  public void testJsonStorage() throws IOException {
+  public void testJsonStorage() throws Exception {
     BufferedReader br = storeJson(rawInput);
     String data = br.readLine();
 
@@ -325,7 +332,7 @@ public class TestJsonLoaderStorage {
   }
 
   @Test
-  public void testJsonStorageNull() throws IOException {
+  public void testJsonStorageNull() throws Exception {
     BufferedReader br = storeJson(nullInput);
     String data = br.readLine();
 
@@ -343,7 +350,7 @@ public class TestJsonLoaderStorage {
   }
 
   @Test
-  public void testJsonLoaderStorage() throws IOException {
+  public void testJsonLoaderStorage() throws Exception {
 
     String pattInputFile = createInput(rawInput);
     String pattJsonFile = getTempOutputPath();
@@ -358,7 +365,9 @@ public class TestJsonLoaderStorage {
     pigServer.registerQuery("store json into '" + pattJson2File
         + "' using JsonStorage();");
 
-    FileReader r = new FileReader(pattJson2File + "/part-m-00000");
+    Path p = new Path(pattJson2File);
+    FileSystem fs = FileSystem.get(p.toUri(), new Configuration());
+    Reader r = new InputStreamReader(fs.open(Util.getFirstPartFile(p)));
 
     BufferedReader br = new BufferedReader(r);
     String data = br.readLine();
@@ -386,7 +395,9 @@ public class TestJsonLoaderStorage {
     pigServer.registerQuery("data = limit data 2;");
     pigServer.registerQuery("store data into '" + outPath + "' using JsonStorage();");
 
-    FileReader r = new FileReader(outPath + "/part-r-00000");
+    Path p = new Path(outPath);
+    FileSystem fs = FileSystem.get(p.toUri(), new Configuration());
+    Reader r = new InputStreamReader(fs.open(Util.getFirstPartFile(p)));
 
     BufferedReader br = new BufferedReader(r);
 
@@ -432,7 +443,11 @@ public class TestJsonLoaderStorage {
             "};");
     pigServer.store("uniqcnt", tempJsonFile.getAbsolutePath(), "JsonStorage");
 
-    BufferedReader br = new BufferedReader(new FileReader(tempJsonFile.getAbsolutePath()+ "/part-r-00000"));
+    Path p = new Path(tempJsonFile.getAbsolutePath());
+    FileSystem fs = FileSystem.get(p.toUri(), new Configuration());
+    Reader r = new InputStreamReader(fs.open(Util.getFirstPartFile(p)));
+
+    BufferedReader br = new BufferedReader(r);
     String data = br.readLine();
 
     assertEquals(jsonOutput, data);

Modified: pig/trunk/test/org/apache/pig/test/TestLOLoadDeterminedSchema.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestLOLoadDeterminedSchema.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestLOLoadDeterminedSchema.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestLOLoadDeterminedSchema.java Mon Dec  8 22:29:31 2014
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.util.Properties;
 
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.pig.ExecType;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.PigServer;
 import org.apache.pig.impl.PigContext;
@@ -86,7 +85,7 @@ public class TestLOLoadDeterminedSchema
     @Before
     public void setUp() throws Exception {
         FileLocalizer.deleteTempFiles();
-        server = new PigServer(ExecType.LOCAL, new Properties());
+        server = new PigServer(Util.getLocalTestMode(), new Properties());
 
         baseDir = new File("build/testLoLoadDeterminedSchema");
 

Modified: pig/trunk/test/org/apache/pig/test/TestLimitSchemaStore.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestLimitSchemaStore.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestLimitSchemaStore.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestLimitSchemaStore.java Mon Dec  8 22:29:31 2014
@@ -19,7 +19,6 @@ package org.apache.pig.test;
 
 import java.io.File;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
@@ -34,7 +33,7 @@ public class TestLimitSchemaStore{
 
     @Before
     public void setUp() throws Exception{
-        pigServer = new PigServer(ExecType.LOCAL);
+        pigServer = new PigServer(Util.getLocalTestMode());
     }
     
     

Modified: pig/trunk/test/org/apache/pig/test/TestLoad.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestLoad.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestLoad.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestLoad.java Mon Dec  8 22:29:31 2014
@@ -33,7 +33,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.PigConfiguration;
 import org.apache.pig.PigServer;
@@ -73,7 +72,7 @@ public class TestLoad {
         FileLocalizer.deleteTempFiles();
         servers = new PigServer[] {
                     new PigServer(cluster.getExecType(), cluster.getProperties()),
-                    new PigServer(ExecType.LOCAL, new Properties())
+                    new PigServer(Util.getLocalTestMode(), new Properties())
         };
     }
 

Modified: pig/trunk/test/org/apache/pig/test/TestLocal.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestLocal.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestLocal.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestLocal.java Mon Dec  8 22:29:31 2014
@@ -31,7 +31,6 @@ import java.util.Iterator;
 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.executionengine.ExecException;
@@ -56,7 +55,7 @@ public class TestLocal {
 
     @Before
     public void setUp() throws Exception {
-        pig = new PigServer(ExecType.LOCAL);
+        pig = new PigServer(Util.getLocalTestMode());
     }
 
     @Test

Modified: pig/trunk/test/org/apache/pig/test/TestLocal2.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestLocal2.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestLocal2.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestLocal2.java Mon Dec  8 22:29:31 2014
@@ -32,7 +32,6 @@ import java.util.Iterator;
 import junit.framework.Assert;
 
 import org.apache.pig.EvalFunc;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.DataBag;
@@ -45,7 +44,7 @@ public class TestLocal2 {
     private PigServer pig ;
 
     public TestLocal2() throws Throwable {
-        pig = new PigServer(ExecType.LOCAL) ;
+        pig = new PigServer(Util.getLocalTestMode()) ;
     }
 
 

Modified: pig/trunk/test/org/apache/pig/test/TestMacroExpansion.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestMacroExpansion.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestMacroExpansion.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestMacroExpansion.java Mon Dec  8 22:29:31 2014
@@ -29,7 +29,6 @@ import java.io.PrintWriter;
 import java.io.StringReader;
 import java.util.Properties;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigRunner;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.parser.DryRunGruntParser;
@@ -1434,7 +1433,7 @@ public class TestMacroExpansion {
     @Test
     public void test2() throws Exception {
         String query = "A = load 'x' as ( u:int, v:long, w:bytearray); " + 
-                       "B = distinct A partition by org.apache.pig.Identity; " +
+                       "B = distinct A partition by org.apache.pig.test.utils.SimpleCustomPartitioner; " +
                        "C = sample B 0.49; " +
                        "D = order C by $0, $1; " +
                        "E = load 'y' as (d1, d2); " +
@@ -1446,7 +1445,7 @@ public class TestMacroExpansion {
         
         String expected =
             "macro_mymacro_A_0 = load 'x' as (u:int, v:long, w:bytearray);\n" +
-            "macro_mymacro_B_0 = distinct macro_mymacro_A_0 partition BY org.apache.pig.Identity;\n" +
+            "macro_mymacro_B_0 = distinct macro_mymacro_A_0 partition BY org.apache.pig.test.utils.SimpleCustomPartitioner;\n" +
             "macro_mymacro_C_0 = sample macro_mymacro_B_0 0.49;\n" +
             "macro_mymacro_D_0 = order macro_mymacro_C_0 BY $0, $1;\n" +
             "macro_mymacro_E_0 = load 'y' as (d1, d2);\n" + 
@@ -2277,7 +2276,8 @@ public class TestMacroExpansion {
     private void verify(String s, String expected) throws Exception {
         createFile("myscript.pig", s);
 
-        String[] args = { "-Dpig.import.search.path=/tmp", "-x", "local", "-c", "myscript.pig" };
+        String mode = Util.getLocalTestMode().toString();
+        String[] args = { "-Dpig.import.search.path=/tmp", "-x", mode, "-c", "myscript.pig" };
         PigStats stats = PigRunner.run(args, null);
         
         if (!stats.isSuccessful()) {
@@ -2286,7 +2286,7 @@ public class TestMacroExpansion {
         
         assertTrue(stats.isSuccessful());
         
-        String[] args2 = { "-Dpig.import.search.path=/tmp", "-x", "local", "-r", "myscript.pig" };
+        String[] args2 = { "-Dpig.import.search.path=/tmp", "-x", mode, "-r", "myscript.pig" };
         PigRunner.run(args2, null);
         
         File f2 = new File("myscript.pig.expanded");
@@ -2316,7 +2316,7 @@ public class TestMacroExpansion {
         
         try {
             BufferedReader br = new BufferedReader(new StringReader(piglatin));
-            Grunt grunt = new Grunt(br, new PigContext(ExecType.LOCAL, new Properties()));
+            Grunt grunt = new Grunt(br, new PigContext(Util.getLocalTestMode(), new Properties()));
             
             PrintWriter w = new PrintWriter(new FileWriter(scriptFile));
             w.print(piglatin);
@@ -2348,7 +2348,7 @@ public class TestMacroExpansion {
         try {
             BufferedReader br = new BufferedReader(new StringReader(piglatin));
             DryRunGruntParser parser = new DryRunGruntParser(br, scriptFile,
-                    new PigContext(ExecType.LOCAL, new Properties()));
+                    new PigContext(Util.getLocalTestMode(), new Properties()));
 
             PrintWriter w = new PrintWriter(new FileWriter(scriptFile));
             w.print(piglatin);

Modified: pig/trunk/test/org/apache/pig/test/TestNullConstant.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestNullConstant.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestNullConstant.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestNullConstant.java Mon Dec  8 22:29:31 2014
@@ -27,7 +27,6 @@ import java.io.IOException;
 import java.util.Iterator;
 import java.util.Map;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.builtin.mock.Storage.Data;
@@ -43,7 +42,7 @@ public class TestNullConstant {
 
     @Before
     public void setUp() throws Exception{
-        pigServer = new PigServer(ExecType.LOCAL);
+        pigServer = new PigServer(Util.getLocalTestMode());
     }
 
     @Test

Modified: pig/trunk/test/org/apache/pig/test/TestOrderBy3.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestOrderBy3.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestOrderBy3.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestOrderBy3.java Mon Dec  8 22:29:31 2014
@@ -29,7 +29,6 @@ import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.DataType;
@@ -53,7 +52,7 @@ public class TestOrderBy3 {
 
         log.info("Setting up");
 
-        pigServer = new PigServer(ExecType.LOCAL);
+        pigServer = new PigServer(Util.getLocalTestMode());
         data = resetData(pigServer);
 
         Random r = new Random();

Modified: pig/trunk/test/org/apache/pig/test/TestPONegative.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPONegative.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPONegative.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPONegative.java Mon Dec  8 22:29:31 2014
@@ -25,7 +25,6 @@ import java.util.Iterator;
 import java.util.Properties;
 import java.util.Random;
 
-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.expressionOperators.PONegative;
@@ -293,7 +292,7 @@ public class TestPONegative {
 
     @Test
     public void testPONegType() throws Exception {
-        PigServer pig = new PigServer(ExecType.LOCAL, new Properties());
+        PigServer pig = new PigServer(Util.getLocalTestMode(), new Properties());
         File f = Util.createInputFile("tmp", "", new String[] {"a", "b", "c"});
         pig.registerQuery("a = load '"
                 + Util.encodeEscape(Util.generateURI(f.toString(), pig.getPigContext())) + "';");

Modified: pig/trunk/test/org/apache/pig/test/TestParser.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestParser.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestParser.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestParser.java Mon Dec  8 22:29:31 2014
@@ -16,7 +16,6 @@
 
 package org.apache.pig.test;
 
-import static org.apache.pig.ExecType.LOCAL;
 import static org.apache.pig.builtin.mock.Storage.resetData;
 import static org.apache.pig.builtin.mock.Storage.tuple;
 import static org.junit.Assert.assertEquals;
@@ -32,7 +31,6 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.conf.Configuration;
 import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
-import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
 import org.apache.pig.builtin.mock.Storage;
@@ -64,17 +62,17 @@ public class TestParser {
             cluster.shutDown();
     }
 
-    public void setUp(ExecType execType) throws ExecException{
+    public void setUp(ExecType execType) throws Exception{
         Util.resetStateForExecModeSwitch();
         if (execType == cluster.getExecType()) {
             pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
         } else {
-            pigServer = new PigServer(LOCAL);
+            pigServer = new PigServer(Util.getLocalTestMode());
         }
     }
 
     @Test(expected = IOException.class)
-    public void testLoadingNonexistentFile() throws ExecException, IOException {
+    public void testLoadingNonexistentFile() throws Exception {
         for (ExecType execType : execTypes) {
             setUp(execType);
             pigServer.registerQuery("vals = load 'nonexistentfile';");
@@ -83,7 +81,7 @@ public class TestParser {
     }
 
     @Test
-    public void testRemoteServerList() throws ExecException, IOException {
+    public void testRemoteServerList() throws Exception {
         for (ExecType execType : execTypes) {
             setUp(execType);
             Properties pigProperties = pigServer.getPigContext().getProperties();
@@ -129,8 +127,8 @@ public class TestParser {
     }
 
     @Test
-    public void testRemoteServerList2() throws ExecException, IOException {
-        pigServer = new PigServer(LOCAL);
+    public void testRemoteServerList2() throws Exception {
+        pigServer = new PigServer(Util.getLocalTestMode());
         Properties pigProperties = pigServer.getPigContext().getProperties();
         pigProperties.setProperty("fs.default.name", "hdfs://a.com:8020");
         Configuration conf;
@@ -175,7 +173,7 @@ public class TestParser {
 
     @Test
     public void testRestrictedColumnNamesWhitelist() throws Exception {
-        pigServer = new PigServer(LOCAL);
+        pigServer = new PigServer(Util.getLocalTestMode());
         Data data = resetData(pigServer);
 
         Set<Tuple> tuples = Sets.newHashSet(tuple(1),tuple(2),tuple(3));

Modified: pig/trunk/test/org/apache/pig/test/TestPigContext.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPigContext.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigContext.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPigContext.java Mon Dec  8 22:29:31 2014
@@ -28,18 +28,30 @@ import java.util.Properties;
 import java.util.Random;
 
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
 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.test.junit.OrderedJUnit4Runner;
+import org.apache.pig.test.junit.OrderedJUnit4Runner.TestOrder;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
 
+//Need to run testImportList first due to TEZ-1802
+@RunWith(OrderedJUnit4Runner.class)
+@TestOrder({
+    "testImportList",
+    "testScriptFiles",
+    "testSetProperties_way_num01",
+    "testSetProperties_way_num02",
+    "testSetProperties_way_num03",
+    "testHadoopExceptionCreation"
+})
 public class TestPigContext {
     private static final String TMP_DIR_PROP = "/tmp/hadoop-hadoop";
     private static final String FS_NAME = "file:///";
@@ -60,7 +72,7 @@ public class TestPigContext {
     @Before
     public void setUp() throws Exception {
         Util.resetStateForExecModeSwitch();
-        pigContext = new PigContext(ExecType.LOCAL, getProperties());
+        pigContext = new PigContext(Util.getLocalTestMode(), getProperties());
         input = File.createTempFile("PigContextTest-", ".txt");
     }
 
@@ -85,7 +97,7 @@ public class TestPigContext {
      */
     @Test
     public void testSetProperties_way_num02() throws Exception {
-        PigServer pigServer = new PigServer(ExecType.LOCAL, getProperties());
+        PigServer pigServer = new PigServer(Util.getLocalTestMode(), getProperties());
         registerAndStore(pigServer);
 
         check_asserts(pigServer);
@@ -204,7 +216,7 @@ public class TestPigContext {
     @SuppressWarnings("deprecation")
     @Test
     public void testScriptFiles() throws Exception {
-        PigContext pc = new PigContext(ExecType.LOCAL, getProperties());
+        PigContext pc = new PigContext(Util.getLocalTestMode(), getProperties());
         final int n = pc.scriptFiles.size();
         pc.addScriptFile("test/path-1824");
         assertEquals("test" + File.separator + "path-1824", pc.getScriptFiles().get("test/path-1824").toString());

Modified: pig/trunk/test/org/apache/pig/test/TestPigSplit.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPigSplit.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigSplit.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPigSplit.java Mon Dec  8 22:29:31 2014
@@ -16,7 +16,6 @@
 
 package org.apache.pig.test;
 
-import static org.apache.pig.ExecType.LOCAL;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -31,6 +30,7 @@ import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
 import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestPigSplit {
@@ -38,7 +38,7 @@ public class TestPigSplit {
     protected final Log log = LogFactory.getLog(getClass());
 
     private static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
-    private static ExecType[] execTypes = new ExecType[] { ExecType.LOCAL, cluster.getExecType() };
+    private static ExecType[] execTypes;
     protected PigServer pigServer;
 
     /**
@@ -46,6 +46,11 @@ public class TestPigSplit {
      */
     String inputFileName;
 
+    @BeforeClass
+    public static void oneTimeSetup() throws Exception {
+        execTypes = new ExecType[] { cluster.getExecType(), Util.getLocalTestMode() };
+    }
+
     @AfterClass
     public static void oneTimeTearDown() throws Exception {
         if (cluster != null)
@@ -57,14 +62,14 @@ public class TestPigSplit {
         if (execType == cluster.getExecType()) {
             pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
         } else {
-            pigServer = new PigServer(LOCAL);
+            pigServer = new PigServer(Util.getLocalTestMode());
         }
     }
 
     private void createInput(String[] data, ExecType execType) throws IOException {
         if (execType == cluster.getExecType()) {
             Util.createInputFile(cluster, inputFileName, data);
-        } else if (execType == ExecType.LOCAL) {
+        } else if (execType.isLocal()) {
             Util.createLocalInputFile(inputFileName, data);
         } else {
             throw new IOException("unknown exectype:" + execType.toString());
@@ -74,7 +79,7 @@ public class TestPigSplit {
     public void tearDown(ExecType execType) throws Exception {
         if (execType == cluster.getExecType()) {
             Util.deleteFile(cluster, inputFileName);
-        } else if (execType == ExecType.LOCAL) {
+        } else if (execType.isLocal()) {
             new File(inputFileName).delete();
         } else {
             throw new IOException("unknown exectype:" + execType.toString());

Modified: pig/trunk/test/org/apache/pig/test/TestPigStorage.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPigStorage.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigStorage.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPigStorage.java Mon Dec  8 22:29:31 2014
@@ -57,7 +57,6 @@ import org.codehaus.jackson.JsonGenerati
 import org.codehaus.jackson.map.JsonMappingException;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -66,8 +65,6 @@ import org.junit.Test;
 public class TestPigStorage  {
     private static PigServer pig;
     private static PigContext pigContext;
-    private static Properties properties;
-    private static MiniGenericCluster cluster;
     private static final String datadir = "build/test/tmpdata/";
 
     @Before
@@ -99,16 +96,9 @@ public class TestPigStorage  {
 
     @BeforeClass
     public static void oneTimeSetup() {
-        cluster = MiniGenericCluster.buildCluster();
-        properties = cluster.getProperties();
         pigContext = new PigContext(ExecType.LOCAL, new Properties());
     }
 
-    @AfterClass
-    public static void shutdown() {
-        cluster.shutDown();
-    }
-
     private static void assertAliasIs(String alias, List<Tuple> expectedResults)
             throws IOException {
         Iterator<Tuple> iter = pig.openIterator(alias);
@@ -122,6 +112,9 @@ public class TestPigStorage  {
     @Test
     public void testBlockBoundary() throws ExecException {
 
+        MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
+        Properties properties = cluster.getProperties();
+
         // This tests PigStorage loader with records exactly
         // on the boundary of the file blocks.
         Properties props = new Properties();
@@ -178,6 +171,7 @@ public class TestPigStorage  {
                 e.printStackTrace();
                 Assert.fail();
             }
+            cluster.shutDown();
         }
     }
 

Modified: pig/trunk/test/org/apache/pig/test/TestPoissonSampleLoader.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPoissonSampleLoader.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPoissonSampleLoader.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPoissonSampleLoader.java Mon Dec  8 22:29:31 2014
@@ -24,9 +24,7 @@ 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.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
 import org.apache.pig.data.Tuple;
 import org.junit.After;
@@ -38,8 +36,8 @@ public class TestPoissonSampleLoader {
 
     private PigServer pigServer;
 
-    public TestPoissonSampleLoader() throws ExecException, IOException {
-        pigServer = new PigServer(ExecType.LOCAL);
+    public TestPoissonSampleLoader() throws Exception {
+        pigServer = new PigServer(Util.getLocalTestMode());
         pigServer.getPigContext().getProperties()
                 .setProperty("pig.skewedjoin.reduce.maxtuple", "5");
         pigServer.getPigContext().getProperties()

Modified: pig/trunk/test/org/apache/pig/test/TestProject.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestProject.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestProject.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestProject.java Mon Dec  8 22:29:31 2014
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Random;
 
-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;
@@ -267,7 +266,7 @@ public class TestProject {
                 + "' as (s1:chararray, s2:chararray, extra:chararray);" +
                 "b = foreach a generate s1, s2, extra;";
 
-        PigServer ps = new PigServer(ExecType.LOCAL);
+        PigServer ps = new PigServer(Util.getLocalTestMode());
         Util.registerMultiLineQuery(ps, query);
         Iterator<Tuple> it = ps.openIterator("b");
         Tuple[] expectedResults = new Tuple[] {
@@ -291,7 +290,7 @@ public class TestProject {
                 "t:tuple(s1:chararray, s2:chararray, s3:chararray));" +
                 "b = foreach a generate t.(s2,s3);";
 
-        PigServer ps = new PigServer(ExecType.LOCAL);
+        PigServer ps = new PigServer(Util.getLocalTestMode());
         Util.registerMultiLineQuery(ps, query);
         Iterator<Tuple> it = ps.openIterator("b");
         Tuple[] expectedResults = new Tuple[] {
@@ -313,7 +312,7 @@ public class TestProject {
                 "b = group a all;" +
                 "c = foreach b generate flatten(a.($1, $2)),a.$2;";
 
-        PigServer ps = new PigServer(ExecType.LOCAL);
+        PigServer ps = new PigServer(Util.getLocalTestMode());
         Util.registerMultiLineQuery(ps, query);
         Iterator<Tuple> it = ps.openIterator("c");
         Tuple[] expectedResults = new Tuple[] {
@@ -345,7 +344,7 @@ public class TestProject {
                 "t:tuple(s1:chararray, s2:chararray));" +
                 "b = foreach a generate t.s1, t.s2;";
 
-        PigServer ps = new PigServer(ExecType.LOCAL);
+        PigServer ps = new PigServer(Util.getLocalTestMode());
         Util.registerMultiLineQuery(ps, query);
         Iterator<Tuple> it = ps.openIterator("b");
         Tuple[] expectedResults = new Tuple[] {

Modified: pig/trunk/test/org/apache/pig/test/TestProjectStarExpander.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestProjectStarExpander.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestProjectStarExpander.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestProjectStarExpander.java Mon Dec  8 22:29:31 2014
@@ -26,13 +26,11 @@ import java.io.PrintWriter;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.util.Utils;
-import org.apache.pig.parser.ParserException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -75,8 +73,8 @@ public class TestProjectStarExpander  {
 
 
     @Test
-    public void testProjectStarForeach() throws IOException, ParserException {
-        PigServer pig = new PigServer(ExecType.LOCAL);
+    public void testProjectStarForeach() throws Exception {
+        PigServer pig = new PigServer(Util.getLocalTestMode());
         
         //specifying the new aliases only for initial set of fields
         String query =
@@ -118,8 +116,8 @@ public class TestProjectStarExpander  {
      * @throws ParseException
      */
     @Test
-    public void testProjectStarMulti() throws IOException, ParserException {
-        PigServer pig = new PigServer(ExecType.LOCAL);
+    public void testProjectStarMulti() throws Exception {
+        PigServer pig = new PigServer(Util.getLocalTestMode());
         String query =
             "  l1 = load '" + INP_FILE_5FIELDS + "' as (a : int, b : int, c : int);"
             + "f = foreach l1 generate * as (aa, bb, cc), *;"

Modified: pig/trunk/test/org/apache/pig/test/TestRank1.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestRank1.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestRank1.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestRank1.java Mon Dec  8 22:29:31 2014
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.Tuple;
@@ -43,7 +42,7 @@ public class TestRank1 {
 
     @Before
     public void setUp() throws Exception {
-        pigServer = new PigServer(ExecType.LOCAL);
+        pigServer = new PigServer(Util.getLocalTestMode());
 
         data = resetData(pigServer);
         data.set("test01", tuple("A", 1, "N"), tuple("B", 2, "N"),

Modified: pig/trunk/test/org/apache/pig/test/TestRank2.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestRank2.java?rev=1643944&r1=1643943&r2=1643944&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestRank2.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestRank2.java Mon Dec  8 22:29:31 2014
@@ -42,7 +42,7 @@ public class TestRank2 {
 
     @Before
     public void setUp() throws Exception {
-        pigServer = new PigServer("local");
+        pigServer = new PigServer(Util.getLocalTestMode());
 
         data = resetData(pigServer);
         data.set("test01", tuple("A", 1, "N"), tuple("B", 2, "N"),