PIG-5264: Remove deprecated keys from PigConfiguration (nkollar via rohini)

git-svn-id: https://svn.apache.org/repos/asf/pig/trunk@1803010 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/CHANGES.txt b/CHANGES.txt
index f1ac720..ac0aebc 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -24,6 +24,8 @@
  
 IMPROVEMENTS
 
+PIG-5264: Remove deprecated keys from PigConfiguration (nkollar via rohini)
+
 PIG-5157: Upgrade to Spark 2.0 (nkollar via liyunzhang)
 
 PIG-5237: Fix DOT file parsing to enable DOT-based physical plan testing (YaShock via szita)
diff --git a/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestLookupInFiles.java b/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestLookupInFiles.java
index f8e27a3..8d9a8ae 100644
--- a/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestLookupInFiles.java
+++ b/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/evaluation/string/TestLookupInFiles.java
@@ -28,20 +28,22 @@
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.data.Tuple;
-import org.apache.pig.test.MiniCluster;
+import org.apache.pig.test.MiniGenericCluster;
 import org.apache.pig.test.Util;
+import org.junit.Before;
 import org.junit.Test;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.assertTrue;
 
-public class TestLookupInFiles extends TestCase {
-    MiniCluster cluster = MiniCluster.buildCluster();
+public class TestLookupInFiles {
+    MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private PigServer pigServer;
    
-    @Override
+    @Before
     public void setUp() throws Exception{
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
+
     @Test
     public void testLookupInFiles() throws Exception {
         File tmpFile = File.createTempFile("test", ".txt");
diff --git a/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestCSVStorage.java b/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestCSVStorage.java
index 52485bf..0d4ab6a 100644
--- a/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestCSVStorage.java
+++ b/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestCSVStorage.java
@@ -31,7 +31,7 @@
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
-import org.apache.pig.test.MiniCluster;
+import org.apache.pig.test.MiniGenericCluster;
 import org.apache.pig.test.Util;
 import org.junit.Assert;
 import org.junit.Test;
@@ -40,10 +40,10 @@
     protected static final Log LOG = LogFactory.getLog(TestCSVStorage.class);
 
     private PigServer pigServer;
-    private MiniCluster cluster;
+    private MiniGenericCluster cluster;
 
     public TestCSVStorage() throws ExecException, IOException {
-        cluster = MiniCluster.buildCluster();
+        cluster = MiniGenericCluster.buildCluster();
         pigServer = new PigServer(ExecType.LOCAL, new Properties());
         pigServer.getPigContext().getProperties()
                 .setProperty(MRConfiguration.MAP_MAX_ATTEMPTS, "1");
diff --git a/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestDBStorage.java b/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestDBStorage.java
index 89d126c..5c6b8cc 100644
--- a/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestDBStorage.java
+++ b/contrib/piggybank/java/src/test/java/org/apache/pig/piggybank/test/storage/TestDBStorage.java
@@ -30,25 +30,28 @@
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.Date;
-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.backend.executionengine.ExecJob;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
-import org.apache.pig.test.MiniCluster;
+import org.apache.pig.test.MiniGenericCluster;
 import org.apache.pig.test.Util;
 import org.hsqldb.Server;
 import org.junit.After;
 import org.junit.Before;
 
-import junit.framework.TestCase;
+import org.junit.Test;
 
-public class TestDBStorage extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+
+public class TestDBStorage {
 
     private PigServer pigServer;
-    private MiniCluster cluster;
+    private MiniGenericCluster cluster;
     private Server dbServer;
     private String driver = "org.hsqldb.jdbcDriver";
     // private String url = "jdbc:hsqldb:mem:.";
@@ -63,7 +66,7 @@
 
     public TestDBStorage() throws ExecException, IOException {
         // Initialise Pig server
-        cluster = MiniCluster.buildCluster();
+        cluster = MiniGenericCluster.buildCluster();
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         pigServer.getPigContext().getProperties()
                 .setProperty(MRConfiguration.MAP_MAX_ATTEMPTS, "1");
@@ -155,6 +158,7 @@
         }
     }
 
+    @Test
     public void testWriteToDB() throws IOException {
         String insertQuery = "insert into ttt (id, name, ratio, dt) values (?,?,?,?)";
         pigServer.setBatchOn();
diff --git a/src/org/apache/pig/PigConfiguration.java b/src/org/apache/pig/PigConfiguration.java
index 4c29da1..ac32a5b 100644
--- a/src/org/apache/pig/PigConfiguration.java
+++ b/src/org/apache/pig/PigConfiguration.java
@@ -505,21 +505,11 @@
     public static final String PIG_SPARK_USE_NETTY_FILESERVER = "pig.spark.rpc.useNettyFileServer";
 
     /**
-     * @deprecated use {@link #PIG_LOG_TRACE_ID} instead. Will be removed in Pig 0.18
-     */
-    public static final String CALLER_ID = PIG_LOG_TRACE_ID;
-
-    /**
      * Enable ATS for Pig
      */
     public static final String PIG_ATS_ENABLED = "pig.ats.enabled";
 
     /**
-     * @deprecated use {@link #PIG_ATS_ENABLED} instead. Will be removed in Pig 0.18
-     */
-    public static final String ENABLE_ATS = PIG_ATS_ENABLED;
-
-    /**
      * If set, Pig will override tez.am.launch.cmd-opts and tez.am.resource.memory.mb to optimal
      */
     public static final String PIG_TEZ_CONFIGURE_AM_MEMORY = "pig.tez.configure.am.memory";
@@ -533,66 +523,4 @@
 
     public static final String PIG_PRINT_EXEC_PLAN = "pig.print.exec.plan";
 
-    // Deprecated settings of Pig 0.13
-
-    /**
-     * @deprecated use {@link #PIG_OPT_FETCH} instead. Will be removed in Pig 0.16
-     */
-    @Deprecated
-    public static final String OPT_FETCH = PIG_OPT_FETCH;
-
-    /**
-     * @deprecated use {@link #PIG_CACHEDBAG_MEMUSAGE} instead. Will be removed in Pig 0.16
-     */
-    @Deprecated
-    public static final String PROP_CACHEDBAG_MEMUSAGE = PIG_CACHEDBAG_MEMUSAGE;
-
-    /**
-     * @deprecated use {@link #PIG_EXEC_MAP_PARTAGG} instead. Will be removed in Pig 0.16
-     */
-    @Deprecated
-    public static final String PROP_EXEC_MAP_PARTAGG = PIG_EXEC_MAP_PARTAGG;
-
-    /**
-     * @deprecated use {@link #PIG_EXEC_MAP_PARTAGG_MINREDUCTION} instead. Will be removed in Pig 0.16
-     */
-    @Deprecated
-    public static final String PARTAGG_MINREDUCTION = PIG_EXEC_MAP_PARTAGG_MINREDUCTION;
-
-    /**
-     * @deprecated use {@link #PROP_NO_COMBINER1} instead. Will be removed in Pig 0.16
-     */
-    @Deprecated
-    public static final String PROP_NO_COMBINER = PIG_EXEC_NO_COMBINER;
-
-    @Deprecated
-    public static final String SHOULD_USE_SCHEMA_TUPLE = PIG_SCHEMA_TUPLE_ENABLED;
-
-    @Deprecated
-    public static final String SCHEMA_TUPLE_SHOULD_USE_IN_UDF = PIG_SCHEMA_TUPLE_USE_IN_UDF;
-
-    @Deprecated
-    public static final String SCHEMA_TUPLE_SHOULD_USE_IN_FOREACH = PIG_SCHEMA_TUPLE_USE_IN_FOREACH;
-
-    @Deprecated
-    public static final String SCHEMA_TUPLE_SHOULD_USE_IN_FRJOIN = PIG_SCHEMA_TUPLE_USE_IN_FRJOIN;
-
-    @Deprecated
-    public static final String SCHEMA_TUPLE_SHOULD_USE_IN_MERGEJOIN = PIG_SCHEMA_TUPLE_USE_IN_MERGEJOIN;
-
-    @Deprecated
-    public static final String SCHEMA_TUPLE_SHOULD_ALLOW_FORCE = PIG_SCHEMA_TUPLE_ALLOW_FORCE;
-
-    /**
-     * @deprecated use {@link #PIG_SCRIPT_INFO_ENABLED} instead. Will be removed in Pig 0.16
-     */
-    @Deprecated
-    public static final String INSERT_ENABLED = PIG_SCRIPT_INFO_ENABLED;
-
-    /**
-     * @deprecated use {@link #PIG_SCRIPT_MAX_SIZE} instead. Will be removed in Pig 0.16
-     */
-    @Deprecated
-    public static final String MAX_SCRIPT_SIZE = PIG_SCRIPT_MAX_SIZE;
-
 }
diff --git a/src/org/apache/pig/ResourceStatistics.java b/src/org/apache/pig/ResourceStatistics.java
index ffb6b12..9f83580 100644
--- a/src/org/apache/pig/ResourceStatistics.java
+++ b/src/org/apache/pig/ResourceStatistics.java
@@ -198,17 +198,6 @@
     }
 
     /**
-     * 
-     * @param mBytes
-     * @deprecated Use {@link ResourceStatistics#setSizeInBytes(Long)} instead
-     */
-    @Deprecated
-    public ResourceStatistics setmBytes(Long mBytes) {
-        this.bytes = mBytes * 1024 * 1024;
-        return this;
-    }
-    
-    /**
      * Sets the size in bytes
      * 
      * @param bytes
diff --git a/src/org/apache/pig/builtin/RoundRobinPartitioner.java b/src/org/apache/pig/builtin/RoundRobinPartitioner.java
deleted file mode 100644
index 775d7ea..0000000
--- a/src/org/apache/pig/builtin/RoundRobinPartitioner.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.pig.builtin;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.Partitioner;
-
-/**
- * This partitioner should be used with extreme caution and only in cases
- * where the order of output records is guaranteed to be same. If the order of
- * output records can vary on retries which is mostly the case, map reruns
- * due to shuffle fetch failures can lead to data being partitioned differently
- * and result in incorrect output due to loss or duplication of data.
- * Refer PIG-5041 for more details.
- *
- * This will be removed in the next release as it is risky to use in most cases.
- */
-@Deprecated
-public class RoundRobinPartitioner extends Partitioner<Writable, Writable>
-        implements Configurable {
-
-    /**
-     * Batch size for round robin partitioning. Batch size number of records
-     * will be distributed to each partition in a round robin fashion. Default
-     * value is 0 which distributes each record in a circular fashion. Higher
-     * number for batch size can be used to increase probability of keeping
-     * similar records in the same partition if output is already sorted and get
-     * better compression.
-     */
-    public static String PIG_ROUND_ROBIN_PARTITIONER_BATCH_SIZE = "pig.round.robin.partitioner.batch.size";
-    private int num = -1;
-    private int batchSize = 0;
-    private int currentBatchCount = 0;
-    private Configuration conf;
-
-    @Override
-    public int getPartition(Writable key, Writable value, int numPartitions) {
-        if (batchSize > 0) {
-            if (currentBatchCount == 0) {
-                num = ++num % numPartitions;
-            }
-            if (++currentBatchCount == batchSize) {
-                currentBatchCount = 0;
-            }
-        } else {
-            num = ++num % numPartitions;
-        }
-        return num;
-    }
-
-    @Override
-    public void setConf(Configuration conf) {
-        this.conf = conf;
-        batchSize = conf.getInt(PIG_ROUND_ROBIN_PARTITIONER_BATCH_SIZE, 0);
-    }
-
-    @Override
-    public Configuration getConf() {
-        return conf;
-    }
-
-}
diff --git a/src/org/apache/pig/impl/util/WrappedIOException.java b/src/org/apache/pig/impl/util/WrappedIOException.java
deleted file mode 100644
index 1c27b0b..0000000
--- a/src/org/apache/pig/impl/util/WrappedIOException.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.pig.impl.util;
-
-import java.io.IOException;
-
-import org.apache.pig.EvalFunc;
-import org.apache.pig.LoadFunc;
-
-/**
- * @deprecated This class was introduced to overcome the limitation that before 
- * Java 1.6, {@link IOException} did not have a constructor which took a 
- * {@link Throwable} argument. Since Pig code is now compiled with Java 1.6 and 
- * {@link EvalFunc} and {@link LoadFunc} user implementations should also use 
- * Java 1.6, they can use {@link IOException} instead. From Java 1.6, 
- * {@link IOException} has constructors which take a {@link Throwable}
- * argument.
- */
-@Deprecated 
-public class WrappedIOException {
-
-    public static IOException wrap(final Throwable e) {
-        return wrap(e.getMessage(), e);
-    }
-    
-    public static IOException wrap(final String message, final Throwable e) {
-        final IOException wrappedException = new IOException(message + " [" +
-            e.getMessage() + "]");
-        wrappedException.setStackTrace(e.getStackTrace());
-        wrappedException.initCause(e);
-        return wrappedException;
-    }
-}
diff --git a/src/org/apache/pig/tools/pigstats/PigStatsUtil.java b/src/org/apache/pig/tools/pigstats/PigStatsUtil.java
index 53a0df4..0715e54 100644
--- a/src/org/apache/pig/tools/pigstats/PigStatsUtil.java
+++ b/src/org/apache/pig/tools/pigstats/PigStatsUtil.java
@@ -24,7 +24,6 @@
 import java.util.regex.Pattern;
 
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
-import org.apache.pig.tools.pigstats.mapreduce.MRPigStatsUtil;
 import org.apache.pig.tools.pigstats.mapreduce.SimplePigStats;
 
 /**
@@ -60,20 +59,6 @@
             = "MultiStoreCounters";
 
     /**
-     * @deprecated use {@link org.apache.pig.tools.pigstats.mapreduce.MRPigStatsUtil#TASK_COUNTER_GROUP} instead.
-     */
-    @Deprecated
-    public static final String TASK_COUNTER_GROUP
-            = "org.apache.hadoop.mapred.Task$Counter";
-
-    /**
-     * @deprecated use {@link org.apache.pig.tools.pigstats.mapreduce.MRPigStatsUtil#FS_COUNTER_GROUP} instead.
-     */
-    @Deprecated
-    public static final String FS_COUNTER_GROUP
-            = MRPigStatsUtil.FS_COUNTER_GROUP;
-
-    /**
      * Returns an empty PigStats object Use of this method is not advised as it
      * will return the MR execution engine version of PigStats by default, and
      * is not necessarily empty depending on the timing.
diff --git a/test/org/apache/pig/test/MiniCluster.java b/test/org/apache/pig/test/MiniCluster.java
index e0dd428..a7532ad 100644
--- a/test/org/apache/pig/test/MiniCluster.java
+++ b/test/org/apache/pig/test/MiniCluster.java
@@ -48,15 +48,6 @@
     private Configuration m_dfs_conf = null;
     private Configuration m_mr_conf = null;
 
-    /**
-     * @deprecated use {@link org.apache.pig.test.MiniGenericCluster.buildCluster() instead.
-     */
-    @Deprecated
-    public static MiniCluster buildCluster() {
-        System.setProperty("test.exec.type", "mr");
-        return (MiniCluster)MiniGenericCluster.buildCluster("mr");
-    }
-
     @Override
     public ExecType getExecType() {
         return ExecType.MAPREDUCE;
diff --git a/test/org/apache/pig/test/TestAlgebraicInstantiation.java b/test/org/apache/pig/test/TestAlgebraicInstantiation.java
index 425d45b..9e68cd6 100644
--- a/test/org/apache/pig/test/TestAlgebraicInstantiation.java
+++ b/test/org/apache/pig/test/TestAlgebraicInstantiation.java
@@ -39,7 +39,7 @@
 public class TestAlgebraicInstantiation {
 
     Boolean[] nullFlags = new Boolean[]{ false, true};
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private PigServer pig;
     private File tmpFile;
     
diff --git a/test/org/apache/pig/test/TestAutoLocalMode.java b/test/org/apache/pig/test/TestAutoLocalMode.java
index b11a520..6dab274 100644
--- a/test/org/apache/pig/test/TestAutoLocalMode.java
+++ b/test/org/apache/pig/test/TestAutoLocalMode.java
@@ -42,7 +42,7 @@
 
 public class TestAutoLocalMode {
 
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private PigServer pigServer;
     private File logFile;
 
diff --git a/test/org/apache/pig/test/TestLargeFile.java b/test/org/apache/pig/test/TestLargeFile.java
index 9c1e2ce..c523c15 100644
--- a/test/org/apache/pig/test/TestLargeFile.java
+++ b/test/org/apache/pig/test/TestLargeFile.java
@@ -45,11 +45,11 @@
 
     File datFile;
 
-    private long defaultBlockSize = (new Configuration()).getLong("dfs.block.size", 0);
+    private long defaultBlockSize = (new Configuration()).getLong("dfs.blocksize", 0);
 
     private long total = defaultBlockSize >> 1;
     private int max_rand = 500;
-    static MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
 
     Integer[] COUNT = new Integer[max_rand];
 
diff --git a/test/org/apache/pig/test/TestMRCompiler.java b/test/org/apache/pig/test/TestMRCompiler.java
index f7d5884..5c2b76b 100644
--- a/test/org/apache/pig/test/TestMRCompiler.java
+++ b/test/org/apache/pig/test/TestMRCompiler.java
@@ -134,7 +134,7 @@
     "testStorerLimit",
     "testFetchOptimizerSideEffect"})
 public class TestMRCompiler {
-    static MiniCluster cluster;
+    static MiniGenericCluster cluster;
 
     static PigContext pc;
     static PigContext pcMR;
@@ -157,7 +157,7 @@
 
     @BeforeClass
     public static void setUpBeforeClass() throws Exception {
-        cluster = MiniCluster.buildCluster();
+        cluster = MiniGenericCluster.buildCluster();
         pc = new PigContext(ExecType.LOCAL, new Properties());
         pcMR = new PigContext(ExecType.MAPREDUCE, cluster.getProperties());
         pc.connect();
diff --git a/test/org/apache/pig/test/TestMultiQueryCompiler.java b/test/org/apache/pig/test/TestMultiQueryCompiler.java
index d846064..fec4440 100644
--- a/test/org/apache/pig/test/TestMultiQueryCompiler.java
+++ b/test/org/apache/pig/test/TestMultiQueryCompiler.java
@@ -40,7 +40,6 @@
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
 import org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil;
-import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.plan.Operator;
 import org.apache.pig.impl.plan.OperatorPlan;
@@ -57,16 +56,17 @@
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
+
 @RunWith(JUnit4.class)
 public class TestMultiQueryCompiler {
 
-    private static MiniCluster cluster;
+    private static MiniGenericCluster cluster;
 
     private PigServer myPig;
 
     @BeforeClass
     public static void setUpBeforeClass() throws IOException {
-        cluster = MiniCluster.buildCluster();
+        cluster = MiniGenericCluster.buildCluster();
         Util.copyFromLocalToCluster(cluster,
                 "test/org/apache/pig/test/data/passwd", "passwd");
         Util.copyFromLocalToCluster(cluster,
diff --git a/test/org/apache/pig/test/TestNumberOfReducers.java b/test/org/apache/pig/test/TestNumberOfReducers.java
index 53edcbf..079a8d8 100644
--- a/test/org/apache/pig/test/TestNumberOfReducers.java
+++ b/test/org/apache/pig/test/TestNumberOfReducers.java
@@ -25,7 +25,6 @@
 import org.apache.pig.PigRunner;
 import org.apache.pig.PigServer;
 import org.apache.pig.impl.PigContext;
-import org.apache.pig.tools.pigstats.JobStats;
 import org.apache.pig.tools.pigstats.PigStats;
 import org.apache.pig.tools.pigstats.mapreduce.MRJobStats;
 import org.junit.AfterClass;
@@ -34,7 +33,9 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 /**
  * This class tests whether the number of reducers is correctly set for queries
@@ -53,11 +54,11 @@
 
     static PigContext pc;
     static PigServer pigServer;
-    private static MiniCluster cluster;
+    private static MiniGenericCluster cluster;
 
     @BeforeClass
     public static void setUpBeforeClass() throws Exception {
-        cluster = MiniCluster.buildCluster();
+        cluster = MiniGenericCluster.buildCluster();
 
         Util.copyFromLocalToCluster(cluster, LOCAL_INPUT_FILE, HDFS_INPUT_FILE);
     }
diff --git a/test/org/apache/pig/test/TestOrderBy.java b/test/org/apache/pig/test/TestOrderBy.java
index a4e3d53..2985411 100644
--- a/test/org/apache/pig/test/TestOrderBy.java
+++ b/test/org/apache/pig/test/TestOrderBy.java
@@ -34,7 +34,6 @@
 import org.apache.pig.data.DataType;

 import org.apache.pig.data.Tuple;

 import org.joda.time.DateTime;

-import org.joda.time.DateTimeZone;

 import org.junit.After;

 import org.junit.AfterClass;

 import org.junit.Before;

@@ -43,7 +42,7 @@
 public class TestOrderBy {

     private static final int DATALEN = 1024;

     private String[][] DATA = new String[2][DATALEN];

-    static MiniCluster cluster = MiniCluster.buildCluster();

+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();

     

     private PigServer pig;

     private File tmpFile;

diff --git a/test/org/apache/pig/test/TestSampleOptimizer.java b/test/org/apache/pig/test/TestSampleOptimizer.java
index 14a8b36..0ad5d13 100644
--- a/test/org/apache/pig/test/TestSampleOptimizer.java
+++ b/test/org/apache/pig/test/TestSampleOptimizer.java
@@ -17,8 +17,6 @@
  */
 package org.apache.pig.test;
 
-import static org.junit.Assert.*;
-
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.PrintStream;
@@ -43,13 +41,18 @@
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 @RunWith(JUnit4.class)
 public class TestSampleOptimizer {
 
     static PigContext pc;
     static PigServer pigServer;
+    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     static{
-        pc = new PigContext(ExecType.MAPREDUCE,MiniCluster.buildCluster().getProperties());
+        pc = new PigContext(ExecType.MAPREDUCE, cluster.getProperties());
         try {
             pc.connect();
             pigServer = new PigServer( pc );
@@ -60,7 +63,7 @@
 
     @AfterClass
     public static void oneTimeTearDown() throws Exception {
-        MiniCluster.buildCluster().shutDown();
+        cluster.shutDown();
     }
     
     @Test