You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by th...@apache.org on 2010/05/13 00:33:18 UTC

svn commit: r943728 [1/2] - in /hadoop/pig/branches/branch-0.6: ./ test/org/apache/pig/test/ test/org/apache/pig/test/utils/

Author: thejas
Date: Wed May 12 22:33:16 2010
New Revision: 943728

URL: http://svn.apache.org/viewvc?rev=943728&view=rev
Log:
PIG-1391: pig unit tests leave behind files in temp directory because MiniCluster files don't get deleted (tejas)

Modified:
    hadoop/pig/branches/branch-0.6/CHANGES.txt
    hadoop/pig/branches/branch-0.6/build.xml
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/MiniCluster.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/PigExecTestCase.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAccumulator.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAlgebraicEval.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAlgebraicEvalLocal.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBZip.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBestFitCast.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBuiltin.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCollectedGroup.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCombiner.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCommit.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCompressedFiles.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCounters.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCustomSlicer.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestDataBagAccess.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipeline.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipeline2.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipelineLocal.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestExampleGenerator.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFRJoin.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterOpNumeric.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterOpString.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterUDF.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestForEachNestedPlan.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestForEachNestedPlanLocal.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestGrunt.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestHBaseStorage.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestInfixArithmetic.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestInputOutputFileValidator.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestJobSubmission.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestJoin.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestKeyTypeDiscoveryVisitor.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLargeFile.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLoad.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocal.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocal2.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocalJobSubmission.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocalPOSplit.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMRCompiler.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMapReduce.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMapReduce2.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMergeJoin.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMultiQuery.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestNullConstant.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestOrderBy.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPONegative.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestParser.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPi.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigContext.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigServer.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigStorage.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPoissonSampleLoader.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestProject.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPruneColumn.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestRelationToExprProject.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSampleOptimizer.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSchema.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSecondarySort.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSkewedJoin.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSplitStore.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestStore.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestStoreOld.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUDFContext.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUDFReturnMap.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUTF8.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUnion.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/Util.java
    hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/utils/GenPhyOp.java

Modified: hadoop/pig/branches/branch-0.6/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/CHANGES.txt?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/CHANGES.txt (original)
+++ hadoop/pig/branches/branch-0.6/CHANGES.txt Wed May 12 22:33:16 2010
@@ -156,6 +156,9 @@ PIG-922: Logical optimizer: push up proj
 
 BUG FIXES
 
+PIG-1391: pig unit tests leave behind files in temp directory because MiniCluster 
+files don't get deleted (tejas)
+
 PIG-1290: WeightedRangePartitioner should not check if input is empty if
 quantile file is empty (pradeepkth)
 

Modified: hadoop/pig/branches/branch-0.6/build.xml
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/build.xml?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/build.xml (original)
+++ hadoop/pig/branches/branch-0.6/build.xml Wed May 12 22:33:16 2010
@@ -495,9 +495,13 @@
       <sequential>
         <delete dir="${test.log.dir}"/>
         <mkdir dir="${test.log.dir}"/>
+        <tempfile property="junit.tmp.dir" prefix="pig_junit_tmp" destDir="${java.io.tmpdir}" />
+        <mkdir dir="${junit.tmp.dir}/"/>
+
         <junit showoutput="${test.output}" printsummary="yes" haltonfailure="no" fork="yes" maxmemory="256m" dir="${basedir}" timeout="${test.timeout}" errorProperty="tests.failed" failureProperty="tests.failed">
             <sysproperty key="ssh.gateway" value="${ssh.gateway}" />
             <sysproperty key="hod.server" value="${hod.server}" />
+            <sysproperty key="java.io.tmpdir" value="${junit.tmp.dir}" />
             <!-- <sysproperty key="hod.command" value="${hod.command}"/>
             			<sysproperty key="hod.param" value="${hod.param}"/> -->
             <sysproperty key="hadoop.log.dir" value="${test.log.dir}"/>
@@ -546,6 +550,7 @@
                 <fileset dir="test" includes="**/${testcase}.java"/>
             </batchtest>
         </junit>
+        <delete dir="${junit.tmp.dir}/"/>
         <fail if="tests.failed">Tests failed!</fail>
       </sequential>
     </macrodef>

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/MiniCluster.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/MiniCluster.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/MiniCluster.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/MiniCluster.java Wed May 12 22:33:16 2010
@@ -42,6 +42,7 @@ public class MiniCluster {
     private JobConf m_conf = null;
     
     private final static MiniCluster INSTANCE = new MiniCluster();
+    private static boolean isSetup = true;
     
     private MiniCluster() {
         setupMiniDfsAndMrClusters();
@@ -85,32 +86,57 @@ public class MiniCluster {
      * mapreduce cluster. 
      */
     public static MiniCluster buildCluster() {
+        if(! isSetup){
+            INSTANCE.setupMiniDfsAndMrClusters();
+            isSetup = true;
+        }
         return INSTANCE;
     }
+
+    public void shutDown(){
+        INSTANCE.shutdownMiniDfsAndMrClusters();
+    }
     
     protected void finalize() {
         shutdownMiniDfsAndMrClusters();
     }
     
     private void shutdownMiniDfsAndMrClusters() {
+        isSetup = false;
         try {
             if (m_fileSys != null) { m_fileSys.close(); }
         } catch (IOException e) {
             e.printStackTrace();
         }
         if (m_dfs != null) { m_dfs.shutdown(); }
-        if (m_mr != null) { m_mr.shutdown(); }        
-	}
+        if (m_mr != null) { m_mr.shutdown(); }     
+        m_fileSys = null;
+        m_dfs = null;
+        m_mr = null;
+    }
 
     public Properties getProperties() {
+        errorIfNotSetup();
         return ConfigurationUtil.toProperties(m_conf);
     }
 
     public void setProperty(String name, String value) {
+        errorIfNotSetup();
         m_conf.set(name, value);
     }
     
     public FileSystem getFileSystem() {
+        errorIfNotSetup();
         return m_fileSys;
     }
+    
+    /**
+     * Throw RunTimeException if isSetup is false
+     */
+    private void errorIfNotSetup(){
+        if(isSetup)
+            return;
+        String msg = "function called on MiniCluster that has been shutdown";
+        throw new RuntimeException(msg);
+    }
 }

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/PigExecTestCase.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/PigExecTestCase.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/PigExecTestCase.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/PigExecTestCase.java Wed May 12 22:33:16 2010
@@ -26,20 +26,24 @@ import org.apache.commons.logging.LogFac
 import org.apache.pig.PigServer;
 import org.apache.pig.ExecType;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+@RunWith(JUnit4.class)
 public abstract class PigExecTestCase extends TestCase {
 
     protected final Log log = LogFactory.getLog(getClass());
     
     protected ExecType execType = LOCAL;
     
-    private MiniCluster cluster;
+    private static MiniCluster cluster;
     protected PigServer pigServer;
     
     @Before
     @Override
-    protected void setUp() throws Exception {
+    public void setUp() throws Exception {
         
         String execTypeString = System.getProperty("test.exectype");
         if(execTypeString!=null && execTypeString.length()>0){
@@ -55,7 +59,13 @@ public abstract class PigExecTestCase ex
 
     @After
     @Override
-    protected void tearDown() throws Exception {
+    public void tearDown() throws Exception {
         pigServer.shutdown();
     }
+    
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        if(cluster != null)
+            cluster.shutDown();
+    }
 }

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAccumulator.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAccumulator.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAccumulator.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAccumulator.java Wed May 12 22:33:16 2010
@@ -27,15 +27,20 @@ import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.Tuple;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+@RunWith(JUnit4.class)
 public class TestAccumulator extends TestCase{
     private static final String INPUT_FILE = "AccumulatorInput.txt";
     private static final String INPUT_FILE2 = "AccumulatorInput2.txt";
     private static final String INPUT_FILE3 = "AccumulatorInput3.txt";
  
     private PigServer pigServer;
-    private MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniCluster cluster = MiniCluster.buildCluster();
     
     public TestAccumulator() throws ExecException, IOException{
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
@@ -49,6 +54,11 @@ public class TestAccumulator extends Tes
     public void setUp() throws Exception {
         createFiles();
     }
+    
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
 
     private void createFiles() throws IOException {
         PrintWriter w = new PrintWriter(new FileWriter(INPUT_FILE));
@@ -101,6 +111,7 @@ public class TestAccumulator extends Tes
         Util.deleteFile(cluster, INPUT_FILE3);
     }
     
+    @Test
     public void testAccumBasic() throws IOException{
         // test group by
         pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, fruit);");
@@ -159,6 +170,7 @@ public class TestAccumulator extends Tes
         }            
     }      
     
+    @Test
     public void testAccumWithNegative() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, fruit);");
         pigServer.registerQuery("B = group A by id;");
@@ -179,6 +191,7 @@ public class TestAccumulator extends Tes
         }            
     }
     
+    @Test
     public void testAccumWithAdd() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, fruit);");
         pigServer.registerQuery("B = group A by id;");
@@ -220,6 +233,7 @@ public class TestAccumulator extends Tes
         }
     }      
     
+    @Test
     public void testAccumWithMinus() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, fruit);");
         pigServer.registerQuery("B = group A by id;");
@@ -241,6 +255,7 @@ public class TestAccumulator extends Tes
         }                                   
     }              
     
+    @Test
     public void testAccumWithMod() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, fruit);");
         pigServer.registerQuery("B = group A by id;");
@@ -262,6 +277,7 @@ public class TestAccumulator extends Tes
         }                                   
     }             
     
+    @Test
     public void testAccumWithDivide() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, fruit);");
         pigServer.registerQuery("B = group A by id;");
@@ -283,6 +299,7 @@ public class TestAccumulator extends Tes
         }                                   
     }        
     
+    @Test
     public void testAccumWithAnd() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, fruit);");
         pigServer.registerQuery("B = group A by id;");
@@ -305,6 +322,7 @@ public class TestAccumulator extends Tes
         }                                   
     }          
     
+    @Test
     public void testAccumWithOr() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, fruit);");
         pigServer.registerQuery("B = group A by id;");
@@ -327,6 +345,7 @@ public class TestAccumulator extends Tes
         }                                   
     }  
     
+    @Test
     public void testAccumWithRegexp() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, fruit);");
         pigServer.registerQuery("B = group A by id;");
@@ -348,7 +367,7 @@ public class TestAccumulator extends Tes
         }                                   
     }              
     
-
+    @Test
     public void testAccumWithIsNull() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE2 + "' as (id:int, fruit);");
         pigServer.registerQuery("B = group A by id;");
@@ -368,6 +387,7 @@ public class TestAccumulator extends Tes
         }                                   
     }              
     
+    @Test
     public void testAccumWithDistinct() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, f);");
         pigServer.registerQuery("B = group A by id;");
@@ -387,6 +407,7 @@ public class TestAccumulator extends Tes
         }                                   
     }             
     
+    @Test
     public void testAccumWithSort() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, f);");
         pigServer.registerQuery("B = foreach A generate id, f, id as t;");
@@ -407,6 +428,7 @@ public class TestAccumulator extends Tes
         }                                   
     }
     
+    @Test
     public void testAccumWithBuildin() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE3 + "' as (id:int, v:double);");
         pigServer.registerQuery("C = group A by id;");
@@ -428,6 +450,7 @@ public class TestAccumulator extends Tes
         }    
     }
     
+    @Test
     public void testAccumWithMultiBuildin() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, c:chararray);");
         pigServer.registerQuery("C = group A by 1;");
@@ -443,6 +466,7 @@ public class TestAccumulator extends Tes
     }
 
 	// Pig 1105
+    @Test
     public void testAccumCountStar() throws IOException{
         pigServer.registerQuery("A = load '" + INPUT_FILE3 + "' as (id:int, v:double);");
         pigServer.registerQuery("C = group A by id;");

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAlgebraicEval.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAlgebraicEval.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAlgebraicEval.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAlgebraicEval.java Wed May 12 22:33:16 2010
@@ -25,8 +25,11 @@ import java.util.Random;
 
 import junit.framework.TestCase;
 
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
@@ -35,6 +38,7 @@ import org.apache.pig.data.DataByteArray
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 
+@RunWith(JUnit4.class)
 public class TestAlgebraicEval extends TestCase {
     
     private int LOOP_COUNT = 1024;
@@ -44,13 +48,18 @@ public class TestAlgebraicEval extends T
     
     @Before
     @Override
-    protected void setUp() throws Exception {
+    public void setUp() throws Exception {
         pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     Boolean[] nullFlags = new Boolean[]{ false, true};
 
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     @Test
     public void testGroupCountWithMultipleFields() throws Throwable {
         File tmpFile = File.createTempFile("test", "txt");

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAlgebraicEvalLocal.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAlgebraicEvalLocal.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAlgebraicEvalLocal.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestAlgebraicEvalLocal.java Wed May 12 22:33:16 2010
@@ -50,7 +50,6 @@ public class TestAlgebraicEvalLocal exte
     
     Boolean[] nullFlags = new Boolean[]{ false, true};
 
-    //MiniCluster cluster = MiniCluster.buildCluster();
     @Test
     public void testGroupCountWithMultipleFields() throws Throwable {
         File tmpFile = File.createTempFile("test", "txt");

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBZip.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBZip.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBZip.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBZip.java Wed May 12 22:33:16 2010
@@ -36,7 +36,6 @@ import org.apache.tools.bzip2r.CBZip2Out
 import org.junit.Test;
 
 public class TestBZip extends TestCase {
-    MiniCluster cluster = MiniCluster.buildCluster();
 
     /**
      * Tests the end-to-end writing and reading of a BZip file.

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBestFitCast.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBestFitCast.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBestFitCast.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBestFitCast.java Wed May 12 22:33:16 2010
@@ -40,21 +40,25 @@ import org.apache.pig.impl.logicalLayer.
 import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.util.LogUtils;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import junit.framework.TestCase;
 
+@RunWith(JUnit4.class)
 public class TestBestFitCast extends TestCase {
     private PigServer pigServer;
-    private MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniCluster cluster = MiniCluster.buildCluster();
     private File tmpFile, tmpFile2;
     int LOOP_SIZE = 20;
     
     public TestBestFitCast() throws ExecException, IOException{
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
 //        pigServer = new PigServer(ExecType.LOCAL);
-        tmpFile = File.createTempFile("test", "txt");
+        tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         long l = 0;
         for(int i = 1; i <= LOOP_SIZE; i++) {
@@ -62,7 +66,7 @@ public class TestBestFitCast extends Tes
         }
         ps.close();
         
-        tmpFile2 = File.createTempFile("test2", "txt");
+        tmpFile2 = Util.createTempFileDelOnExit("test2", "txt");
         ps = new PrintStream(new FileOutputStream(tmpFile2));
         l = 0;
         for(int i = 1; i <= LOOP_SIZE; i++) {
@@ -80,6 +84,11 @@ public class TestBestFitCast extends Tes
     public void tearDown() throws Exception {
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     public static class UDF1 extends EvalFunc<Tuple>{
         /**
          * java level API

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBuiltin.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBuiltin.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBuiltin.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestBuiltin.java Wed May 12 22:33:16 2010
@@ -48,7 +48,6 @@ public class TestBuiltin extends TestCas
     
     private String initString = "mapreduce";
     //private String initString = "local";
-    MiniCluster cluster = MiniCluster.buildCluster();
 
     TupleFactory tupleFactory = DefaultTupleFactory.getInstance();
     BagFactory bagFactory = DefaultBagFactory.getInstance();

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCollectedGroup.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCollectedGroup.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCollectedGroup.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCollectedGroup.java Wed May 12 22:33:16 2010
@@ -37,13 +37,18 @@ import org.apache.pig.backend.hadoop.exe
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
 import org.apache.pig.impl.plan.OperatorKey;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+@RunWith(JUnit4.class)
 public class TestCollectedGroup extends TestCase {
     private static final String INPUT_FILE = "MapSideGroupInput.txt";
     
     private PigServer pigServer;
-    private MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniCluster cluster = MiniCluster.buildCluster();
 
     public TestCollectedGroup() throws ExecException, IOException{
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
@@ -76,14 +81,21 @@ public class TestCollectedGroup extends 
         Util.deleteFile(cluster, INPUT_FILE);
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
+    @Test
     public void testPOMapsideGroupNoNullPlans() throws IOException {
         POCollectedGroup pmg = new POCollectedGroup(new OperatorKey());
         List<PhysicalPlan> plans = pmg.getPlans();
 
         Assert.assertTrue(plans != null);
         Assert.assertTrue(plans.size() == 0);
-    }      
-     
+    }     
+    
+    @Test  
     public void testMapsideGroupParserNoSupportForMultipleInputs() throws IOException {
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
 
@@ -99,6 +111,7 @@ public class TestCollectedGroup extends 
         }
     }
     
+    @Test
     public void testMapsideGroupParserNoSupportForGroupAll() throws IOException {
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
 
@@ -113,6 +126,7 @@ public class TestCollectedGroup extends 
         }
     }
      
+    @Test
     public void testMapsideGroupParserNoSupportForByExpression() throws IOException {
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
 
@@ -127,6 +141,7 @@ public class TestCollectedGroup extends 
         }
     }
 
+    @Test
     public void testMapsideGroupByOneColumn() throws IOException{
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
 
@@ -161,6 +176,7 @@ public class TestCollectedGroup extends 
         }
     }
  
+    @Test
     public void testMapsideGroupByMultipleColumns() throws IOException{
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
 
@@ -195,6 +211,7 @@ public class TestCollectedGroup extends 
         }
     }
   
+    @Test
     public void testMapsideGroupByStar() throws IOException{
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
 

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCombiner.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCombiner.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCombiner.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCombiner.java Wed May 12 22:33:16 2010
@@ -28,7 +28,10 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 
+import org.junit.AfterClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import junit.framework.Assert;
 import junit.framework.TestCase;
@@ -41,11 +44,17 @@ import org.apache.pig.data.Tuple;
 
 import org.apache.pig.impl.io.FileLocalizer;
 
-public class TestCombiner extends TestCase {
+@RunWith(JUnit4.class)
+public class TestCombiner extends PigExecTestCase {
 
     
 
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
+    
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
     
     @Test
     public void testOnCluster() throws Exception {

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCommit.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCommit.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCommit.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCommit.java Wed May 12 22:33:16 2010
@@ -33,8 +33,11 @@ import java.util.Map;
 import java.util.Random;
 import java.util.StringTokenizer;
 
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import org.apache.pig.ComparisonFunc;
 import org.apache.pig.EvalFunc;
@@ -56,9 +59,10 @@ import org.apache.pig.test.utils.Identit
 
 import junit.framework.TestCase;
 
+@RunWith(JUnit4.class)
 public class TestCommit extends TestCase {
     
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     private PigServer pigServer;
 
     TupleFactory mTf = TupleFactory.getInstance();
@@ -69,6 +73,11 @@ public class TestCommit extends TestCase
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testCheckin1() throws Exception{
         Tuple expected1 = mTf.newTuple(2);             

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCompressedFiles.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCompressedFiles.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCompressedFiles.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCompressedFiles.java Wed May 12 22:33:16 2010
@@ -24,8 +24,11 @@ import java.util.Iterator;
 import java.util.Random;
 import java.util.zip.GZIPOutputStream;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.pig.ExecType;
@@ -33,16 +36,17 @@ import org.apache.pig.PigServer;
 import org.apache.pig.builtin.DIFF;
 import junit.framework.TestCase;
 
+@RunWith(JUnit4.class)
 public class TestCompressedFiles extends TestCase {
     
     private final Log log = LogFactory.getLog(getClass());
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
 
     File datFile;
     File gzFile;
     @Override
     @Before
-    protected void setUp() throws Exception {
+    public void setUp() throws Exception {
         datFile = File.createTempFile("compTest", ".dat");
         gzFile = File.createTempFile("compTest", ".gz");
         FileOutputStream dat = new FileOutputStream(datFile);
@@ -66,11 +70,16 @@ public class TestCompressedFiles extends
 
     @Override
     @After
-    protected void tearDown() throws Exception {
+    public void tearDown() throws Exception {
         datFile.delete();
         gzFile.delete();
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testCompressed1() throws Throwable {
         PigServer pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCounters.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCounters.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCounters.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCounters.java Wed May 12 22:33:16 2010
@@ -34,16 +34,25 @@ import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.tools.pigstats.PigStats;
+import org.junit.AfterClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+@RunWith(JUnit4.class)
 public class TestCounters extends TestCase {
     String file = "input.txt";
 
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
 
     final int MAX = 100*1000;
     Random r = new Random();
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testMapOnly() throws IOException, ExecException {
         int count = 0;

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCustomSlicer.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCustomSlicer.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCustomSlicer.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestCustomSlicer.java Wed May 12 22:33:16 2010
@@ -31,21 +31,25 @@ import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.Tuple;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+@RunWith(JUnit4.class)
 public class TestCustomSlicer extends TestCase {
     
 protected final Log log = LogFactory.getLog(getClass());
     
     protected ExecType execType = ExecType.MAPREDUCE;
     
-    private MiniCluster cluster;
+    private static MiniCluster cluster;
     protected PigServer pigServer;
     
     @Before
     @Override
-    protected void setUp() throws Exception {
+    public void setUp() throws Exception {
         
         /*
         String execTypeString = System.getProperty("test.exectype");
@@ -63,10 +67,15 @@ protected final Log log = LogFactory.get
 
     @After
     @Override
-    protected void tearDown() throws Exception {
+    public void tearDown() throws Exception {
         pigServer.shutdown();
     }
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     /**
      * Uses RangeSlicer in place of pig's default Slicer to generate a few
      * values and count them.

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestDataBagAccess.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestDataBagAccess.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestDataBagAccess.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestDataBagAccess.java Wed May 12 22:33:16 2010
@@ -36,17 +36,21 @@ import org.apache.pig.data.NonSpillableD
 import org.apache.pig.data.SingleTupleBag;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.util.MultiMap;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import junit.framework.TestCase;
 
 /**
  *
  */
+@RunWith(JUnit4.class)
 public class TestDataBagAccess extends TestCase {
 
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     private PigServer pigServer;
 
     @Before
@@ -55,6 +59,11 @@ public class TestDataBagAccess extends T
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testSingleTupleBagAcess() throws Exception {
         Tuple inputTuple = new DefaultTuple();

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipeline.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipeline.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipeline.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipeline.java Wed May 12 22:33:16 2010
@@ -33,8 +33,11 @@ import java.util.Map;
 import java.util.Random;
 import java.util.StringTokenizer;
 
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import org.apache.pig.ComparisonFunc;
 import org.apache.pig.EvalFunc;
@@ -56,9 +59,10 @@ import org.apache.pig.test.utils.Identit
 
 import junit.framework.TestCase;
 
+@RunWith(JUnit4.class)
 public class TestEvalPipeline extends TestCase {
     
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     private PigServer pigServer;
 
     TupleFactory mTf = TupleFactory.getInstance();
@@ -72,6 +76,11 @@ public class TestEvalPipeline extends Te
 //        pigServer = new PigServer(ExecType.LOCAL);
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     static public class MyBagFunction extends EvalFunc<DataBag>{
         @Override
         public DataBag exec(Tuple input) throws IOException {
@@ -124,7 +133,7 @@ public class TestEvalPipeline extends Te
     
     @Test
     public void testDriverMethod() throws Exception{
-        File f = File.createTempFile("tmp", "");
+        File f = Util.createTempFileDelOnExit("tmp", "");
         PrintWriter pw = new PrintWriter(f);
         pw.println("a");
         pw.println("a");
@@ -160,7 +169,7 @@ public class TestEvalPipeline extends Te
         t.append(weights);
         b.add(t);
         
-        String fileName = "file:"+File.createTempFile("tmp", "");
+        String fileName = "file:"+Util.createTempFileDelOnExit("tmp", "");
         PigFile f = new PigFile(fileName);
         f.store(b, new BinStorage(), pigServer.getPigContext());
         
@@ -357,7 +366,7 @@ public class TestEvalPipeline extends Te
 
     private void testSortDistinct(boolean eliminateDuplicates, boolean useUDF) throws Exception{
         int LOOP_SIZE = 1024*16;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_SIZE; i++) {
@@ -398,7 +407,7 @@ public class TestEvalPipeline extends Te
     
     public void testNestedPlan() throws Exception{
         int LOOP_COUNT = 10;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {
@@ -439,7 +448,7 @@ public class TestEvalPipeline extends Te
 
     public void testNestedPlanWithExpressionAssignment() throws Exception{
         int LOOP_COUNT = 10;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {
@@ -485,7 +494,7 @@ public class TestEvalPipeline extends Te
 
     public void testLimit() throws Exception{
         int LOOP_COUNT = 20;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {
@@ -727,7 +736,7 @@ public class TestEvalPipeline extends Te
 
     public void testMapUDF() throws Exception{
         int LOOP_COUNT = 2;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {
@@ -768,7 +777,7 @@ public class TestEvalPipeline extends Te
 
     public void testMapUDFFail() throws Exception{
         int LOOP_COUNT = 2;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {
@@ -815,7 +824,7 @@ public class TestEvalPipeline extends Te
     @Test
     public void testNestedPlanForCloning() throws Exception{
         int LOOP_COUNT = 10;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {
@@ -859,7 +868,7 @@ public class TestEvalPipeline extends Te
     @Test
     public void testArithmeticCloning() throws Exception{
         int LOOP_COUNT = 10;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {
@@ -908,7 +917,7 @@ public class TestEvalPipeline extends Te
     @Test
     public void testExpressionReUse() throws Exception{
         int LOOP_COUNT = 10;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipeline2.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipeline2.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipeline2.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipeline2.java Wed May 12 22:33:16 2010
@@ -29,8 +29,11 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Random;
 
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import org.apache.pig.EvalFunc;
 import org.apache.pig.ExecType;
@@ -43,9 +46,10 @@ import org.apache.pig.builtin.BinStorage
 
 import junit.framework.TestCase;
 
+@RunWith(JUnit4.class)
 public class TestEvalPipeline2 extends TestCase {
     
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     private PigServer pigServer;
 
     TupleFactory mTf = TupleFactory.getInstance();
@@ -59,6 +63,11 @@ public class TestEvalPipeline2 extends T
 //        pigServer = new PigServer(ExecType.LOCAL);
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testUdfInputOrder() throws IOException {
         String[] input = {
@@ -92,7 +101,7 @@ public class TestEvalPipeline2 extends T
     @Test
     public void testUDFwithStarInput() throws Exception {
         int LOOP_COUNT = 10;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {
@@ -315,14 +324,14 @@ public class TestEvalPipeline2 extends T
             numRows++;
         }
         assertEquals(3, numRows);
-        
+        Util.deleteFile(cluster, "testPigStorageWithCtrlCharsInput.txt");
     }
 
     @Test
     // Test case added for PIG-850
     public void testLimitedSortWithDump() throws Exception{
         int LOOP_COUNT = 40;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random(1);
         int rand;
@@ -349,7 +358,7 @@ public class TestEvalPipeline2 extends T
     @Test
     public void testLimitAfterSort() throws Exception{
         int LOOP_COUNT = 40;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random(1);
         int rand;
@@ -380,7 +389,7 @@ public class TestEvalPipeline2 extends T
     @Test
     public void testLimitAfterSortDesc() throws Exception{
         int LOOP_COUNT = 40;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random(1);
         int rand;
@@ -411,7 +420,7 @@ public class TestEvalPipeline2 extends T
     @Test
     // See PIG-894
     public void testEmptySort() throws Exception{
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         pigServer.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "';");
         pigServer.registerQuery("B = order A by $0;");
         Iterator<Tuple> iter = pigServer.openIterator("B");
@@ -422,13 +431,13 @@ public class TestEvalPipeline2 extends T
     // See PIG-761
     @Test
     public void testLimitPOPackageAnnotator() throws Exception{
-        File tmpFile1 = File.createTempFile("test1", "txt");
+        File tmpFile1 = Util.createTempFileDelOnExit("test1", "txt");
         PrintStream ps1 = new PrintStream(new FileOutputStream(tmpFile1));
         ps1.println("1\t2\t3");
         ps1.println("2\t5\t2");
         ps1.close();
         
-        File tmpFile2 = File.createTempFile("test2", "txt");
+        File tmpFile2 = Util.createTempFileDelOnExit("test2", "txt");
         PrintStream ps2 = new PrintStream(new FileOutputStream(tmpFile2));
         ps2.println("1\t1");
         ps2.println("2\t2");

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipelineLocal.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipelineLocal.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipelineLocal.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestEvalPipelineLocal.java Wed May 12 22:33:16 2010
@@ -58,7 +58,6 @@ import org.junit.Test;
 
 public class TestEvalPipelineLocal extends TestCase {
     
-    //MiniCluster cluster = MiniCluster.buildCluster();
     private PigServer pigServer;
 
     TupleFactory mTf = TupleFactory.getInstance();
@@ -92,6 +91,7 @@ public class TestEvalPipelineLocal exten
             pw.println(data[i]);
         }
         pw.close();
+        f.deleteOnExit();
         return f;
     }
     
@@ -366,7 +366,7 @@ public class TestEvalPipelineLocal exten
 
     private void testSortDistinct(boolean eliminateDuplicates, boolean useUDF) throws Exception{
         int LOOP_SIZE = 1024*16;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_SIZE; i++) {
@@ -410,7 +410,7 @@ public class TestEvalPipelineLocal exten
     
     public void testNestedPlan() throws Exception{
         int LOOP_COUNT = 10;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {
@@ -451,7 +451,7 @@ public class TestEvalPipelineLocal exten
 
     public void testNestedPlanWithExpressionAssignment() throws Exception{
         int LOOP_COUNT = 10;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {
@@ -497,7 +497,7 @@ public class TestEvalPipelineLocal exten
 
     public void testLimit() throws Exception{
         int LOOP_COUNT = 20;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {
@@ -742,7 +742,7 @@ public class TestEvalPipelineLocal exten
 
     public void testMapUDF() throws Exception{
         int LOOP_COUNT = 2;
-        File tmpFile = File.createTempFile("test", "txt");
+        File tmpFile = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         Random r = new Random();
         for(int i = 0; i < LOOP_COUNT; i++) {

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestExampleGenerator.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestExampleGenerator.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestExampleGenerator.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestExampleGenerator.java Wed May 12 22:33:16 2010
@@ -33,12 +33,16 @@ import org.apache.pig.data.DataBag;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.logicalLayer.LogicalOperator;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+@RunWith(JUnit4.class)
 public class TestExampleGenerator extends TestCase {
 
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     PigContext pigContext = new PigContext(ExecType.MAPREDUCE, cluster
             .getProperties());
 
@@ -69,7 +73,15 @@ public class TestExampleGenerator extend
         B = "'" + FileLocalizer.hadoopify(fileB.toString(), pigContext) + "'";
         System.out.println("A : " + A + "\n" + "B : " + B);
         System.out.println("Test data created.");
+        fileA.deleteOnExit();
+        fileB.deleteOnExit();
+          
+    }
 
+    
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
     }
 
     private void writeData(File dataFile) throws Exception {

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFRJoin.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFRJoin.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFRJoin.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFRJoin.java Wed May 12 22:33:16 2010
@@ -54,14 +54,18 @@ import org.apache.pig.impl.logicalLayer.
 import org.apache.pig.impl.plan.OperatorKey;
 import org.apache.pig.test.utils.TestHelper;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+@RunWith(JUnit4.class)
 public class TestFRJoin extends TestCase{
     private static final String INPUT_FILE = "testFrJoinInput.txt";
     private static final String INPUT_FILE2 = "testFrJoinInput2.txt";
     private PigServer pigServer;
-    private MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniCluster cluster = MiniCluster.buildCluster();
     private File tmpFile;
     
     public TestFRJoin() throws ExecException, IOException{
@@ -92,6 +96,11 @@ public class TestFRJoin extends TestCase
         Util.createInputFile(cluster, INPUT_FILE2, input2);
     }
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @After
     public void tearDown() throws Exception {
         Util.deleteFile(cluster, INPUT_FILE);
@@ -160,6 +169,7 @@ public class TestFRJoin extends TestCase
         
     }
     
+    @Test
     public void testSortFRJoin() throws IOException{
       pigServer.registerQuery("A = LOAD '" + INPUT_FILE + "' as (x:int,y:int);");
       pigServer.registerQuery("B = LOAD '" + INPUT_FILE + "' as (x:int,y:int);");
@@ -186,6 +196,7 @@ public class TestFRJoin extends TestCase
       Assert.assertEquals(true, TestHelper.compareBags(dbfrj, dbshj));        
     }
     
+    @Test
     public void testDistinctFRJoin() throws IOException{
         pigServer.registerQuery("A = LOAD '" + INPUT_FILE + "' as (x:int,y:int);");
         pigServer.registerQuery("B = LOAD '" + INPUT_FILE + "' as (x:int,y:int);");

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterOpNumeric.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterOpNumeric.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterOpNumeric.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterOpNumeric.java Wed May 12 22:33:16 2010
@@ -22,8 +22,11 @@ import java.io.FileOutputStream;
 import java.io.PrintStream;
 import java.util.Iterator;
 
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -34,21 +37,26 @@ import org.apache.pig.data.Tuple;
 
 import junit.framework.TestCase;
 
+@RunWith(JUnit4.class)
 public class TestFilterOpNumeric extends TestCase {
 
     private final Log log = LogFactory.getLog(getClass());
 
     private static int LOOP_COUNT = 1024;
     private String initString = "mapreduce";
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     private PigServer pig;
     
     @Before
     @Override
-    protected void setUp() throws Exception {
+    public void setUp() throws Exception {
         pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
 
     @Test
     public void testNumericEq() throws Throwable {

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterOpString.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterOpString.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterOpString.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterOpString.java Wed May 12 22:33:16 2010
@@ -22,8 +22,11 @@ import java.io.FileOutputStream;
 import java.io.PrintStream;
 import java.util.Iterator;
 
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -36,22 +39,27 @@ import org.apache.pig.impl.io.FileLocali
 
 import junit.framework.TestCase;
 
+@RunWith(JUnit4.class)
 public class TestFilterOpString extends TestCase {
 
     private final Log log = LogFactory.getLog(getClass());
     private static int LOOP_COUNT = 1024;    
     private String initString = "mapreduce";
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
 
     private PigServer pig;
     
     @Before
     @Override
-    protected void setUp() throws Exception {
+    public void setUp() throws Exception {
         FileLocalizer.deleteTempFiles();
         pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
 
     @Test
     public void testStringEq() throws Throwable {

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterUDF.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterUDF.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterUDF.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestFilterUDF.java Wed May 12 22:33:16 2010
@@ -36,12 +36,16 @@ import org.apache.pig.data.*;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.test.utils.*;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+@RunWith(JUnit4.class)
 public class TestFilterUDF extends TestCase {
     private PigServer pigServer;
-    private MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniCluster cluster = MiniCluster.buildCluster();
     private File tmpFile;
     
     TupleFactory tf = TupleFactory.getInstance();
@@ -73,6 +77,11 @@ public class TestFilterUDF extends TestC
         tmpFile.delete();
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     private File createFile(String[] data) throws Exception{
         File f = File.createTempFile("tmp", "");
         PrintWriter pw = new PrintWriter(f);

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestForEachNestedPlan.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestForEachNestedPlan.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestForEachNestedPlan.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestForEachNestedPlan.java Wed May 12 22:33:16 2010
@@ -34,7 +34,6 @@ import java.text.DecimalFormat;
 public class TestForEachNestedPlan extends TestCase {
 
     private String initString = "mapreduce";
-    MiniCluster cluster = MiniCluster.buildCluster();
 
     private PigServer pig ;
 

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestForEachNestedPlanLocal.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestForEachNestedPlanLocal.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestForEachNestedPlanLocal.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestForEachNestedPlanLocal.java Wed May 12 22:33:16 2010
@@ -33,7 +33,6 @@ import java.text.DecimalFormat;
 public class TestForEachNestedPlanLocal extends TestCase {
 
     private String initString = "local";
-    //MiniCluster cluster = MiniCluster.buildCluster();
 
     private PigServer pig ;
 

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestGrunt.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestGrunt.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestGrunt.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestGrunt.java Wed May 12 22:33:16 2010
@@ -17,7 +17,12 @@
  */
 package org.apache.pig.test;
 
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
 import junit.framework.TestCase;
 import junit.framework.AssertionFailedError;
 
@@ -36,18 +41,24 @@ import java.io.ByteArrayInputStream;
 import java.io.InputStreamReader;
 import java.io.BufferedReader;
 
+@RunWith(JUnit4.class)
 public class TestGrunt extends TestCase {
-    MiniCluster cluster = MiniCluster.buildCluster();
-    private String basedir;
-
+    static MiniCluster cluster = MiniCluster.buildCluster();
+    private String basedir = "test/org/apache/pig/test/data";
     private final Log log = LogFactory.getLog(getClass());
 
-    public TestGrunt(String name) {
-        super(name);
+    @BeforeClass
+    public static void oneTimeSetup() throws Exception {
         cluster.setProperty("opt.multiquery","true");
-        basedir = "test/org/apache/pig/test/data";
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
+
+    
 /*    @Test 
     public void testCopyFromLocal() throws Throwable {
         PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestHBaseStorage.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestHBaseStorage.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestHBaseStorage.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestHBaseStorage.java Wed May 12 22:33:16 2010
@@ -46,16 +46,21 @@ import org.apache.pig.backend.hadoop.dat
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.io.FileLocalizer;
+import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 /** {@link org.apache.pig.backend.hadoop.hbase.HBaseStorage} Test Case **/
+@RunWith(JUnit4.class)
 public class TestHBaseStorage extends TestCase {
 
     private static final Log LOG =
         LogFactory.getLog(TestHBaseStorage.class);
     
-    private MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniCluster cluster = MiniCluster.buildCluster();
     private HBaseConfiguration conf;
     private MiniHBaseCluster hbaseCluster;
     private MiniZooKeeperCluster zooKeeperCluster;
@@ -76,7 +81,7 @@ public class TestHBaseStorage extends Te
     
     @Before
     @Override
-    protected void setUp() throws Exception {
+    public void setUp() throws Exception {
         super.setUp();
         
         conf = new HBaseConfiguration(ConfigurationUtil.
@@ -107,6 +112,11 @@ public class TestHBaseStorage extends Te
         pig = new PigServer(ExecType.MAPREDUCE, ConfigurationUtil.toProperties(conf));
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     /**
      * Actually start the MiniHBase instance.
      */
@@ -129,7 +139,8 @@ public class TestHBaseStorage extends Te
     }
 
     @Override
-    protected void tearDown() throws Exception {
+    @After
+    public void tearDown() throws Exception {
         // clear the table
         deleteTable();
         super.tearDown();

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestInfixArithmetic.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestInfixArithmetic.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestInfixArithmetic.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestInfixArithmetic.java Wed May 12 22:33:16 2010
@@ -23,8 +23,11 @@ import java.io.PrintStream;
 import java.util.Iterator;
 import java.util.Random;
 
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -38,21 +41,27 @@ import org.apache.pig.data.Tuple;
 
 import junit.framework.TestCase;
 
+@RunWith(JUnit4.class)
 public class TestInfixArithmetic extends TestCase {
 
     private final Log log = LogFactory.getLog(getClass());
 
     private static int LOOP_COUNT = 1024;    
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
 
     private PigServer pig;
     
     @Before
     @Override
-    protected void setUp() throws Exception {
+    public void setUp() throws Exception {
         pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     Boolean[] nullFlags = new Boolean[] { false, true };
 
     @Test

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestInputOutputFileValidator.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestInputOutputFileValidator.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestInputOutputFileValidator.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestInputOutputFileValidator.java Wed May 12 22:33:16 2010
@@ -36,13 +36,23 @@ import org.apache.pig.impl.plan.NodeIdGe
 import org.apache.pig.impl.logicalLayer.validators.* ;
 import org.apache.pig.impl.plan.CompilationMessageCollector;
 import org.apache.pig.impl.plan.CompilationMessageCollector.MessageType; 
+import org.junit.AfterClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
 import junit.framework.TestCase;
 
+@RunWith(JUnit4.class)
 public class TestInputOutputFileValidator extends TestCase {
     
     
-    private MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniCluster cluster = MiniCluster.buildCluster();
+    
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
     
     @Test
     public void testLocalModeInputPositive() throws Throwable {

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestJobSubmission.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestJobSubmission.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestJobSubmission.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestJobSubmission.java Wed May 12 22:33:16 2010
@@ -77,9 +77,13 @@ import org.apache.pig.test.utils.GenRand
 import org.apache.pig.test.utils.LogicalPlanTester;
 import org.apache.pig.test.utils.TestHelper;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+@RunWith(JUnit4.class)
 public class TestJobSubmission extends junit.framework.TestCase{
     
     
@@ -94,9 +98,10 @@ public class TestJobSubmission extends j
     String curDir;
     String inpDir;
     String golDir;
+    static MiniCluster cluster = MiniCluster.buildCluster();
     
     static {
-        MiniCluster cluster = MiniCluster.buildCluster();
+
         pc = new PigContext(ExecType.MAPREDUCE, cluster.getProperties());
         try {
             pc.connect();
@@ -122,6 +127,11 @@ public class TestJobSubmission extends j
     public void tearDown() throws Exception {
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
 /*    private void generateInput(int numTuples) throws ExecException{
         
         DataBag inpDb = GenRandomData.genRandSmallTupDataBag(r, numTuples, 1000);

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestJoin.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestJoin.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestJoin.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestJoin.java Wed May 12 22:33:16 2010
@@ -37,17 +37,22 @@ import org.apache.pig.impl.logicalLayer.
 import org.apache.pig.impl.util.LogUtils;
 import org.apache.pig.test.utils.Identity;
 import org.apache.pig.test.utils.LogicalPlanTester;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import junit.framework.TestCase;
 
 /**
  * Test cases to test join statement
  */
+
+@RunWith(JUnit4.class)
 public class TestJoin extends TestCase {
     
-    MiniCluster cluster;
+    static MiniCluster cluster;
     private PigServer pigServer;
 
     TupleFactory mTf = TupleFactory.getInstance();
@@ -60,6 +65,10 @@ public class TestJoin extends TestCase {
         FileLocalizer.setR(new Random());
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
     
     private void setUp(ExecType execType) throws ExecException {
         // cause a reinitialization of FileLocalizer's

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestKeyTypeDiscoveryVisitor.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestKeyTypeDiscoveryVisitor.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestKeyTypeDiscoveryVisitor.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestKeyTypeDiscoveryVisitor.java Wed May 12 22:33:16 2010
@@ -27,8 +27,11 @@ import org.apache.pig.data.BagFactory;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.data.TupleFactory;
 import org.apache.pig.impl.io.FileLocalizer;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import junit.framework.TestCase;
 
@@ -37,9 +40,10 @@ import junit.framework.TestCase;
  * is correctly determines for use when the key is null. In 
  * particular it tests KeyTypeDiscoveryVisitor
  */
+@RunWith(JUnit4.class)
 public class TestKeyTypeDiscoveryVisitor extends TestCase {
 
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     private PigServer pigServer;
 
     TupleFactory mTf = TupleFactory.getInstance();
@@ -49,6 +53,11 @@ public class TestKeyTypeDiscoveryVisitor
     public void setUp() throws Exception{
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
+   
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
     
     @Test
     public void testNullJoin() throws Exception {

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLargeFile.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLargeFile.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLargeFile.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLargeFile.java Wed May 12 22:33:16 2010
@@ -27,8 +27,12 @@ import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.io.FileLocalizer;
 
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
 import junit.framework.TestCase;
 
 import java.io.File;
@@ -44,6 +48,7 @@ import org.apache.hadoop.conf.Configurat
 //Order and Distinct functions are also tested.
 //This test would take a long time because of the large test files.
 
+@RunWith(JUnit4.class)
 public class TestLargeFile extends TestCase {
     
     File datFile;
@@ -53,7 +58,7 @@ public class TestLargeFile extends TestC
     private long total = defaultBlockSize >> 1;
     private int max_rand = 500;
 //    private double sum = 0.0, sumIn = 0.0;
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     
     Integer [] COUNT = new Integer[max_rand];
 
@@ -63,7 +68,7 @@ public class TestLargeFile extends TestC
     
     @Override
     @Before
-    protected void setUp() throws Exception{
+    public void setUp() throws Exception{
 
         System.out.println("Generating test data...");
         System.out.println("Default block size = " + defaultBlockSize);
@@ -105,12 +110,16 @@ public class TestLargeFile extends TestC
     
     @Override
     @After
-    protected void tearDown() throws Exception {
+    public void tearDown() throws Exception {
 
         
     }
 
-
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testLargeFile () throws Exception {
         System.out.println("Running testLargeFile...");

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLoad.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLoad.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLoad.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLoad.java Wed May 12 22:33:16 2010
@@ -53,9 +53,13 @@ import org.apache.pig.backend.datastorag
 import org.apache.pig.backend.datastorage.DataStorageException;
 import org.apache.pig.backend.datastorage.ElementDescriptor;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+@RunWith(JUnit4.class)
 public class TestLoad extends junit.framework.TestCase {
     FileSpec inpFSpec;
     POLoad ld;
@@ -101,6 +105,11 @@ public class TestLoad extends junit.fram
     public void tearDown() throws Exception {
     }
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testGetNextTuple() throws ExecException {
         Tuple t=null;

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocal.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocal.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocal.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocal.java Wed May 12 22:33:16 2010
@@ -66,7 +66,6 @@ public class TestLocal extends TestCase 
 
     private Log log = LogFactory.getLog(getClass());
     
-    //MiniCluster cluster = MiniCluster.buildCluster();
 
     private PigServer pig;
     
@@ -342,7 +341,7 @@ public class TestLocal extends TestCase 
         try {
             pig.deleteFile("frog");
         } catch(Exception e) {}
-
+        tmpFile.delete();
 
     }
 
@@ -393,7 +392,7 @@ public class TestLocal extends TestCase 
         try {
             pig.deleteFile("frog");
         } catch(Exception e) {}
-
+        tmpFile.delete();
 
     }
 
@@ -426,6 +425,7 @@ public class TestLocal extends TestCase 
         }
 
         assertEquals( MyStorage.COUNT, count );
+        tmpFile.delete();
     }
     
     @Test
@@ -460,6 +460,7 @@ public class TestLocal extends TestCase 
         }
 
         assertEquals( MyStorage.COUNT, count );
+        tmpFile.delete();
     }
     
 

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocal2.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocal2.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocal2.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocal2.java Wed May 12 22:33:16 2010
@@ -39,7 +39,6 @@ import java.util.Iterator;
 public class TestLocal2 extends TestCase {
 
     private String initString = "local";
-    //MiniCluster cluster = MiniCluster.buildCluster();
 
     private PigServer pig ;
 
@@ -114,6 +113,7 @@ public class TestLocal2 extends TestCase
         Iterator<Tuple> iter = pig.openIterator("C");
         // Before PIG-800 was fixed this went into an infinite loop, so just
         // managing to open the iterator is sufficient.
+        fp1.delete();
         
     }
     
@@ -133,6 +133,7 @@ public class TestLocal2 extends TestCase
         Iterator<Tuple> iter = pig.openIterator("C");
         // Before PIG-800 was fixed this went into an infinite loop, so just
         // managing to open the iterator is sufficient.
+        fp1.delete();
         
     }
     
@@ -174,6 +175,7 @@ public class TestLocal2 extends TestCase
         assertTrue(t.get(3).equals(new Integer(2)));
         
         assertTrue(!iter.hasNext());
+        fp1.delete();
     }
     
     
@@ -247,7 +249,9 @@ public class TestLocal2 extends TestCase
             }
 
         }
-        return TestHelper.createTempFile(data) ;
+        File tmpFile = TestHelper.createTempFile(data) ;
+        tmpFile.deleteOnExit();
+        return tmpFile;
     }
 
 

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocalJobSubmission.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocalJobSubmission.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocalJobSubmission.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocalJobSubmission.java Wed May 12 22:33:16 2010
@@ -69,7 +69,6 @@ public class TestLocalJobSubmission exte
     Random r = new Random();
     
     static {
-//        MiniCluster cluster = MiniCluster.buildCluster();
         pc = new PigContext();
         try {
             pc.connect();

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocalPOSplit.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocalPOSplit.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocalPOSplit.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestLocalPOSplit.java Wed May 12 22:33:16 2010
@@ -134,6 +134,7 @@ public class TestLocalPOSplit extends Te
                 assertEquals(t.get(0), firstValue);
             }
         }
+        datFile.delete();
     }
 
     @Test
@@ -141,20 +142,20 @@ public class TestLocalPOSplit extends Te
         init();
 
         pigContext.connect();
-        File datFile = File.createTempFile("tempN1", ".dat");
-        String path1 = Util.encodeEscape(datFile.getAbsolutePath());
+        File datFile1 = File.createTempFile("tempN1", ".dat");
+        String path1 = Util.encodeEscape(datFile1.getAbsolutePath());
 
-        FileOutputStream dat = new FileOutputStream(datFile);
+        FileOutputStream dat = new FileOutputStream(datFile1);
         String s1 = "1\n2\n3\n42\n4\n5\n";
         dat.write(s1.getBytes());
         dat.close();
 
         String s2 = "1\n2\n43\n3\n4\n5\n";
 
-        datFile = File.createTempFile("tempN2", ".dat");
-        String path2 = Util.encodeEscape(datFile.getAbsolutePath());
+        File datFile2 = File.createTempFile("tempN2", ".dat");
+        String path2 = Util.encodeEscape(datFile2.getAbsolutePath());
                 
-        dat = new FileOutputStream(datFile);
+        dat = new FileOutputStream(datFile2);
         dat.write(s2.getBytes());
         dat.close();
 
@@ -204,6 +205,8 @@ public class TestLocalPOSplit extends Te
         for (Integer j: seen.values()) {
             assertEquals(j, new Integer(2));
         }
+        datFile1.delete();
+        datFile2.delete();
     }
 
     public PhysicalPlan buildPhysicalPlan(LogicalPlan lp)

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMRCompiler.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMRCompiler.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMRCompiler.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMRCompiler.java Wed May 12 22:33:16 2010
@@ -71,7 +71,6 @@ import org.junit.Test;
  * Random will be different.
  */
 public class TestMRCompiler extends junit.framework.TestCase {
-//    MiniCluster cluster = MiniCluster.buildCluster();
     
     static PigContext pc;
 

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMapReduce.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMapReduce.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMapReduce.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMapReduce.java Wed May 12 22:33:16 2010
@@ -34,6 +34,7 @@ import java.util.Properties;
 
 import junit.framework.TestCase;
 
+import org.junit.AfterClass;
 import org.junit.Test;
 
 import org.apache.commons.logging.Log;
@@ -60,27 +61,35 @@ import org.apache.pig.impl.logicalLayer.
 import org.apache.pig.backend.datastorage.DataStorage;
 import org.apache.pig.backend.datastorage.ElementDescriptor;
 import org.junit.Before;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 import org.apache.pig.test.utils.TestHelper;
 
+@RunWith(JUnit4.class)
 public class TestMapReduce extends TestCase {
 
     private Log log = LogFactory.getLog(getClass());
     
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
 
     private PigServer pig;
     
     @Before
     @Override
-    protected void setUp() throws Exception {
+    public void setUp() throws Exception {
         pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testBigGroupAll() throws Throwable {
 
         int LOOP_COUNT = 4*1024;
-        File tmpFile = File.createTempFile( this.getName(), ".txt");
+        File tmpFile = File.createTempFile(this.getClass().getName(), ".txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         for(int i = 0; i < LOOP_COUNT; i++) {
             ps.println(i);
@@ -95,7 +104,7 @@ public class TestMapReduce extends TestC
     public void testBigGroupAllWithNull() throws Throwable {
 
         int LOOP_COUNT = 4*1024;
-        File tmpFile = File.createTempFile( this.getName(), ".txt");
+        File tmpFile = File.createTempFile(this.getClass().getName(), ".txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         long nonNullCnt = 0;
         for(int i = 0; i < LOOP_COUNT; i++) {
@@ -140,7 +149,6 @@ public class TestMapReduce extends TestC
         }
     }
     
-    @Test
     public Double bigGroupAll( File tmpFile ) throws Throwable {
 
         String query = "foreach (group (load '" + Util.generateURI(tmpFile.toString()) + "') all) generate " + COUNT.class.getName() + "($1) ;";
@@ -367,7 +375,7 @@ public class TestMapReduce extends TestC
         try {
             pig.deleteFile("frog");
         } catch(Exception e) {}
-
+        tmpFile.delete();
 
     }
 
@@ -418,7 +426,7 @@ public class TestMapReduce extends TestC
         try {
             pig.deleteFile("frog");
         } catch(Exception e) {}
-
+        tmpFile.delete();
 
     }
 
@@ -451,6 +459,7 @@ public class TestMapReduce extends TestC
         }
 
         assertEquals( MyStorage.COUNT, count );
+        tmpFile.delete();
     }
     
     @Test
@@ -485,6 +494,7 @@ public class TestMapReduce extends TestC
         }
 
         assertEquals( MyStorage.COUNT, count );
+        tmpFile.delete();
     }
     
 
@@ -533,7 +543,6 @@ public class TestMapReduce extends TestC
     }
 
 
-    @Test
     public void definedFunctions(String[][] data) throws Throwable {
 
         File tmpFile=TestHelper.createTempFile(data) ;