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 [2/2] - in /hadoop/pig/branches/branch-0.6: ./ test/org/apache/pig/test/ test/org/apache/pig/test/utils/

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMapReduce2.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMapReduce2.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMapReduce2.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMapReduce2.java Wed May 12 22:33:16 2010
@@ -23,17 +23,21 @@ import junit.framework.TestCase;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.test.utils.TestHelper;
+import org.junit.AfterClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
 import java.io.File;
 import java.io.IOException;
 import java.text.DecimalFormat;
 import java.util.Iterator;
 
+@RunWith(JUnit4.class)
 public class TestMapReduce2 extends TestCase {
 
     private String initString = "mapreduce";
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
 
     private PigServer pig ;
 
@@ -41,6 +45,10 @@ public class TestMapReduce2 extends Test
         pig = new PigServer(initString) ;
     }
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
 
     @Test
     public void testUnion1() throws Exception {
@@ -153,7 +161,9 @@ public class TestMapReduce2 extends Test
             }
 
         }
-        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/TestMergeJoin.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMergeJoin.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMergeJoin.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMergeJoin.java Wed May 12 22:33:16 2010
@@ -45,14 +45,18 @@ import org.apache.pig.impl.util.LogUtils
 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 TestMergeJoin {
 
     private static final String INPUT_FILE = "testMergeJoinInput.txt";
     private PigServer pigServer;
-    private MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniCluster cluster = MiniCluster.buildCluster();
 
     public TestMergeJoin() throws ExecException, IOException{
 
@@ -83,6 +87,10 @@ public class TestMergeJoin {
         Util.deleteFile(cluster, INPUT_FILE);
     }
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
     /**
      * Test method for {@link org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin#getNext(org.apache.pig.data.Tuple)}.
      */

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMultiQuery.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMultiQuery.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMultiQuery.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestMultiQuery.java Wed May 12 22:33:16 2010
@@ -67,9 +67,13 @@ import org.apache.pig.impl.util.LogUtils
 import org.apache.pig.tools.grunt.GruntParser;
 import org.apache.pig.tools.pigscript.parser.ParseException;
 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 TestMultiQuery extends TestCase {
 
     private static final MiniCluster cluster = MiniCluster.buildCluster();
@@ -88,6 +92,11 @@ public class TestMultiQuery extends Test
         myPig = null;
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testMultiQueryJiraPig1108() {
         

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestNullConstant.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestNullConstant.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestNullConstant.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestNullConstant.java Wed May 12 22:33:16 2010
@@ -30,16 +30,17 @@ import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.Tuple;
+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 TestNullConstant extends TestCase {
     
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     private PigServer pigServer;
 
     @Before
@@ -48,6 +49,11 @@ public class TestNullConstant extends Te
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testArithExpressions() throws IOException, ExecException {
         File input = Util.createInputFile("tmp", "", 

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestOrderBy.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestOrderBy.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestOrderBy.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestOrderBy.java Wed May 12 22:33:16 2010
@@ -25,17 +25,23 @@ import java.util.Iterator;
 
 import junit.framework.TestCase;
 
+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.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 
+@RunWith(JUnit4.class)
 public class TestOrderBy extends TestCase {
     private static final int DATALEN = 1024;
     private String[][] DATA = new String[2][DATALEN];
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     
     private PigServer pig;
     private File tmpFile;
@@ -49,7 +55,8 @@ public class TestOrderBy extends TestCas
         pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
     
-    protected void setUp() throws Exception {
+    @Before
+    public void setUp() throws Exception {
         tmpFile = File.createTempFile("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         for(int i = 0; i < DATALEN; i++) {
@@ -58,10 +65,16 @@ public class TestOrderBy extends TestCas
         ps.close();
     }
     
-    protected void tearDown() throws Exception {
+    @After
+    public void tearDown() throws Exception {
         tmpFile.delete();
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     private void verify(String query, boolean descending) throws Exception {
         pig.registerQuery(query);
         Iterator<Tuple> it = pig.openIterator("myid");

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPONegative.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPONegative.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPONegative.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPONegative.java Wed May 12 22:33:16 2010
@@ -34,18 +34,29 @@ import org.apache.pig.backend.hadoop.exe
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POProject;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.PONegative;
 import org.apache.pig.impl.plan.PlanException;
+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 TestPONegative extends TestCase {
     
     DataBag bag = BagFactory.getInstance().newDefaultBag();
     Random r = new Random();
     TupleFactory tf = TupleFactory.getInstance();
-    MiniCluster miniCluster = MiniCluster.buildCluster();
+    static MiniCluster miniCluster = MiniCluster.buildCluster();
     final int MAX = 10;
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        miniCluster.shutDown();
+    }
+    
+    @Test
     public void testPONegInt () throws PlanException, ExecException {
         for(int i = 0; i < MAX; i++) {
             Tuple t = tf.newTuple();
@@ -72,6 +83,7 @@ public class TestPONegative extends Test
         
     }
     
+    @Test    
     public void testPONegIntAndNull () throws PlanException, ExecException {
     	
         for(int i = 0; i < MAX; i++) {
@@ -114,6 +126,7 @@ public class TestPONegative extends Test
         
     }
     
+    @Test    
     public void testPONegLong () throws PlanException, ExecException {
         for(int i = 0; i < MAX; i++) {
             Tuple t = tf.newTuple();
@@ -140,6 +153,7 @@ public class TestPONegative extends Test
         
     }
     
+    @Test    
     public void testPONegLongAndNull () throws PlanException, ExecException {
         for(int i = 0; i < MAX; i++) {
             Tuple t = tf.newTuple();
@@ -182,6 +196,7 @@ public class TestPONegative extends Test
         
     }
     
+    @Test
     public void testPONegDouble() throws PlanException, ExecException {
         for(int i = 0; i < MAX; i++) {
             Tuple t = tf.newTuple();
@@ -209,7 +224,7 @@ public class TestPONegative extends Test
         
     }
   
-    
+    @Test
     public void testPONegDoubleAndNull() throws PlanException, ExecException {
         for(int i = 0; i < MAX; i++) {
             Tuple t = tf.newTuple();
@@ -251,6 +266,7 @@ public class TestPONegative extends Test
         
     }
 
+    @Test
     public void testPONegFloat() throws PlanException, ExecException {
         for(int i = 0; i < MAX; i++) {
             Tuple t = tf.newTuple();
@@ -277,7 +293,7 @@ public class TestPONegative extends Test
         
     }
     
- 
+    @Test
     public void testPONegFloatAndNull() throws PlanException, ExecException {
         for(int i = 0; i < MAX; i++) {
             Tuple t = tf.newTuple();
@@ -319,6 +335,7 @@ public class TestPONegative extends Test
         
     }
     
+    @Test
     public void testPONegType() throws Exception {
         PigServer pig = new PigServer(ExecType.MAPREDUCE, miniCluster.getProperties());
         File f = Util.createInputFile("tmp", "", new String[] {"a", "b", "c"});

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestParser.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestParser.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestParser.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestParser.java Wed May 12 22:33:16 2010
@@ -31,20 +31,25 @@ import org.apache.pig.PigServer;
 import org.apache.pig.ExecType;
 import org.apache.pig.backend.executionengine.ExecException;
 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 TestParser extends TestCase {
 
 protected final Log log = LogFactory.getLog(getClass());
     
     protected 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");
         if(execTypeString!=null && execTypeString.length()>0){
@@ -59,11 +64,17 @@ 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 {
+        if(cluster != null)
+            cluster.shutDown();
+    }
+    
+    @Test
     public void testLoadingNonexistentFile() throws ExecException, IOException {
         try {
             // FIXME : this should be tested in all modes

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPi.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPi.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPi.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPi.java Wed May 12 22:33:16 2010
@@ -29,8 +29,12 @@ import org.apache.pig.impl.io.FileLocali
 import org.apache.pig.backend.executionengine.ExecException;
 
 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;
@@ -45,6 +49,7 @@ import org.apache.hadoop.conf.Configurat
 /*
  * Testcase aimed at testing pig with large file sizes and filter and group functions
 */
+@RunWith(JUnit4.class)
 public class TestPi extends TestCase {
     
     private final Log log = LogFactory.getLog(getClass());
@@ -54,7 +59,7 @@ public class TestPi extends TestCase {
     
     private long total = ((defaultBlockSize >> 20) / 10) << 20;
     private int inCircle = 0;
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
 
     private long totalLength = 0, totalLengthTest = 0;
 
@@ -65,7 +70,7 @@ public class TestPi extends TestCase {
     
     @Override
     @Before
-    protected void setUp() throws Exception{
+    public void setUp() throws Exception{
 
         log.info("Generating test data...");
         log.info("Default block size = " + defaultBlockSize);
@@ -116,10 +121,15 @@ public class TestPi extends TestCase {
     
     @Override
     @After
-    protected void tearDown() throws Exception {
+    public void tearDown() throws Exception {
         
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testPi () throws Exception {
         

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigContext.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigContext.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigContext.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigContext.java Wed May 12 22:33:16 2010
@@ -40,9 +40,14 @@ import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+@RunWith(JUnit4.class)
 public class TestPigContext extends TestCase {
 
     private static final String TMP_DIR_PROP = "/tmp/hadoop-hadoop";
@@ -51,11 +56,16 @@ public class TestPigContext extends Test
 
     private File input;
     private PigContext pigContext;
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = null;
+    
+    @BeforeClass 
+    public static void oneTimeSetup(){
+        cluster = MiniCluster.buildCluster();
+    }
     
     @Before
     @Override
-    protected void setUp() throws Exception {
+    public void setUp() throws Exception {
         pigContext = new PigContext(ExecType.LOCAL, getProperties());
         input = File.createTempFile("PigContextTest-", ".txt");
     }
@@ -198,10 +208,16 @@ public class TestPigContext extends Test
 
     @After
     @Override
-    protected void tearDown() throws Exception {
+    public void tearDown() throws Exception {
         input.delete();
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
+    
     private static Properties getProperties() {
         Properties props = new Properties();
         props.put("mapred.job.tracker", JOB_TRACKER);
@@ -225,7 +241,9 @@ public class TestPigContext extends Test
         for (final String command : commands) {
             pigServer.registerQuery(command);
         }
-        pigServer.store("counts", input.getAbsolutePath() + ".out");
+        String outFile = input.getAbsolutePath() + ".out";
+        pigServer.store("counts", outFile);
+        Util.deleteFile(cluster, outFile);
     }
 
     private void check_asserts() {

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigServer.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigServer.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigServer.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigServer.java Wed May 12 22:33:16 2010
@@ -40,15 +40,19 @@ import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.After;
 import org.junit.Test;
-import junit.framework.TestCase;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+import junit.framework.TestCase;
 
+@RunWith(JUnit4.class)
 public class TestPigServer extends TestCase {
     private PigServer pig = null;
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     private File stdOutRedirectedFile;
 
     @Before
@@ -72,6 +76,10 @@ public class TestPigServer extends TestC
         stdOutRedirectedFile.delete();
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
     
     private final static String FILE_SEPARATOR = System.getProperty("file.separator");
     

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigStorage.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigStorage.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigStorage.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPigStorage.java Wed May 12 22:33:16 2010
@@ -59,6 +59,7 @@ public class TestPigStorage {
     @AfterClass
     public static void shutdown() {
         pigServer.shutdown();
+        cluster.shutDown();
     }
     
     @Test

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPoissonSampleLoader.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPoissonSampleLoader.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPoissonSampleLoader.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPoissonSampleLoader.java Wed May 12 22:33:16 2010
@@ -42,18 +42,22 @@ import org.apache.pig.impl.util.Pair;
 import org.apache.pig.impl.util.UDFContext;
 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;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.io.FileSpec;
 
 import java.util.Properties;
 
-
+@RunWith(JUnit4.class)
 public class TestPoissonSampleLoader extends TestCase{
     private static final String INPUT_FILE1 = "SkewedJoinInput1.txt";
 
     private PigServer pigServer;
-    private MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniCluster cluster = MiniCluster.buildCluster();
     
     public TestPoissonSampleLoader() throws ExecException, IOException{
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
@@ -69,6 +73,11 @@ public class TestPoissonSampleLoader ext
     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_FILE1));
@@ -96,7 +105,7 @@ public class TestPoissonSampleLoader ext
         Util.deleteFile(cluster, INPUT_FILE1);
     }
     
-    
+    @Test
     public void testComputeSamples() throws IOException{
  		FileSpec fs = new FileSpec(INPUT_FILE1, new FuncSpec(PigStorage.class.getName()));
   		

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestProject.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestProject.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestProject.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestProject.java Wed May 12 22:33:16 2010
@@ -38,8 +38,11 @@ import org.apache.pig.test.utils.GenPhyO
 import org.apache.pig.test.utils.GenRandomData;
 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;
 
 public class TestProject extends  junit.framework.TestCase {
     Random r;
@@ -62,7 +65,7 @@ public class TestProject extends  junit.
     @After
     public void tearDown() throws Exception {
     }
-
+   
     @Test
     public void testGetNext() throws ExecException, IOException {
     	t=tRandom;
@@ -299,6 +302,7 @@ public class TestProject extends  junit.
         while(it.hasNext()) {
             assertEquals(expectedResults[i++], it.next());
         }
+        cluster.shutDown();
     }
 
 }

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPruneColumn.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPruneColumn.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPruneColumn.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestPruneColumn.java Wed May 12 22:33:16 2010
@@ -40,13 +40,13 @@ import org.apache.pig.builtin.PigStorage
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.logicalLayer.optimizer.PruneColumns;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 import junit.framework.TestCase;
 
 public class TestPruneColumn extends TestCase {
-    //MiniCluster cluster = MiniCluster.buildCluster();
     private PigServer pigServer;
     File tmpFile1;
     File tmpFile2;
@@ -171,6 +171,22 @@ public class TestPruneColumn extends Tes
 
     }
     
+    @After
+    @Override
+    public void tearDown() throws Exception{
+        tmpFile1.delete();
+        tmpFile2.delete();
+        tmpFile3.delete();
+        tmpFile4.delete();
+        tmpFile5.delete();
+        tmpFile6.delete();
+        tmpFile7.delete();
+        tmpFile8.delete();
+        tmpFile9.delete();
+        tmpFile10.delete();
+        logFile.delete();
+    }
+    
     public boolean checkLogFileMessage(String[] messages)
     {
         BufferedReader reader = null;
@@ -1330,8 +1346,6 @@ public class TestPruneColumn extends Tes
         
         assertTrue(checkLogFileMessage(new String[]{"Columns pruned for A: $1, $2", 
             "No map keys pruned for A"}));
-        
-        intermediateFile.delete();
     }
     
     @Test
@@ -1365,7 +1379,6 @@ public class TestPruneColumn extends Tes
         assertTrue(checkLogFileMessage(new String[]{"Columns pruned for A: $1", 
             "No map keys pruned for A"}));
         
-        intermediateFile.delete();
     }
 
     

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestRelationToExprProject.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestRelationToExprProject.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestRelationToExprProject.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestRelationToExprProject.java Wed May 12 22:33:16 2010
@@ -33,8 +33,11 @@ import org.apache.pig.data.TupleFactory;
 import org.apache.pig.impl.logicalLayer.parser.ParseException;
 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;
 
@@ -55,9 +58,10 @@ import junit.framework.TestCase;
  * a = load 'baginp.txt' as (b:bag{t:tuple()}); b = foreach a generate $0; dump b;
  * will go through a regular project (without the following flag)
  */
+@RunWith(JUnit4.class)
 public class TestRelationToExprProject extends TestCase {
 
-    private MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniCluster cluster = MiniCluster.buildCluster();
     private PigServer pigServer;
     private static final String TEST_FILTER_COUNT3_INPUT="test/org/apache/pig/test/data/TestRelationToExprProjectInput.txt"; 
     
@@ -65,7 +69,7 @@ public class TestRelationToExprProject e
      * @see junit.framework.TestCase#setUp()
      */
     @Before
-    protected void setUp() throws Exception {
+    public void setUp() throws Exception {
         pigServer = new PigServer(MAPREDUCE, cluster.getProperties());
     }
     
@@ -73,10 +77,15 @@ public class TestRelationToExprProject e
      * @see junit.framework.TestCase#tearDown()
      */
     @After
-    protected void tearDown() throws Exception {
+    public void tearDown() throws Exception {
         pigServer.shutdown();
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     // based on the script provided in the jira issue:PIG-514
     // tests that when a filter inside a foreach filters away all tuples
     // for a group, an empty bag is still provided to udfs whose

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSampleOptimizer.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSampleOptimizer.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSampleOptimizer.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSampleOptimizer.java Wed May 12 22:33:16 2010
@@ -39,8 +39,12 @@ import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.logicalLayer.LogicalPlan;
 import org.apache.pig.test.utils.LogicalPlanTester;
+import org.junit.AfterClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
 
+@RunWith(JUnit4.class)
 public class TestSampleOptimizer {
 
     static PigContext pc;
@@ -53,6 +57,11 @@ public class TestSampleOptimizer {
         }
     }
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        MiniCluster.buildCluster().shutDown();
+    }
+    
     @Test
     public void testOptimizerFired() throws Exception{
 

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSchema.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSchema.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSchema.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSchema.java Wed May 12 22:33:16 2010
@@ -655,5 +655,6 @@ public class TestSchema extends TestCase
             Tuple t = it.next();
             assertEquals("{a: {f1: chararray,f2: int}}", t.get(0));
         }
+        cluster.shutDown();
     }
 }

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSecondarySort.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSecondarySort.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSecondarySort.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSecondarySort.java Wed May 12 22:33:16 2010
@@ -38,10 +38,15 @@ import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.logicalLayer.LogicalPlan;
 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;
 
+@RunWith(JUnit4.class)
 public class TestSecondarySort extends TestCase {
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     private PigServer pigServer;
 
     static PigContext pc;
@@ -54,6 +59,11 @@ public class TestSecondarySort extends T
         }
     }
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Before
     @Override
     public void setUp() throws Exception{
@@ -61,6 +71,7 @@ public class TestSecondarySort extends T
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
 
+    @Test
     public void testDistinctOptimization1() throws Exception{
         // Limit in the foreach plan
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -81,6 +92,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==1);
     }
     
+    @Test
     public void testDistinctOptimization2() throws Exception{
         // Distinct on one entire input 
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -100,6 +112,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==1);
     }
     
+    @Test
     public void testDistinctOptimization3() throws Exception{
         // Distinct on the prefix of main sort key
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -119,6 +132,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==1);
     }
     
+    @Test
     public void testDistinctOptimization4() throws Exception{
         // Distinct on secondary key again, should remove
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -138,6 +152,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==2);
     }
     
+    @Test
     public void testDistinctOptimization5() throws Exception{
         // Filter in foreach plan
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -157,6 +172,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==1);
     }
     
+    @Test
     public void testDistinctOptimization6() throws Exception{
         // group by * with no schema, and distinct key is not part of main key
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -176,6 +192,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==1);
     }
 
+    @Test
     public void testDistinctOptimization7() throws Exception{
         // group by * with no schema, distinct key is more specific than the main key
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -195,6 +212,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==1);
     }
     
+    @Test
     public void testDistinctOptimization8() throws Exception{
         // local arrange plan is an expression
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -214,6 +232,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==1);
     }
     
+    @Test
     public void testDistinctOptimization9() throws Exception{
         // local arrange plan is nested project
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -233,6 +252,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==1);
     }
     
+    @Test
     public void testSortOptimization1() throws Exception{
         // Sort on something other than the main key
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -252,6 +272,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==0);
     }
     
+    @Test
     public void testSortOptimization2() throws Exception{
         // Sort on the prefix of the main key
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -271,6 +292,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==0);
     }
     
+    @Test
     public void testSortOptimization3() throws Exception{
         // Sort on the main key prefix / non main key prefix mixed
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -290,6 +312,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==0);
     }
     
+    @Test
     public void testSortOptimization4() throws Exception{
         // Sort on the main key again
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -309,6 +332,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==0);
     }
     
+    @Test
     public void testSortOptimization5() throws Exception{
         // Sort on the two keys, we can only take off 1
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -328,6 +352,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==0);
     }
     
+    @Test
     public void testSortOptimization6() throws Exception{
         // Sort desc
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -347,6 +372,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==0);
     }
     
+    @Test
     public void testSortOptimization7() throws Exception{
         // Sort asc on 1st key, desc on 2nd key
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -367,6 +393,7 @@ public class TestSecondarySort extends T
     }
     
     // See PIG-1193
+    @Test
     public void testSortOptimization8() throws Exception{
         // Sort desc, used in UDF twice
         LogicalPlanTester planTester = new LogicalPlanTester() ;
@@ -386,6 +413,7 @@ public class TestSecondarySort extends T
         assertTrue(so.getDistinctChanged()==0);
     }
     
+    @Test
     public void testNestedDistinctEndToEnd1() throws Exception{
         File tmpFile1 = File.createTempFile("test", "txt");
         PrintStream ps1 = new PrintStream(new FileOutputStream(tmpFile1));
@@ -413,8 +441,11 @@ public class TestSecondarySort extends T
         assertTrue(iter.hasNext());
         assertTrue(iter.next().toString().equals("(2,1L)"));
         assertFalse(iter.hasNext());
+        tmpFile1.delete();
+        tmpFile2.delete();
     }
     
+    @Test
     public void testNestedDistinctEndToEnd2() throws Exception{
         File tmpFile1 = File.createTempFile("test", "txt");
         PrintStream ps1 = new PrintStream(new FileOutputStream(tmpFile1));
@@ -437,6 +468,7 @@ public class TestSecondarySort extends T
         assertFalse(iter.hasNext());
     }
     
+    @Test
     public void testNestedSortEndToEnd1() throws Exception{
         File tmpFile1 = File.createTempFile("test", "txt");
         PrintStream ps1 = new PrintStream(new FileOutputStream(tmpFile1));
@@ -457,8 +489,10 @@ public class TestSecondarySort extends T
         assertTrue(iter.hasNext());
         assertTrue(iter.next().toString().equals("(2,{(2,3,4)})"));
         assertFalse(iter.hasNext());
+        tmpFile1.delete();
     }
     
+    @Test
     public void testNestedSortEndToEnd2() throws Exception{
         File tmpFile1 = File.createTempFile("test", "txt");
         PrintStream ps1 = new PrintStream(new FileOutputStream(tmpFile1));
@@ -479,8 +513,10 @@ public class TestSecondarySort extends T
         assertTrue(iter.hasNext());
         assertTrue(iter.next().toString().equals("(2,{(2,3,4)})"));
         assertFalse(iter.hasNext());
+        tmpFile1.delete();
     }
     
+    @Test
     public void testNestedSortMultiQueryEndToEnd1() throws Exception{
         pigServer.setBatchOn();
         pigServer.registerQuery("a = load 'file:test/org/apache/pig/test/data/passwd'" +

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSkewedJoin.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSkewedJoin.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSkewedJoin.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSkewedJoin.java Wed May 12 22:33:16 2010
@@ -38,8 +38,13 @@ import org.apache.pig.data.DataBag;
 import org.apache.pig.data.Tuple;
 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 TestSkewedJoin extends TestCase{
     private static final String INPUT_FILE1 = "SkewedJoinInput1.txt";
     private static final String INPUT_FILE2 = "SkewedJoinInput2.txt";
@@ -50,7 +55,7 @@ public class TestSkewedJoin extends Test
     private static final String INPUT_FILE7 = "SkewedJoinInput7.txt";
     
     private PigServer pigServer;
-    private MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniCluster cluster = MiniCluster.buildCluster();
     
     public TestSkewedJoin() throws ExecException, IOException{
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
@@ -64,6 +69,11 @@ public class TestSkewedJoin extends Test
         createFiles();
     }
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     private void createFiles() throws IOException {
     	PrintWriter w = new PrintWriter(new FileWriter(INPUT_FILE1));
     	    	
@@ -167,6 +177,7 @@ public class TestSkewedJoin extends Test
         Util.deleteFile(cluster, INPUT_FILE7);
     }
     
+    @Test
     public void testSkewedJoinWithGroup() throws IOException{
         pigServer.registerQuery("A = LOAD '" + INPUT_FILE1 + "' as (id, name, n);");
         pigServer.registerQuery("B = LOAD '" + INPUT_FILE2 + "' as (id, name);");
@@ -192,8 +203,9 @@ public class TestSkewedJoin extends Test
         }
         Assert.assertTrue(dbfrj.size()>0 && dbshj.size()>0);
         Assert.assertEquals(true, TestHelper.compareBags(dbfrj, dbshj));
-    }      
+    }    
     
+    @Test
     public void testSkewedJoinWithNoProperties() throws IOException{
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
 
@@ -225,7 +237,8 @@ public class TestSkewedJoin extends Test
              fail(e.getMessage());
         }
     }
-
+    
+    @Test
     public void testSkewedJoinReducers() throws IOException{
         pigServer.registerQuery("A = LOAD '" + INPUT_FILE1 + "' as (id, name, n);");
         pigServer.registerQuery("B = LOAD '" + INPUT_FILE2 + "' as (id, name);");
@@ -245,6 +258,7 @@ public class TestSkewedJoin extends Test
         
     }
     
+    @Test
     public void testSkewedJoin3Way() throws IOException{
         pigServer.registerQuery("A = LOAD '" + INPUT_FILE1 + "' as (id, name, n);");
         pigServer.registerQuery("B = LOAD '" + INPUT_FILE2 + "' as (id, name);");
@@ -266,6 +280,7 @@ public class TestSkewedJoin extends Test
         fail("Should throw exception, do not support 3 way join");
     }       
 
+    @Test
     public void testSkewedJoinMapKey() throws IOException{
         pigServer.registerQuery("A = LOAD '" + INPUT_FILE4 + "' as (m:[]);");
         pigServer.registerQuery("B = LOAD '" + INPUT_FILE4 + "' as (n:[]);");
@@ -289,6 +304,7 @@ public class TestSkewedJoin extends Test
 	}
 
 
+    @Test 
     public void testSkewedJoinKeyPartition() throws IOException {
     	try{
     	     Util.deleteFile(cluster, "skewedjoin");
@@ -332,6 +348,7 @@ public class TestSkewedJoin extends Test
          assertTrue(fc > 3);
     }
     
+    @Test 
     public void testSkewedJoinNullKeys() throws IOException {
         pigServer.registerQuery("A = LOAD '" + INPUT_FILE5 + "' as (id,name);");
         pigServer.registerQuery("B = LOAD '" + INPUT_FILE5 + "' as (id,name);");
@@ -353,6 +370,7 @@ public class TestSkewedJoin extends Test
         return;
     }
     
+    @Test 
     public void testSkewedJoinOuter() throws IOException {
         pigServer.registerQuery("A = LOAD '" + INPUT_FILE5 + "' as (id,name);");
         pigServer.registerQuery("B = LOAD '" + INPUT_FILE5 + "' as (id,name);");
@@ -391,6 +409,7 @@ public class TestSkewedJoin extends Test
     }
     
     // pig 1048
+    @Test 
     public void testSkewedJoinOneValue() throws IOException {
         pigServer.registerQuery("A = LOAD '" + INPUT_FILE3 + "' as (id,name);");
         pigServer.registerQuery("B = LOAD '" + INPUT_FILE3 + "' as (id,name);");
@@ -422,6 +441,7 @@ public class TestSkewedJoin extends Test
        
     }
 
+    @Test 
     public void testSkewedJoinManyReducers() throws IOException {
         pigServer.getPigContext().getProperties().setProperty("pig.skewedjoin.reduce.maxtuple", "2");
         pigServer.registerQuery("A = LOAD '" + INPUT_FILE6 + "' as (id,name);");
@@ -452,6 +472,7 @@ public class TestSkewedJoin extends Test
     /* Test to check if the samplers sample different input files in the case of
      * serial successive joins
      */
+    @Test 
     public void testSuccessiveJoins() throws IOException {
         pigServer.registerQuery("A = LOAD '" + INPUT_FILE1 + "' as (id,name);");
         pigServer.registerQuery("B = LOAD '" + INPUT_FILE2 + "' as (id,name);");
@@ -480,6 +501,7 @@ public class TestSkewedJoin extends Test
         Assert.assertEquals(true, TestHelper.compareBags(dbfrj, dbrj));
     }
     
+    @Test 
     public void testMultiQuery() throws IOException {
         pigServer.registerQuery("A = LOAD '" + INPUT_FILE1 + "' as (id,name);");
         pigServer.registerQuery("B = FILTER A by id == 100;");

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSplitStore.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSplitStore.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSplitStore.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestSplitStore.java Wed May 12 22:33:16 2010
@@ -29,22 +29,27 @@ import org.apache.pig.backend.executione
 import org.apache.pig.impl.PigContext;
 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;
 
+@RunWith(JUnit4.class)
 public class TestSplitStore extends TestCase{
     private PigServer pig;
     private PigContext pigContext;
     private File tmpFile;
-    private MiniCluster cluster = MiniCluster.buildCluster();
+    private static MiniCluster cluster = MiniCluster.buildCluster();
     
     public TestSplitStore() throws ExecException, IOException{
         pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         pigContext = pig.getPigContext();
         int LOOP_SIZE = 20;
         tmpFile = File.createTempFile("test", "txt");
+        tmpFile.deleteOnExit();
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         for(int i = 1; i <= LOOP_SIZE; i++) {
             ps.println(i);
@@ -60,6 +65,11 @@ public class TestSplitStore extends Test
     public void tearDown() throws Exception {
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void test1() throws Exception{
         pig.registerQuery("A = LOAD '" + Util.generateURI(tmpFile.toString()) + "';");

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestStore.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestStore.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestStore.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestStore.java Wed May 12 22:33:16 2010
@@ -70,9 +70,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 TestStore extends junit.framework.TestCase {
     POStore st;
     FileSpec fSpec;
@@ -108,6 +112,11 @@ public class TestStore extends junit.fra
     public void tearDown() throws Exception {
     }
 
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     private PigStats store() throws Exception {
         PhysicalPlan pp = new PhysicalPlan();
         pp.add(proj);

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestStoreOld.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestStoreOld.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestStoreOld.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestStoreOld.java Wed May 12 22:33:16 2010
@@ -26,18 +26,30 @@ import org.apache.pig.PigServer;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 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;
 
+@RunWith(JUnit4.class)
 public class TestStoreOld extends TestCase {
 
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     private int LOOP_COUNT = 1024;
     
     String fileName;
     String tmpFile1, tmpFile2;
     PigServer pig;
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
+    @Test
     public void testSingleStore() throws Exception{
         pig.registerQuery("A = load " + fileName + ";");
         
@@ -55,6 +67,7 @@ public class TestStoreOld extends TestCa
         }
     }
     
+    @Test
     public void testMultipleStore() throws Exception{
         pig.registerQuery("A = load " + fileName + ";");
         
@@ -76,6 +89,7 @@ public class TestStoreOld extends TestCa
         
     }
     
+    @Test
     public void testStoreWithMultipleMRJobs() throws Exception{
         pig.registerQuery("A = load " + fileName + ";");        
         pig.registerQuery("B = foreach (group A by $0) generate $0, SUM($1.$0);");
@@ -98,7 +112,8 @@ public class TestStoreOld extends TestCa
 
     
     @Override
-    protected void setUp() throws Exception {
+    @Before
+    public void setUp() throws Exception {
         super.setUp();
         File f = File.createTempFile("tmp", "");
         PrintWriter pw = new PrintWriter(f);

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUDFContext.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUDFContext.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUDFContext.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUDFContext.java Wed May 12 22:33:16 2010
@@ -32,21 +32,30 @@ import org.apache.pig.impl.io.BufferedPo
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.util.UDFContext;
+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 TestUDFContext extends TestCase {
     
     static MiniCluster cluster = null;
     
     @Override 
-    protected void setUp() throws Exception {
+    @Before
+    public void setUp() throws Exception {
         cluster = MiniCluster.buildCluster();
     }
 
-
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testUDFContext() throws Exception {
         Util.createInputFile(cluster, "a.txt", new String[] { "dumb" });

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUDFReturnMap.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUDFReturnMap.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUDFReturnMap.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUDFReturnMap.java Wed May 12 22:33:16 2010
@@ -31,8 +31,14 @@ import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.test.utils.MyUDFReturnMap;
+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 TestUDFReturnMap extends TestCase {
 
 	static String[] ScriptStatement = {
@@ -44,7 +50,8 @@ public class TestUDFReturnMap extends Te
 	static MiniCluster cluster = MiniCluster.buildCluster();
 
 	@Override
-	protected void setUp() throws Exception {
+	@Before
+	public void setUp() throws Exception {
 		TempScriptFile = File.createTempFile("temp_jira_851", ".pig");
 		FileWriter writer = new FileWriter(TempScriptFile);
 		for (String line : ScriptStatement) {
@@ -53,6 +60,11 @@ public class TestUDFReturnMap extends Te
 		writer.close();
 	}
 
+	@AfterClass
+	public static void oneTimeTearDown() throws Exception {
+	    cluster.shutDown();
+	}
+	
 	@Test
 	public void testUDFReturnMap_LocalMode() {
 		try {
@@ -100,7 +112,8 @@ public class TestUDFReturnMap extends Te
 	}
 
 	@Override
-	protected void tearDown() throws Exception {
+	@After
+	public void tearDown() throws Exception {
 		TempScriptFile.delete();
 	}
 }

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUTF8.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUTF8.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUTF8.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUTF8.java Wed May 12 22:33:16 2010
@@ -26,8 +26,11 @@ import java.io.PrintWriter;
 import java.io.StringWriter;
 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.pig.ExecType;
 import org.apache.pig.PigServer;
@@ -37,10 +40,10 @@ import org.apache.pig.tools.grunt.Grunt;
 import org.apache.pig.tools.parameters.ParameterSubstitutionPreprocessor;
 
 import junit.framework.TestCase;
-
+@RunWith(JUnit4.class)
 public class TestUTF8 extends TestCase {
     
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     private PigServer pigServer;
 
     @Before
@@ -49,6 +52,11 @@ public class TestUTF8 extends TestCase {
         pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     @Test
     public void testPigStorage() throws Exception{
         

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUnion.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUnion.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUnion.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/TestUnion.java Wed May 12 22:33:16 2010
@@ -47,8 +47,11 @@ import org.apache.pig.backend.hadoop.exe
 import org.apache.pig.test.utils.GenPhyOp;
 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;
 
 /**
  *  Start Plan - --4430968173902769765
@@ -76,10 +79,12 @@ import org.junit.Test;
  *  conversion from DataByteArray to native types for computation
  *  and back to DataByteArray for comparison with input.
  */
+
+@RunWith(JUnit4.class)
 public class TestUnion extends junit.framework.TestCase {
     POUnion sp;
     DataBag expBag;
-    MiniCluster cluster = MiniCluster.buildCluster();
+    static MiniCluster cluster = MiniCluster.buildCluster();
     PigContext pc = new PigContext();
     @Before
     public void setUp() throws Exception {
@@ -152,6 +157,11 @@ public class TestUnion extends junit.fra
     public void tearDown() throws Exception {
     }
     
+    @AfterClass
+    public static void oneTimeTearDown() throws Exception {
+        cluster.shutDown();
+    }
+    
     private Tuple castToDBA(Tuple in) throws ExecException{
         Tuple res = new DefaultTuple();
         for (int i=0;i<in.size();i++) {

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/Util.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/Util.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/Util.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/Util.java Wed May 12 22:33:16 2010
@@ -207,7 +207,7 @@ public class Util {
 			                           String[] inputData) 
 	throws IOException {
 		File f = File.createTempFile(tmpFilenamePrefix, tmpFilenameSuffix);
-        f.deleteOnExit();
+		f.deleteOnExit();
 		PrintWriter pw = new PrintWriter(new OutputStreamWriter(new FileOutputStream(f), "UTF-8"));
 		for (int i=0; i<inputData.length; i++){
 			pw.println(inputData[i]);
@@ -242,6 +242,21 @@ public class Util {
     }
     
     /**
+     * Helper to create an empty temp file on local file system
+     *  which will be deleted on exit
+     * @param prefix
+     * @param suffix
+     * @return File denoting a newly-created empty file 
+     * @throws IOException
+     */
+    static public File createTempFileDelOnExit(String prefix, String suffix)
+        throws IOException {
+        File tmpFile = File.createTempFile(prefix, suffix);
+        tmpFile.deleteOnExit();
+        return tmpFile;
+    }
+    
+    /**
      * Helper to return the file size on the MiniCluster dfs.
      * 
      * @param miniCluster reference to the Minicluster where the file should be created

Modified: hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/utils/GenPhyOp.java
URL: http://svn.apache.org/viewvc/hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/utils/GenPhyOp.java?rev=943728&r1=943727&r2=943728&view=diff
==============================================================================
--- hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/utils/GenPhyOp.java (original)
+++ hadoop/pig/branches/branch-0.6/test/org/apache/pig/test/utils/GenPhyOp.java Wed May 12 22:33:16 2010
@@ -60,7 +60,6 @@ public class GenPhyOp{
     };
     
     public static PigContext pc;
-//    public static MiniCluster cluster = MiniCluster.buildCluster();
     
     public static ConstantExpression exprConst() {
         ConstantExpression ret = new ConstantExpression(new OperatorKey("", r