Return-Path: Delivered-To: apmail-hadoop-pig-commits-archive@www.apache.org Received: (qmail 1293 invoked from network); 7 May 2010 00:25:53 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 7 May 2010 00:25:53 -0000 Received: (qmail 77914 invoked by uid 500); 7 May 2010 00:25:53 -0000 Delivered-To: apmail-hadoop-pig-commits-archive@hadoop.apache.org Received: (qmail 77892 invoked by uid 500); 7 May 2010 00:25:53 -0000 Mailing-List: contact pig-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: pig-dev@hadoop.apache.org Delivered-To: mailing list pig-commits@hadoop.apache.org Received: (qmail 77885 invoked by uid 500); 7 May 2010 00:25:53 -0000 Delivered-To: apmail-incubator-pig-commits@incubator.apache.org Received: (qmail 77882 invoked by uid 99); 7 May 2010 00:25:53 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 07 May 2010 00:25:53 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 07 May 2010 00:25:48 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id B3EDC23888CD; Fri, 7 May 2010 00:24:58 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r941976 [1/3] - in /hadoop/pig/trunk: ./ test/org/apache/pig/test/ test/org/apache/pig/test/utils/ Date: Fri, 07 May 2010 00:24:57 -0000 To: pig-commits@incubator.apache.org From: thejas@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20100507002458.B3EDC23888CD@eris.apache.org> Author: thejas Date: Fri May 7 00:24:55 2010 New Revision: 941976 URL: http://svn.apache.org/viewvc?rev=941976&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/trunk/CHANGES.txt hadoop/pig/trunk/build.xml hadoop/pig/trunk/test/org/apache/pig/test/MiniCluster.java hadoop/pig/trunk/test/org/apache/pig/test/PigExecTestCase.java hadoop/pig/trunk/test/org/apache/pig/test/TestAccumulator.java hadoop/pig/trunk/test/org/apache/pig/test/TestAlgebraicEval.java hadoop/pig/trunk/test/org/apache/pig/test/TestAlgebraicEvalLocal.java hadoop/pig/trunk/test/org/apache/pig/test/TestBZip.java hadoop/pig/trunk/test/org/apache/pig/test/TestBatchAliases.java hadoop/pig/trunk/test/org/apache/pig/test/TestBestFitCast.java hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java hadoop/pig/trunk/test/org/apache/pig/test/TestCharArrayToNumeric.java hadoop/pig/trunk/test/org/apache/pig/test/TestCollectedGroup.java hadoop/pig/trunk/test/org/apache/pig/test/TestCombiner.java hadoop/pig/trunk/test/org/apache/pig/test/TestCommit.java hadoop/pig/trunk/test/org/apache/pig/test/TestCompressedFiles.java hadoop/pig/trunk/test/org/apache/pig/test/TestCounters.java hadoop/pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java hadoop/pig/trunk/test/org/apache/pig/test/TestEvalPipeline.java hadoop/pig/trunk/test/org/apache/pig/test/TestEvalPipeline2.java hadoop/pig/trunk/test/org/apache/pig/test/TestEvalPipelineLocal.java hadoop/pig/trunk/test/org/apache/pig/test/TestExampleGenerator.java hadoop/pig/trunk/test/org/apache/pig/test/TestFRJoin.java hadoop/pig/trunk/test/org/apache/pig/test/TestFilterOpNumeric.java hadoop/pig/trunk/test/org/apache/pig/test/TestFilterOpString.java hadoop/pig/trunk/test/org/apache/pig/test/TestFilterUDF.java hadoop/pig/trunk/test/org/apache/pig/test/TestFinish.java hadoop/pig/trunk/test/org/apache/pig/test/TestForEachNestedPlan.java hadoop/pig/trunk/test/org/apache/pig/test/TestForEachNestedPlanLocal.java hadoop/pig/trunk/test/org/apache/pig/test/TestGrunt.java hadoop/pig/trunk/test/org/apache/pig/test/TestHBaseStorage.java hadoop/pig/trunk/test/org/apache/pig/test/TestImplicitSplit.java hadoop/pig/trunk/test/org/apache/pig/test/TestInfixArithmetic.java hadoop/pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java hadoop/pig/trunk/test/org/apache/pig/test/TestJobSubmission.java hadoop/pig/trunk/test/org/apache/pig/test/TestJoin.java hadoop/pig/trunk/test/org/apache/pig/test/TestKeyTypeDiscoveryVisitor.java hadoop/pig/trunk/test/org/apache/pig/test/TestLargeFile.java hadoop/pig/trunk/test/org/apache/pig/test/TestLimitAdjuster.java hadoop/pig/trunk/test/org/apache/pig/test/TestLoad.java hadoop/pig/trunk/test/org/apache/pig/test/TestLocal.java hadoop/pig/trunk/test/org/apache/pig/test/TestLocal2.java hadoop/pig/trunk/test/org/apache/pig/test/TestLocalPOSplit.java hadoop/pig/trunk/test/org/apache/pig/test/TestLocalRearrange.java hadoop/pig/trunk/test/org/apache/pig/test/TestMapReduce.java hadoop/pig/trunk/test/org/apache/pig/test/TestMapReduce2.java hadoop/pig/trunk/test/org/apache/pig/test/TestMapSideCogroup.java hadoop/pig/trunk/test/org/apache/pig/test/TestMergeJoin.java hadoop/pig/trunk/test/org/apache/pig/test/TestMergeJoinOuter.java hadoop/pig/trunk/test/org/apache/pig/test/TestMultiQuery.java hadoop/pig/trunk/test/org/apache/pig/test/TestNullConstant.java hadoop/pig/trunk/test/org/apache/pig/test/TestOrderBy.java hadoop/pig/trunk/test/org/apache/pig/test/TestPONegative.java hadoop/pig/trunk/test/org/apache/pig/test/TestParser.java hadoop/pig/trunk/test/org/apache/pig/test/TestPi.java hadoop/pig/trunk/test/org/apache/pig/test/TestPigContext.java hadoop/pig/trunk/test/org/apache/pig/test/TestPigServer.java hadoop/pig/trunk/test/org/apache/pig/test/TestPigSplit.java hadoop/pig/trunk/test/org/apache/pig/test/TestPigStorage.java hadoop/pig/trunk/test/org/apache/pig/test/TestPoissonSampleLoader.java hadoop/pig/trunk/test/org/apache/pig/test/TestProject.java hadoop/pig/trunk/test/org/apache/pig/test/TestPruneColumn.java hadoop/pig/trunk/test/org/apache/pig/test/TestPruneColumnNewLogicalPlan.java hadoop/pig/trunk/test/org/apache/pig/test/TestRelationToExprProject.java hadoop/pig/trunk/test/org/apache/pig/test/TestSample.java hadoop/pig/trunk/test/org/apache/pig/test/TestSampleOptimizer.java hadoop/pig/trunk/test/org/apache/pig/test/TestSchema.java hadoop/pig/trunk/test/org/apache/pig/test/TestSecondarySort.java hadoop/pig/trunk/test/org/apache/pig/test/TestSkewedJoin.java hadoop/pig/trunk/test/org/apache/pig/test/TestSplitStore.java hadoop/pig/trunk/test/org/apache/pig/test/TestStore.java hadoop/pig/trunk/test/org/apache/pig/test/TestStoreOld.java hadoop/pig/trunk/test/org/apache/pig/test/TestStreaming.java hadoop/pig/trunk/test/org/apache/pig/test/TestUDFContext.java hadoop/pig/trunk/test/org/apache/pig/test/TestUDFReturnMap.java hadoop/pig/trunk/test/org/apache/pig/test/TestUTF8.java hadoop/pig/trunk/test/org/apache/pig/test/TestUnion.java hadoop/pig/trunk/test/org/apache/pig/test/Util.java hadoop/pig/trunk/test/org/apache/pig/test/utils/GenPhyOp.java Modified: hadoop/pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/CHANGES.txt?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/CHANGES.txt (original) +++ hadoop/pig/trunk/CHANGES.txt Fri May 7 00:24:55 2010 @@ -54,6 +54,9 @@ PIG-1309: Map-side Cogroup (ashutoshc) BUG FIXES +PIG-1391: pig unit tests leave behind files in temp directory because +MiniCluster files don't get deleted (tejas) + PIG-1211: Pig script runs half way after which it reports syntax error (pradeepkth) Modified: hadoop/pig/trunk/build.xml URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/build.xml?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/build.xml (original) +++ hadoop/pig/trunk/build.xml Fri May 7 00:24:55 2010 @@ -541,9 +541,13 @@ + + + + @@ -591,6 +595,7 @@ + Tests failed! Modified: hadoop/pig/trunk/test/org/apache/pig/test/MiniCluster.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/MiniCluster.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/MiniCluster.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/MiniCluster.java Fri May 7 00:24:55 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/trunk/test/org/apache/pig/test/PigExecTestCase.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/PigExecTestCase.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/PigExecTestCase.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/PigExecTestCase.java Fri May 7 00:24:55 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; - protected MiniCluster cluster; + 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/trunk/test/org/apache/pig/test/TestAccumulator.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestAccumulator.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestAccumulator.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestAccumulator.java Fri May 7 00:24:55 2010 @@ -27,8 +27,13 @@ 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"; @@ -36,8 +41,8 @@ public class TestAccumulator extends Tes private static final String INPUT_FILE4 = "AccumulatorInput4.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()); // pigServer = new PigServer(ExecType.LOCAL); @@ -54,6 +59,11 @@ public class TestAccumulator extends Tes pigServer.getPigContext().getProperties().remove("opt.accumulator"); createFiles(); } + + @AfterClass + public static void oneTimeTearDown() throws Exception { + cluster.shutDown(); + } private void createFiles() throws IOException { PrintWriter w = new PrintWriter(new FileWriter(INPUT_FILE)); @@ -120,6 +130,7 @@ public class TestAccumulator extends Tes Util.deleteFile(cluster, INPUT_FILE4); } + @Test public void testAccumBasic() throws IOException{ // test group by pigServer.registerQuery("A = load '" + INPUT_FILE + "' as (id:int, fruit);"); @@ -178,6 +189,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;"); @@ -198,6 +210,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;"); @@ -239,6 +252,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;"); @@ -260,6 +274,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;"); @@ -281,6 +296,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;"); @@ -302,6 +318,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;"); @@ -324,6 +341,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;"); @@ -346,6 +364,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;"); @@ -367,7 +386,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;"); @@ -387,6 +406,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;"); @@ -406,6 +426,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;"); @@ -458,6 +479,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;"); @@ -486,6 +508,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;"); @@ -501,6 +524,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;"); @@ -513,7 +537,7 @@ public class TestAccumulator extends Tes } } - + @Test public void testAccumulatorOff() throws IOException{ pigServer.getPigContext().getProperties().setProperty("opt.accumulator", "false"); @@ -533,8 +557,9 @@ public class TestAccumulator extends Tes // we should get exception } - } + } + @Test public void testAccumWithMap() throws IOException{ pigServer.registerQuery("A = load '" + INPUT_FILE4 + "' as (id, url);"); pigServer.registerQuery("B = group A by (id, url);"); Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestAlgebraicEval.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestAlgebraicEval.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestAlgebraicEval.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestAlgebraicEval.java Fri May 7 00:24:55 2010 @@ -30,9 +30,13 @@ import org.apache.pig.PigServer; import org.apache.pig.builtin.PigStorage; import org.apache.pig.data.DataType; 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 TestAlgebraicEval extends TestCase { private int LOOP_COUNT = 1024; @@ -41,13 +45,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/trunk/test/org/apache/pig/test/TestAlgebraicEvalLocal.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestAlgebraicEvalLocal.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestAlgebraicEvalLocal.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestAlgebraicEvalLocal.java Fri May 7 00:24:55 2010 @@ -47,7 +47,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/trunk/test/org/apache/pig/test/TestBZip.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestBZip.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestBZip.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestBZip.java Fri May 7 00:24:55 2010 @@ -44,10 +44,18 @@ import org.apache.pig.data.Tuple; import org.apache.pig.impl.PigContext; import org.apache.tools.bzip2r.CBZip2InputStream; import org.apache.tools.bzip2r.CBZip2OutputStream; +import org.junit.AfterClass; import org.junit.Test; - +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +@RunWith(JUnit4.class) public class TestBZip { - MiniCluster cluster = MiniCluster.buildCluster(); + static MiniCluster cluster = MiniCluster.buildCluster(); + + @AfterClass + public static void oneTimeTearDown() throws Exception { + cluster.shutDown(); + } /** * Tests the end-to-end writing and reading of a BZip file. Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestBatchAliases.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestBatchAliases.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestBatchAliases.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestBatchAliases.java Fri May 7 00:24:55 2010 @@ -28,9 +28,12 @@ import org.apache.pig.PigServer; import org.apache.pig.backend.executionengine.ExecJob; 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; +@RunWith(JUnit4.class) public class TestBatchAliases extends TestCase { private static final MiniCluster cluster = MiniCluster.buildCluster(); @@ -52,6 +55,11 @@ public class TestBatchAliases extends Te Util.deleteFile(cluster, "passwd"); } + @AfterClass + public static void oneTimeTearDown() throws Exception { + cluster.shutDown(); + } + @Test public void testBatchAliases() throws IOException { Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestBestFitCast.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestBestFitCast.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestBestFitCast.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestBestFitCast.java Fri May 7 00:24:55 2010 @@ -17,6 +17,8 @@ */ package org.apache.pig.test; +import static org.junit.Assert.*; + import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -38,12 +40,13 @@ import org.apache.pig.impl.logicalLayer. import org.apache.pig.impl.logicalLayer.schema.Schema; import org.apache.pig.impl.util.LogUtils; import org.junit.After; +import org.junit.AfterClass; import org.junit.Before; import org.junit.Test; -public class TestBestFitCast extends TestCase { +public class TestBestFitCast { private PigServer pigServer; - private MiniCluster cluster = MiniCluster.buildCluster(); + private static MiniCluster cluster = MiniCluster.buildCluster(); String inputFile, inputFile2; int LOOP_SIZE = 20; @@ -76,6 +79,11 @@ public class TestBestFitCast extends Tes Util.deleteFile(cluster, inputFile2); } + @AfterClass + public static void oneTimeTearDown() throws Exception { + cluster.shutDown(); + } + public static class UDF1 extends EvalFunc{ /** * java level API @@ -237,7 +245,7 @@ public class TestBestFitCast extends Tes while(iter.hasNext()){ Tuple t = iter.next(); assertTrue(((Tuple)t.get(1)).get(0) instanceof Float); - assertEquals((Float)((Tuple)t.get(1)).get(0), 0.0f); + assertEquals((Float)((Tuple)t.get(1)).get(0), (Float)0.0f); assertTrue(((Tuple)t.get(1)).get(1) instanceof Long); assertEquals((Long)((Tuple)t.get(1)).get(1), new Long(cnt + 1)); ++cnt; @@ -258,7 +266,7 @@ public class TestBestFitCast extends Tes while(iter.hasNext()){ Tuple t = iter.next(); assertTrue(((Tuple)t.get(1)).get(0) instanceof Float); - assertEquals((Float)((Tuple)t.get(1)).get(0), 0.0f); + assertEquals((Float)((Tuple)t.get(1)).get(0), (Float)0.0f); assertTrue(((Tuple)t.get(1)).get(1) instanceof Long); assertEquals((Long)((Tuple)t.get(1)).get(1), new Long(cnt + 1)); ++cnt; @@ -279,7 +287,7 @@ public class TestBestFitCast extends Tes while(iter.hasNext()){ Tuple t = iter.next(); assertTrue(((Tuple)t.get(1)).get(0) instanceof Float); - assertEquals((Float)((Tuple)t.get(1)).get(0), 0.0f); + assertEquals((Float)((Tuple)t.get(1)).get(0), (Float)0.0f); assertTrue(((Tuple)t.get(1)).get(1) instanceof Double); assertEquals((Double)((Tuple)t.get(1)).get(1), new Double(cnt + 1)); ++cnt; @@ -362,7 +370,7 @@ public class TestBestFitCast extends Tes while(iter.hasNext()){ Tuple t = iter.next(); assertTrue(((Tuple)t.get(1)).get(0) instanceof Float); - assertEquals((Float)((Tuple)t.get(1)).get(0), 0.0f); + assertEquals((Float)((Tuple)t.get(1)).get(0), (Float)0.0f); assertTrue(((Tuple)t.get(1)).get(1) instanceof Long); assertEquals((Long)((Tuple)t.get(1)).get(1), new Long(cnt + 1)); assertTrue(((Tuple)t.get(1)).get(2) instanceof Double); @@ -385,7 +393,7 @@ public class TestBestFitCast extends Tes while(iter.hasNext()){ Tuple t = iter.next(); assertTrue(((Tuple)t.get(1)).get(0) instanceof Float); - assertEquals((Float)((Tuple)t.get(1)).get(0), 0.0f); + assertEquals((Float)((Tuple)t.get(1)).get(0), (Float)0.0f); assertTrue(((Tuple)t.get(1)).get(1) instanceof Double); assertEquals((Double)((Tuple)t.get(1)).get(1), new Double(cnt + 1)); assertTrue(((Tuple)t.get(1)).get(2) instanceof Long); @@ -408,7 +416,7 @@ public class TestBestFitCast extends Tes while(iter.hasNext()){ Tuple t = iter.next(); assertTrue(((Tuple)t.get(1)).get(0) instanceof Float); - assertEquals((Float)((Tuple)t.get(1)).get(0), 0.0f); + assertEquals((Float)((Tuple)t.get(1)).get(0), (Float)0.0f); assertTrue(((Tuple)t.get(1)).get(1) instanceof Long); assertEquals((Long)((Tuple)t.get(1)).get(1), new Long(cnt + 1)); assertTrue(((Tuple)t.get(1)).get(2) instanceof Double); @@ -431,7 +439,7 @@ public class TestBestFitCast extends Tes while(iter.hasNext()){ Tuple t = iter.next(); assertTrue(((Tuple)t.get(1)).get(0) instanceof Float); - assertEquals((Float)((Tuple)t.get(1)).get(0), 0.0f); + assertEquals((Float)((Tuple)t.get(1)).get(0), (Float)0.0f); assertTrue(((Tuple)t.get(1)).get(1) instanceof Double); assertEquals((Double)((Tuple)t.get(1)).get(1), new Double(cnt + 1)); assertTrue(((Tuple)t.get(1)).get(2) instanceof Long); @@ -478,7 +486,7 @@ public class TestBestFitCast extends Tes while(iter.hasNext()){ Tuple t = iter.next(); assertTrue(((Tuple)t.get(1)).get(0) instanceof Float); - assertEquals((Float)((Tuple)t.get(1)).get(0), 0.0f); + assertEquals((Float)((Tuple)t.get(1)).get(0), (Float)0.0f); assertTrue(((Tuple)t.get(1)).get(1) instanceof Long); assertEquals((Long)((Tuple)t.get(1)).get(1), new Long(cnt + 1)); assertTrue(((Tuple)t.get(1)).get(2) instanceof Double); Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestBuiltin.java Fri May 7 00:24:55 2010 @@ -17,13 +17,13 @@ */ package org.apache.pig.test; +import static org.junit.Assert.*; + import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import junit.framework.TestCase; - import org.apache.pig.Algebraic; import org.apache.pig.EvalFunc; import org.apache.pig.ExecType; @@ -56,13 +56,15 @@ import org.apache.pig.data.Tuple; import org.apache.pig.data.TupleFactory; import org.apache.pig.impl.PigContext; import org.apache.pig.impl.io.ReadToEndLoader; +import org.junit.AfterClass; +import org.junit.Before; import org.junit.Test; -public class TestBuiltin extends TestCase { +public class TestBuiltin { private String initString = "mapreduce"; //private String initString = "local"; - MiniCluster cluster = MiniCluster.buildCluster(); + static MiniCluster cluster = MiniCluster.buildCluster(); PigServer pigServer; @@ -119,8 +121,8 @@ public class TestBuiltin extends TestCas }; String[] inputTypeAsString = {"ByteArray", "Integer", "Long", "Float", "Double", "String" }; - - @Override + + @Before public void setUp() throws Exception { pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()); @@ -267,6 +269,11 @@ public class TestBuiltin extends TestCas } } + @AfterClass + public static void oneTimeTearDown() throws Exception { + cluster.shutDown(); + } + /** * Test the case where the combiner is not called - so initial is called * and then final is called Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestCharArrayToNumeric.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestCharArrayToNumeric.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestCharArrayToNumeric.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestCharArrayToNumeric.java Fri May 7 00:24:55 2010 @@ -42,8 +42,13 @@ import org.apache.pig.data.Tuple; import org.apache.pig.data.TupleFactory; import org.apache.pig.impl.plan.NodeIdGenerator; import org.apache.pig.impl.plan.OperatorKey; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; -public class TestCharArrayToNumeric extends TestCase { +import static org.junit.Assert.*; + +public class TestCharArrayToNumeric { private Double dummyDouble = null; @@ -69,22 +74,26 @@ public class TestCharArrayToNumeric exte private Integer MinInteger = Integer.MIN_VALUE; - MiniCluster cluster = MiniCluster.buildCluster(); + static MiniCluster cluster = MiniCluster.buildCluster(); PigServer pig; - /* (non-Javadoc) - * @see junit.framework.TestCase#setUp() - */ - @Override - protected void setUp() throws Exception { + + @Before + public void setUp() throws Exception { pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()); } + + @AfterClass + public static void oneTimeTearDown() throws Exception { + cluster.shutDown(); + } public static OperatorKey newOperatorKey() { long newId = NodeIdGenerator.getGenerator().getNextNodeId("scope"); return new OperatorKey("scope", newId); } + @Test public void testCast() throws ExecException { POCast cast = new POCast(newOperatorKey(), -1); @@ -161,7 +170,7 @@ public class TestCharArrayToNumeric exte } } - + @Test public void testCharArray2FloatAndDoubleScript() throws IOException { // create a input file with format (key,value) @@ -200,7 +209,7 @@ public class TestCharArrayToNumeric exte if (type == DataType.DOUBLE) { Double expected = map.get(key); if (value != null) { - assertEquals(expected, Double.parseDouble(value)); + assertEquals(expected, (Double)Double.parseDouble(value)); } else { assertEquals(expected, null); } @@ -212,7 +221,7 @@ public class TestCharArrayToNumeric exte expected = map.get(key).floatValue(); } if (value != null) { - assertEquals(expected, Float.parseFloat(value)); + assertEquals(expected, (Float)Float.parseFloat(value)); } else { assertEquals(expected, null); } @@ -227,6 +236,7 @@ public class TestCharArrayToNumeric exte } } + @Test public void testCharArrayToIntAndLongScript() throws IOException { // create a input file with format (key,value) Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestCollectedGroup.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestCollectedGroup.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestCollectedGroup.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestCollectedGroup.java Fri May 7 00:24:55 2010 @@ -42,13 +42,18 @@ import org.apache.pig.impl.logicalLayer. import org.apache.pig.impl.logicalLayer.LogicalPlan; 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()); @@ -81,6 +86,7 @@ public class TestCollectedGroup extends Util.deleteFile(cluster, INPUT_FILE); } + @Test public void testNonCollectableLoader() throws Exception{ LogicalPlanTester lpt = new LogicalPlanTester(); lpt.buildPlan("A = LOAD '" + INPUT_FILE + "' as (id, name, grade);"); @@ -95,6 +101,7 @@ public class TestCollectedGroup extends } } + @Test public void testCollectedGrpSpecifiedInSingleQuotes1(){ LogicalPlanTester lpt = new LogicalPlanTester(); @@ -103,6 +110,7 @@ public class TestCollectedGroup extends assertEquals(LOCogroup.GROUPTYPE.COLLECTED, ((LOCogroup)lp.getLeaves().get(0)).getGroupType()); } + @Test public void testCollectedGrpSpecifiedInSingleQuotes2(){ LogicalPlanTester lpt = new LogicalPlanTester(); @@ -111,14 +119,21 @@ public class TestCollectedGroup extends assertEquals(LOCogroup.GROUPTYPE.REGULAR, ((LOCogroup)lp.getLeaves().get(0)).getGroupType()); } + @AfterClass + public static void oneTimeTearDown() throws Exception { + cluster.shutDown(); + } + + @Test public void testPOMapsideGroupNoNullPlans() throws IOException { POCollectedGroup pmg = new POCollectedGroup(new OperatorKey()); List 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()); @@ -134,6 +149,7 @@ public class TestCollectedGroup extends } } + @Test public void testMapsideGroupParserNoSupportForGroupAll() throws IOException { pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()); @@ -148,6 +164,7 @@ public class TestCollectedGroup extends } } + @Test public void testMapsideGroupParserNoSupportForByExpression() throws IOException { pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()); @@ -162,6 +179,7 @@ public class TestCollectedGroup extends } } + @Test public void testMapsideGroupByOneColumn() throws IOException{ pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()); @@ -196,6 +214,7 @@ public class TestCollectedGroup extends } } + @Test public void testMapsideGroupByMultipleColumns() throws IOException{ pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()); @@ -231,6 +250,7 @@ public class TestCollectedGroup extends } } + @Test public void testMapsideGroupByStar() throws IOException{ pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()); Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestCombiner.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestCombiner.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestCombiner.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestCombiner.java Fri May 7 00:24:55 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; @@ -44,10 +47,17 @@ import org.apache.pig.impl.io.FileLocali import org.apache.pig.impl.logicalLayer.LogicalPlan; import org.apache.pig.test.utils.LogicalPlanTester; +@RunWith(JUnit4.class) public class TestCombiner extends TestCase { - MiniCluster cluster = MiniCluster.buildCluster(); + static MiniCluster cluster = MiniCluster.buildCluster(); + @AfterClass + public static void oneTimeTearDown() throws Exception { + cluster.shutDown(); + } + + @Test public void testSuccessiveUserFuncs1() throws Exception{ Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestCommit.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestCommit.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestCommit.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestCommit.java Fri May 7 00:24:55 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/trunk/test/org/apache/pig/test/TestCompressedFiles.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestCompressedFiles.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestCompressedFiles.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestCompressedFiles.java Fri May 7 00:24:55 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; @@ -39,16 +42,17 @@ import org.apache.pig.test.utils.TestHel import junit.framework.Assert; 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); @@ -72,11 +76,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/trunk/test/org/apache/pig/test/TestCounters.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestCounters.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestCounters.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestCounters.java Fri May 7 00:24:55 2010 @@ -37,16 +37,25 @@ import org.apache.pig.backend.executione import org.apache.pig.impl.io.FileLocalizer; import org.apache.pig.tools.pigstats.PigStats; import org.apache.pig.tools.pigstats.PigStatsUtil; +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/trunk/test/org/apache/pig/test/TestDataBagAccess.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestDataBagAccess.java Fri May 7 00:24:55 2010 @@ -37,17 +37,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 @@ -56,6 +60,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/trunk/test/org/apache/pig/test/TestEvalPipeline.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestEvalPipeline.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestEvalPipeline.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestEvalPipeline.java Fri May 7 00:24:55 2010 @@ -56,12 +56,16 @@ import org.apache.pig.impl.logicalLayer. import org.apache.pig.impl.util.Pair; import org.apache.pig.test.utils.GenRandomData; import org.apache.pig.test.utils.Identity; +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 TestEvalPipeline extends TestCase { - MiniCluster cluster = MiniCluster.buildCluster(); + static MiniCluster cluster = MiniCluster.buildCluster(); private PigServer pigServer; private PigContext pigContext; @@ -77,6 +81,11 @@ public class TestEvalPipeline extends Te pigContext = pigServer.getPigContext(); } + @AfterClass + public static void oneTimeTearDown() throws Exception { + cluster.shutDown(); + } + static public class MyBagFunction extends EvalFunc{ @Override public DataBag exec(Tuple input) throws IOException { @@ -129,7 +138,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"); @@ -165,7 +174,7 @@ public class TestEvalPipeline extends Te t.append(weights); b.add(t); - File tmpFile = File.createTempFile("tmp", ""); + File tmpFile = Util.createTempFileDelOnExit("tmp", ""); tmpFile.deleteOnExit(); String fileName = tmpFile.getAbsolutePath(); PigFile f = new PigFile(fileName); @@ -364,7 +373,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++) { @@ -401,9 +410,10 @@ public class TestEvalPipeline extends Te } } + @Test 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)); for(int i = 0; i < LOOP_COUNT; i++) { for(int j=0;j{ @@ -78,14 +81,19 @@ public class TestFinish extends TestCase } } - @Override - protected void setUp() throws Exception { + @Before + public void setUp() throws Exception { // re initialize FileLocalizer so that each test runs correctly without // any side effect of other tests - this is needed here since some // tests are in mapred and some in local mode FileLocalizer.setInitialized(false); } + @AfterClass + public static void oneTimeTearDown() throws Exception { + cluster.shutDown(); + } + private String setUp(ExecType execType) throws Exception{ String inputFileName; if(execType == ExecType.LOCAL) { Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestForEachNestedPlan.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestForEachNestedPlan.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestForEachNestedPlan.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestForEachNestedPlan.java Fri May 7 00:24:55 2010 @@ -18,28 +18,30 @@ package org.apache.pig.test; -import org.apache.pig.ExecType; -import org.apache.pig.PigServer; -import org.apache.pig.ExecType; -import org.apache.pig.test.utils.TestHelper; -import org.apache.pig.data.Tuple; -import org.apache.pig.impl.logicalLayer.parser.ParseException; -import org.junit.Test; -import junit.framework.TestCase; -import junit.framework.Assert; +import static org.junit.Assert.*; -import java.util.Iterator; -import java.util.List; -import java.util.Random; import java.io.File; import java.io.FileWriter; import java.io.IOException; import java.io.PrintWriter; import java.text.DecimalFormat; +import java.util.Iterator; +import java.util.List; +import java.util.Random; + +import junit.framework.Assert; + +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.logicalLayer.parser.ParseException; +import org.apache.pig.test.utils.TestHelper; +import org.junit.AfterClass; +import org.junit.Test; -public class TestForEachNestedPlan extends TestCase { +public class TestForEachNestedPlan { - MiniCluster cluster = MiniCluster.buildCluster(); + static MiniCluster cluster = MiniCluster.buildCluster(); private PigServer pig ; @@ -49,6 +51,11 @@ public class TestForEachNestedPlan exten Boolean[] nullFlags = new Boolean[]{ false, true }; + @AfterClass + public static void oneTimeTearDown() throws Exception { + cluster.shutDown(); + } + @Test public void testInnerOrderBy() throws Exception { for (int i = 0; i < nullFlags.length; i++) { Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestForEachNestedPlanLocal.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestForEachNestedPlanLocal.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestForEachNestedPlanLocal.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestForEachNestedPlanLocal.java Fri May 7 00:24:55 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/trunk/test/org/apache/pig/test/TestGrunt.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestGrunt.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestGrunt.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestGrunt.java Fri May 7 00:24:55 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 org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -44,18 +49,23 @@ import java.io.StringReader; import java.io.StringWriter; import java.util.ArrayList; +@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 { @@ -802,6 +812,7 @@ public class TestGrunt extends TestCase assertTrue(caught); } + @Test public void testFsCommand(){ try { @@ -837,6 +848,7 @@ public class TestGrunt extends TestCase } } + @Test public void testSetPriority() throws Throwable { PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()); PigContext context = server.getPigContext(); @@ -852,6 +864,7 @@ public class TestGrunt extends TestCase assertEquals("high", context.getProperties().getProperty(PigContext.JOB_PRIORITY)); } + @Test public void testSetWithQuotes() throws Throwable { PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()); PigContext context = server.getPigContext(); @@ -867,6 +880,7 @@ public class TestGrunt extends TestCase assertEquals("high", context.getProperties().getProperty(PigContext.JOB_PRIORITY)); } + @Test public void testRegisterWithQuotes() throws Throwable { PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()); PigContext context = server.getPigContext(); @@ -882,6 +896,7 @@ public class TestGrunt extends TestCase assertTrue(context.extraJars.contains(ClassLoader.getSystemResource("pig-withouthadoop.jar"))); } + @Test public void testRegisterWithoutQuotes() throws Throwable { PigServer server = new PigServer(ExecType.MAPREDUCE, cluster.getProperties()); PigContext context = server.getPigContext(); @@ -897,6 +912,7 @@ public class TestGrunt extends TestCase assertTrue(context.extraJars.contains(ClassLoader.getSystemResource("pig-withouthadoop.jar"))); } + @Test public void testScriptMissingLastNewLine() throws Throwable { PigServer server = new PigServer(ExecType.LOCAL); PigContext context = server.getPigContext(); Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestHBaseStorage.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestHBaseStorage.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestHBaseStorage.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestHBaseStorage.java Fri May 7 00:24:55 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/trunk/test/org/apache/pig/test/TestImplicitSplit.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestImplicitSplit.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestImplicitSplit.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestImplicitSplit.java Fri May 7 00:24:55 2010 @@ -20,6 +20,7 @@ package org.apache.pig.test; import static org.apache.pig.test.utils.TypeCheckingTestUtil.printMessageCollector; import static org.apache.pig.test.utils.TypeCheckingTestUtil.printTypeGraph; +import static org.junit.Assert.*; import java.io.File; import java.io.FileOutputStream; @@ -39,12 +40,13 @@ import org.apache.pig.impl.logicalLayer. import org.apache.pig.impl.plan.CompilationMessageCollector; import org.apache.pig.test.utils.LogicalPlanTester; import org.junit.After; +import org.junit.AfterClass; import org.junit.Before; import org.junit.Test; -public class TestImplicitSplit extends TestCase{ +public class TestImplicitSplit { private PigServer pigServer; - MiniCluster cluster = MiniCluster.buildCluster(); + static MiniCluster cluster = MiniCluster.buildCluster(); @Before public void setUp() throws Exception { @@ -55,6 +57,11 @@ public class TestImplicitSplit extends T public void tearDown() throws Exception { } + @AfterClass + public static void oneTimeTearDown() throws Exception { + cluster.shutDown(); + } + @Test public void testImplicitSplit() throws Exception{ int LOOP_SIZE = 20; Modified: hadoop/pig/trunk/test/org/apache/pig/test/TestInfixArithmetic.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestInfixArithmetic.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestInfixArithmetic.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestInfixArithmetic.java Fri May 7 00:24:55 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/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestInputOutputFileValidator.java Fri May 7 00:24:55 2010 @@ -43,15 +43,23 @@ import org.apache.pig.impl.logicalLayer. import org.apache.pig.impl.plan.CompilationMessageCollector; import org.apache.pig.impl.plan.CompilationMessageCollector.MessageType; import org.apache.pig.impl.util.LogUtils; +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; +@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/trunk/test/org/apache/pig/test/TestJobSubmission.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestJobSubmission.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestJobSubmission.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestJobSubmission.java Fri May 7 00:24:55 2010 @@ -79,9 +79,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{ @@ -96,9 +100,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(); @@ -124,6 +129,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/trunk/test/org/apache/pig/test/TestJoin.java URL: http://svn.apache.org/viewvc/hadoop/pig/trunk/test/org/apache/pig/test/TestJoin.java?rev=941976&r1=941975&r2=941976&view=diff ============================================================================== --- hadoop/pig/trunk/test/org/apache/pig/test/TestJoin.java (original) +++ hadoop/pig/trunk/test/org/apache/pig/test/TestJoin.java Fri May 7 00:24:55 2010 @@ -39,17 +39,22 @@ import org.apache.pig.impl.logicalLayer. import org.apache.pig.impl.logicalLayer.schema.Schema; import org.apache.pig.impl.util.LogUtils; 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(); @@ -62,6 +67,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