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

svn commit: r653524 - in /incubator/pig/trunk: ./ test/org/apache/pig/test/

Author: gates
Date: Mon May  5 09:19:58 2008
New Revision: 653524

URL: http://svn.apache.org/viewvc?rev=653524&view=rev
Log:
PIG-219: Change unit tests to run both local and map reduce modes.


Added:
    incubator/pig/trunk/test/org/apache/pig/test/PigExecTestCase.java
Modified:
    incubator/pig/trunk/CHANGES.txt
    incubator/pig/trunk/build.xml
    incubator/pig/trunk/test/org/apache/pig/test/TestAlgebraicEval.java
    incubator/pig/trunk/test/org/apache/pig/test/TestBinaryStorage.java
    incubator/pig/trunk/test/org/apache/pig/test/TestCombiner.java
    incubator/pig/trunk/test/org/apache/pig/test/TestCompressedFiles.java
    incubator/pig/trunk/test/org/apache/pig/test/TestCustomSlicer.java
    incubator/pig/trunk/test/org/apache/pig/test/TestEvalPipeline.java
    incubator/pig/trunk/test/org/apache/pig/test/TestExGenCogroup.java
    incubator/pig/trunk/test/org/apache/pig/test/TestExGenEval.java
    incubator/pig/trunk/test/org/apache/pig/test/TestFilterOpNumeric.java
    incubator/pig/trunk/test/org/apache/pig/test/TestFilterOpString.java
    incubator/pig/trunk/test/org/apache/pig/test/TestMapReduceResultRecycling.java
    incubator/pig/trunk/test/org/apache/pig/test/TestOrderBy.java
    incubator/pig/trunk/test/org/apache/pig/test/TestParser.java
    incubator/pig/trunk/test/org/apache/pig/test/TestPigServer.java
    incubator/pig/trunk/test/org/apache/pig/test/TestPigSplit.java
    incubator/pig/trunk/test/org/apache/pig/test/TestReversibleLoadStore.java
    incubator/pig/trunk/test/org/apache/pig/test/TestStore.java
    incubator/pig/trunk/test/org/apache/pig/test/TestStreaming.java

Modified: incubator/pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/CHANGES.txt?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/CHANGES.txt (original)
+++ incubator/pig/trunk/CHANGES.txt Mon May  5 09:19:58 2008
@@ -274,3 +274,5 @@
     PIG-215: Cleanup a few dangling ends left by PIG-111 (pi_song via gates).
     
     PIG-229: Proper error handling in case of deserializer failure
+
+	PIG-219: Change unit tests to run both local and map reduce modes (kali via gates).

Modified: incubator/pig/trunk/build.xml
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/build.xml?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/build.xml (original)
+++ incubator/pig/trunk/build.xml Mon May  5 09:19:58 2008
@@ -248,15 +248,27 @@
     <!-- ================================================================== -->
     <!-- Run unit tests                                                     -->
     <!-- ================================================================== -->
-    <target name="test" depends="compile-test,jar" description="Run unit tests">
-        <delete dir="${test.log.dir}"/>
-        <mkdir dir="${test.log.dir}"/>
+    <target name="test" depends="test-local,test-mapreduce" description="Run unit tests in local and mapreduce modes"/>
+    <target name="test-local" depends="compile-test,jar" description="Run unit tests in local mode">
+        <antcall target="testOneMode">
+            <param name="test.exectype" value="local" />
+        </antcall>
+    </target>
+    <target name="test-mapreduce" depends="compile-test,jar" description="Run unit tests in mapreduce mode">
+        <antcall target="testOneMode">
+            <param name="test.exectype" value="mapreduce" />
+        </antcall>
+    </target>
+    <target name="testOneMode">
+        <delete dir="${test.log.dir}-${test.exectype}"/>
+        <mkdir dir="${test.log.dir}-${test.exectype}"/>
         <junit showoutput="${test.output}" printsummary="yes" haltonfailure="no" fork="yes" maxmemory="256m" dir="${basedir}" timeout="${test.timeout}" errorProperty="tests.failed" failureProperty="tests.failed">
+            <sysproperty key="test.exectype" value="${test.exectype}" />
             <sysproperty key="ssh.gateway" value="${ssh.gateway}" />
             <sysproperty key="hod.server" value="${hod.server}" />
             <!-- <sysproperty key="hod.command" value="${hod.command}"/>
             			<sysproperty key="hod.param" value="${hod.param}"/> -->
-            <sysproperty key="hadoop.log.dir" value="${test.log.dir}"/>
+            <sysproperty key="hadoop.log.dir" value="${test.log.dir}-${test.exectype}"/>
             <classpath>
                 <pathelement location="${output.jarfile}" />
                 <pathelement location="${test.build.classes}" />
@@ -264,16 +276,17 @@
                 <path refid="classpath"/>
             </classpath>
             <formatter type="${test.junit.output.format}" />
-            <batchtest fork="yes" todir="${test.log.dir}" unless="testcase">
+            <batchtest fork="yes" todir="${test.log.dir}-${test.exectype}" unless="testcase">
                 <fileset dir="test">
                     <include name="**/*Test*.java" />
+                    <exclude name="**/PigExecTestCase.java" />
                     <exclude name="**/TestLargeFile.java" />
                     <exclude name="**/TestOrderBy.java" />
                     <exclude name="**/TestPi.java" />
                     <exclude name="**/nightly/**" />
                 </fileset>
             </batchtest>
-            <batchtest fork="yes" todir="${test.log.dir}" if="testcase">
+            <batchtest fork="yes" todir="${test.log.dir}-${test.exectype}" if="testcase">
                 <fileset dir="test" includes="**/${testcase}.java"/>
             </batchtest>
         </junit>

Added: incubator/pig/trunk/test/org/apache/pig/test/PigExecTestCase.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/PigExecTestCase.java?rev=653524&view=auto
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/PigExecTestCase.java (added)
+++ incubator/pig/trunk/test/org/apache/pig/test/PigExecTestCase.java Mon May  5 09:19:58 2008
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+import static org.apache.pig.PigServer.ExecType.MAPREDUCE;
+import static org.apache.pig.PigServer.ExecType.LOCAL;
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.PigServer;
+import org.apache.pig.PigServer.ExecType;
+import org.junit.After;
+import org.junit.Before;
+
+public abstract class PigExecTestCase extends TestCase {
+
+    protected final Log log = LogFactory.getLog(getClass());
+    
+    protected ExecType execType = MAPREDUCE;
+    
+    private MiniCluster cluster;
+    protected PigServer pigServer;
+    
+    @Before
+    @Override
+    protected void setUp() throws Exception {
+        
+        String execTypeString = System.getProperty("test.exectype");
+        if(execTypeString!=null && execTypeString.length()>0){
+            execType = PigServer.parseExecType(execTypeString);
+        }
+        if(execType == MAPREDUCE) {
+            cluster = MiniCluster.buildCluster();
+            pigServer = new PigServer(MAPREDUCE, cluster.getProperties());
+        } else {
+            pigServer = new PigServer(LOCAL);
+        }
+    }
+
+    @After
+    @Override
+    protected void tearDown() throws Exception {
+        pigServer.shutdown();
+    }
+}

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestAlgebraicEval.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestAlgebraicEval.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestAlgebraicEval.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestAlgebraicEval.java Mon May  5 09:19:58 2008
@@ -17,40 +17,19 @@
  */
 package org.apache.pig.test;
 
-import static org.apache.pig.PigServer.ExecType.MAPREDUCE;
-
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.PrintStream;
 import java.util.Iterator;
 
-import junit.framework.TestCase;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.pig.PigServer;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.Tuple;
+import org.junit.Test;
 
-public class TestAlgebraicEval extends TestCase {
+public class TestAlgebraicEval extends PigExecTestCase {
 
-    private final Log log = LogFactory.getLog(getClass());
-    
     private int LOOP_COUNT = 1024;
 
-    private PigServer pig;
-    
-    @Before
-    @Override
-    protected void setUp() throws Exception {
-        pig = new PigServer(MAPREDUCE, cluster.getProperties());
-    }
-    
-    MiniCluster cluster = MiniCluster.buildCluster();
-
     @Test
     public void testSimpleCount() throws Throwable {
         File tmpFile = File.createTempFile("test", "txt");
@@ -66,8 +45,8 @@
         if (log.isDebugEnabled()) {
             log.debug(query.toString());
         }
-        pig.registerQuery(query.toString());
-        Iterator it = pig.openIterator("myid");
+        pigServer.registerQuery(query.toString());
+        Iterator it = pigServer.openIterator("myid");
         tmpFile.delete();
         Tuple t = (Tuple)it.next();
         Double count = t.getAtomField(0).numval();
@@ -89,8 +68,8 @@
         if (log.isDebugEnabled()) {
             log.debug(query.toString());
         }
-        pig.registerQuery(query.toString());
-        Iterator it = pig.openIterator("myid");
+        pigServer.registerQuery(query.toString());
+        Iterator it = pigServer.openIterator("myid");
         tmpFile.delete();
         Tuple t = (Tuple)it.next();
         Double count = t.getAtomField(1).numval();
@@ -112,8 +91,8 @@
         if (log.isDebugEnabled()) {
             log.debug(query.toString());
         }
-        pig.registerQuery(query.toString());
-        Iterator it = pig.openIterator("myid");
+        pigServer.registerQuery(query.toString());
+        Iterator it = pigServer.openIterator("myid");
         tmpFile.delete();
         Tuple t = (Tuple)it.next();
         Double count = t.getAtomField(0).numval();
@@ -134,8 +113,8 @@
         ps.close();
         String query = "myid = foreach (group (load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':')) by $0) generate group, COUNT($1.$1) ;";
         System.out.println(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("myid");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("myid");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
@@ -160,8 +139,8 @@
         ps.close();
         String query = "myid = foreach (group (load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':')) by $0) generate group, COUNT($1.$1), COUNT($1.$0) ;";
         System.out.println(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("myid");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("myid");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestBinaryStorage.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestBinaryStorage.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestBinaryStorage.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestBinaryStorage.java Mon May  5 09:19:58 2008
@@ -32,12 +32,10 @@
 
 import junit.framework.TestCase;
 
-public class TestBinaryStorage extends TestCase {
+public class TestBinaryStorage extends PigExecTestCase {
     private static final String simpleEchoStreamingCommand = 
         "perl -ne 'print \"$_\"'";
 
-    MiniCluster cluster = MiniCluster.buildCluster();
-
     private static final int MAX_DATA_SIZE = 1024;
     
     @Test
@@ -69,8 +67,6 @@
     
     private void testBinaryStorage(File input) 
     throws Exception {
-        PigServer pigServer = new PigServer(ExecType.MAPREDUCE,
-			cluster.getProperties());
 
         // Get input data 
         byte[] inputData = new byte[MAX_DATA_SIZE];

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestCombiner.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestCombiner.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestCombiner.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestCombiner.java Mon May  5 09:19:58 2008
@@ -1,8 +1,5 @@
 package org.apache.pig.test;
 
-import static org.apache.pig.PigServer.ExecType.MAPREDUCE;
-import static org.apache.pig.PigServer.ExecType.LOCAL;
-
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -10,41 +7,23 @@
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Properties;
-
-import org.junit.Test;
-import junit.framework.TestCase;
 
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.Tuple;
 
-public class TestCombiner extends TestCase {
-
-    @Test
-    public void testLocal() throws Exception {
-        // run the test locally
-        runTest(new PigServer(LOCAL, new Properties()));
-    }
-
-    @Test
-    public void testOnCluster() throws Exception {
-        // run the test on cluster
-        MiniCluster buildCluster = MiniCluster.buildCluster();
-        runTest(new PigServer(MAPREDUCE, buildCluster.getProperties()));
-
-    }
+public class TestCombiner extends PigExecTestCase {
 
-    private void runTest(PigServer pig) throws IOException {
+    public void testCombiner() throws IOException {
         List<String> inputLines = new ArrayList<String>();
         inputLines.add("a,b,1");
         inputLines.add("a,b,1");
         inputLines.add("a,c,1");
-        loadWithTestLoadFunc("A", pig, inputLines);
+        loadWithTestLoadFunc("A", pigServer, inputLines);
 
-        pig.registerQuery("B = group A by ($0, $1);");
-        pig.registerQuery("C = foreach B generate flatten(group), COUNT($1);");
-        Iterator<Tuple> resultIterator = pig.openIterator("C");
+        pigServer.registerQuery("B = group A by ($0, $1);");
+        pigServer.registerQuery("C = foreach B generate flatten(group), COUNT($1);");
+        Iterator<Tuple> resultIterator = pigServer.openIterator("C");
         Tuple tuple = resultIterator.next();
         assertEquals("(a, b, 2)", tuple.toString());
         tuple = resultIterator.next();

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestCompressedFiles.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestCompressedFiles.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestCompressedFiles.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestCompressedFiles.java Mon May  5 09:19:58 2008
@@ -18,31 +18,30 @@
 package org.apache.pig.test;
 
 
-import static org.apache.pig.PigServer.ExecType.MAPREDUCE;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.util.Iterator;
 import java.util.Random;
 import java.util.zip.GZIPOutputStream;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.pig.PigServer;
+import org.apache.pig.PigServer.ExecType;
 import org.apache.pig.builtin.DIFF;
-import junit.framework.TestCase;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
-public class TestCompressedFiles extends TestCase {
+public class TestCompressedFiles extends PigExecTestCase {
     
     private final Log log = LogFactory.getLog(getClass());
-    MiniCluster cluster = MiniCluster.buildCluster();
 
     File datFile;
     File gzFile;
     @Override
 	@Before
     protected void setUp() throws Exception {
+        super.setUp();
         datFile = File.createTempFile("compTest", ".dat");
         gzFile = File.createTempFile("compTest", ".gz");
         FileOutputStream dat = new FileOutputStream(datFile);
@@ -69,13 +68,16 @@
     protected void tearDown() throws Exception {
         datFile.delete();
         gzFile.delete();
+        super.tearDown();
     }
     
     @Test
     public void testCompressed1() throws Throwable {
-        PigServer pig = new PigServer(MAPREDUCE, cluster.getProperties());
-        pig.registerQuery("A = foreach (cogroup (load 'file:"+gzFile+"') by $1, (load 'file:"+datFile + "') by $1) generate flatten( " + DIFF.class.getName() + "($1.$1,$2.$1)) ;");
-        Iterator it = pig.openIterator("A");
+        // FIXME : this should be tested in all modes
+        if(execType == ExecType.LOCAL)
+            return;
+        pigServer.registerQuery("A = foreach (cogroup (load 'file:"+gzFile+"') by $1, (load 'file:"+datFile + "') by $1) generate flatten( " + DIFF.class.getName() + "($1.$1,$2.$1)) ;");
+        Iterator it = pigServer.openIterator("A");
         boolean success = true;
         while(it.hasNext()) {
             success = false;

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestCustomSlicer.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestCustomSlicer.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestCustomSlicer.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestCustomSlicer.java Mon May  5 09:19:58 2008
@@ -1,32 +1,30 @@
 package org.apache.pig.test;
 
-import static org.apache.pig.PigServer.ExecType.MAPREDUCE;
-import static org.junit.Assert.assertEquals;
-
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.pig.PigServer;
+import org.apache.pig.PigServer.ExecType;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.data.DataAtom;
 import org.apache.pig.data.Tuple;
 import org.junit.Test;
-import junit.framework.TestCase;
 
-public class TestCustomSlicer extends TestCase{
+public class TestCustomSlicer extends PigExecTestCase {
     /**
      * Uses RangeSlicer in place of pig's default Slicer to generate a few
      * values and count them.
      */
     @Test
     public void testUseRangeSlicer() throws ExecException, IOException {
-        PigServer pig = new PigServer(MAPREDUCE);
+        // FIXME : this should be tested in all modes
+        if(execType == ExecType.LOCAL)
+            return;
         int numvals = 50;
         String query = "vals = foreach (group (load '"
                 + numvals
                 + "'using org.apache.pig.test.RangeSlicer()) all) generate COUNT($1);";
-        pig.registerQuery(query);
-        Iterator<Tuple> it = pig.openIterator("vals");
+        pigServer.registerQuery(query);
+        Iterator<Tuple> it = pigServer.openIterator("vals");
         Tuple cur = it.next();
         DataAtom val = cur.getAtomField(0);
         assertEquals(numvals, (int) val.longVal());

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestEvalPipeline.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestEvalPipeline.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestEvalPipeline.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestEvalPipeline.java Mon May  5 09:19:58 2008
@@ -47,16 +47,8 @@
 
 import junit.framework.TestCase;
 
-public class TestEvalPipeline extends TestCase {
-	
-	MiniCluster cluster = MiniCluster.buildCluster();
-    private PigServer pigServer;
-	
-	@Override
-    protected void setUp() throws Exception {
-	    pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
-    }
-	
+public class TestEvalPipeline extends PigExecTestCase {
+		
 	static public class MyBagFunction extends EvalFunc<DataBag>{
 		@Override
 		public void exec(Tuple input, DataBag output) throws IOException {

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestExGenCogroup.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestExGenCogroup.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestExGenCogroup.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestExGenCogroup.java Mon May  5 09:19:58 2008
@@ -4,29 +4,21 @@
 import java.io.FileOutputStream;
 import java.util.Random;
 
-import org.apache.pig.PigServer;
 import org.apache.pig.PigServer.ExecType;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import junit.framework.TestCase;
-
-public class TestExGenCogroup extends TestCase{
+public class TestExGenCogroup extends PigExecTestCase{
 	String A, B;
 	private int MAX = 10;
-	
-	String initString = "mapreduce";
-	PigServer pig;
-	
-	MiniCluster cluster = MiniCluster.buildCluster();
-	
 	@Override
 	@Before
 	protected void setUp() throws Exception{
+	    super.setUp();
+	    
 		File fileA, fileB;
-		pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
 		System.out.println("Generating test data...");
 		fileA = File.createTempFile("dataA", ".dat");
 		fileB = File.createTempFile("dataB", ".dat");
@@ -34,8 +26,8 @@
 		writeData(fileA);
 		writeData(fileB);
 		
-		A = "'" + FileLocalizer.hadoopify(fileA.toString(), pig.getPigContext()) + "'";
-		B = "'" + FileLocalizer.hadoopify(fileB.toString(), pig.getPigContext()) + "'";
+		A = "'" + FileLocalizer.hadoopify(fileA.toString(), pigServer.getPigContext()) + "'";
+		B = "'" + FileLocalizer.hadoopify(fileB.toString(), pigServer.getPigContext()) + "'";
 		System.out.println("A : " + A  + "\n" + "B : " + B);
 		System.out.println("Test data created.");
 		
@@ -56,46 +48,53 @@
 	@Override
 	@After
 	protected void tearDown() throws Exception {
-		
-	
+		super.tearDown();
 	}
 	
 	@Test
 	public void testCogroupMultipleCols() throws Exception {
-		//pig = new PigServer(initString);
-		pig.registerQuery("A = load " + A + " as (x, y);");
-		pig.registerQuery("B = load " + B + " as (x, y);");
-		pig.registerQuery("C = cogroup A by (x, y), B by (x, y);");
-		pig.showExamples("C");
+        // FIXME : this should be tested in all modes
+	    if(execType != ExecType.LOCAL)
+	        return;
+		pigServer.registerQuery("A = load " + A + " as (x, y);");
+		pigServer.registerQuery("B = load " + B + " as (x, y);");
+		pigServer.registerQuery("C = cogroup A by (x, y), B by (x, y);");
+		pigServer.showExamples("C");
 	}
 	
 	@Test
 	public void testCogroup() throws Exception {
-		//pig = new PigServer(initString);
-		pig.registerQuery("A = load " + A + " as (x, y);");
-		pig.registerQuery("B = load " + B + " as (x, y);");
-		pig.registerQuery("C = cogroup A by x, B by x;");
-		pig.showExamples("C");
+	    // FIXME : this should be tested in all modes
+        if(execType != ExecType.LOCAL)
+            return;
+		pigServer.registerQuery("A = load " + A + " as (x, y);");
+		pigServer.registerQuery("B = load " + B + " as (x, y);");
+		pigServer.registerQuery("C = cogroup A by x, B by x;");
+		pigServer.showExamples("C");
 	}
 	
 	@Test
 	public void testGroup() throws Exception {
-		//pig = new PigServer(initString);
-		pig.registerQuery("A = load " + A.toString() + " as (x, y);");
-		pig.registerQuery("B = group A by x;");
-		pig.showExamples("B");
+        // FIXME : this should be tested in all modes
+        if(execType != ExecType.LOCAL)
+            return;
+		pigServer.registerQuery("A = load " + A.toString() + " as (x, y);");
+		pigServer.registerQuery("B = group A by x;");
+		pigServer.showExamples("B");
 		
 	}
 	
 	@Test
 	public void testComplexGroup() throws Exception {
-		//pig = new PigServer(initString);
-		pig.registerQuery("A = load " + A.toString() + " as (x, y);");
-		pig.registerQuery("B = load " + B.toString() + " as (x, y);");
-		pig.registerQuery("C = cogroup A by x, B by x;");
-		pig.registerQuery("D = cogroup A by y, B by y;");
-		pig.registerQuery("E = cogroup C by $0, D by $0;");
-		pig.showExamples("E");
+        // FIXME : this should be tested in all modes
+        if(execType != ExecType.LOCAL)
+            return;
+	    pigServer.registerQuery("A = load " + A.toString() + " as (x, y);");
+	    pigServer.registerQuery("B = load " + B.toString() + " as (x, y);");
+	    pigServer.registerQuery("C = cogroup A by x, B by x;");
+	    pigServer.registerQuery("D = cogroup A by y, B by y;");
+	    pigServer.registerQuery("E = cogroup C by $0, D by $0;");
+	    pigServer.showExamples("E");
 	}
 	
 }

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestExGenEval.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestExGenEval.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestExGenEval.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestExGenEval.java Mon May  5 09:19:58 2008
@@ -16,25 +16,21 @@
 
 import junit.framework.TestCase;
 
-public class TestExGenEval extends TestCase {
+public class TestExGenEval extends PigExecTestCase {
 	
 	String A, B, C, D;
 	private int MAX = 10;
 	
-	String initString = "mapreduce";
-	PigServer pig;
 	PigContext pigContext;
 	
-	MiniCluster cluster = MiniCluster.buildCluster();
-	
 	@Override
 	@Before
 	protected void setUp() throws Exception{
+	    super.setUp();
 		System.out.println("Generating test data...");
 		File fileA, fileB, fileC, fileD;
 		
-		pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
-		pigContext = pig.getPigContext();
+		pigContext = pigServer.getPigContext();
 		fileA = File.createTempFile("dataA", ".dat");
 		fileB = File.createTempFile("dataB", ".dat");
 		fileC = File.createTempFile("dataC", ".dat");
@@ -47,10 +43,10 @@
 		writeData(fileD);
 		
 		
-		A = "'" + FileLocalizer.hadoopify(fileA.toString(), pig.getPigContext()) + "'";
-		B = "'" + FileLocalizer.hadoopify(fileB.toString(), pig.getPigContext()) + "'";
-		C = "'" + FileLocalizer.hadoopify(fileC.toString(), pig.getPigContext()) + "'";
-		D = "'" + FileLocalizer.hadoopify(fileD.toString(), pig.getPigContext()) + "'";
+		A = "'" + FileLocalizer.hadoopify(fileA.toString(), pigServer.getPigContext()) + "'";
+		B = "'" + FileLocalizer.hadoopify(fileB.toString(), pigServer.getPigContext()) + "'";
+		C = "'" + FileLocalizer.hadoopify(fileC.toString(), pigServer.getPigContext()) + "'";
+		D = "'" + FileLocalizer.hadoopify(fileD.toString(), pigServer.getPigContext()) + "'";
 		
 		System.out.println("Test data created.");
 		fileA.delete();
@@ -79,34 +75,40 @@
 	
 	@Test
 	public void testForeach() throws Exception {
-		//pig = new PigServer(initString);
-		System.out.println("Testing Foreach statement...");
-		pig.registerQuery("A = load " + A + " as (x, y);");
-		pig.registerQuery("B = foreach A generate x+y as sum;");
-		pig.showExamples("B");
+	    // FIXME : this should be tested in all modes
+        if(execType != ExecType.LOCAL)
+            return;
+	    System.out.println("Testing Foreach statement...");
+		pigServer.registerQuery("A = load " + A + " as (x, y);");
+		pigServer.registerQuery("B = foreach A generate x+y as sum;");
+		pigServer.showExamples("B");
 		assertEquals(1, 1);
 	}
 	
 	@Test 
 	public void testFilter() throws Exception {
-		//pig = new PigServer(initString);
-		pig.registerQuery("A = load " + A + " as (x, y);");
-		pig.registerQuery("B = filter A by x < 10.0;");
-		pig.showExamples("B");
+        // FIXME : this should be tested in all modes
+        if(execType != ExecType.LOCAL)
+            return;
+	    pigServer.registerQuery("A = load " + A + " as (x, y);");
+	    pigServer.registerQuery("B = filter A by x < 10.0;");
+	    pigServer.showExamples("B");
 		assertEquals(1, 1);
 	}
 	
 	@Test
 	public void testFlatten() throws Exception {
-		//pig = new PigServer(initString);
-		pig.registerQuery("A1 = load " + A + " as (x, y);");
-		pig.registerQuery("B1 = load " + B + " as (x, y);");
-		pig.registerQuery("C1 = load " + C + " as (x, y);");
-		pig.registerQuery("D1 = load " + D + " as (x, y);");
-		pig.registerQuery("E = join A1 by x, B1 by x;");
-		pig.registerQuery("F = join C1 by x, D1 by x;");
-		pig.registerQuery("G = join E by $0, F by $0;");
-		pig.showExamples("G");
+        // FIXME : this should be tested in all modes
+        if(execType != ExecType.LOCAL)
+            return;
+	    pigServer.registerQuery("A1 = load " + A + " as (x, y);");
+	    pigServer.registerQuery("B1 = load " + B + " as (x, y);");
+	    pigServer.registerQuery("C1 = load " + C + " as (x, y);");
+        pigServer.registerQuery("D1 = load " + D + " as (x, y);");
+	    pigServer.registerQuery("E = join A1 by x, B1 by x;");
+	    pigServer.registerQuery("F = join C1 by x, D1 by x;");
+	    pigServer.registerQuery("G = join E by $0, F by $0;");
+	    pigServer.showExamples("G");
 		assertEquals(1, 1);
 	}
 }

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestFilterOpNumeric.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestFilterOpNumeric.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestFilterOpNumeric.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestFilterOpNumeric.java Mon May  5 09:19:58 2008
@@ -34,21 +34,12 @@
 
 import junit.framework.TestCase;
 
-public class TestFilterOpNumeric extends TestCase {
+public class TestFilterOpNumeric extends PigExecTestCase {
 
     private final Log log = LogFactory.getLog(getClass());
 
     private static int LOOP_COUNT = 1024;
-    MiniCluster cluster = MiniCluster.buildCluster();
     
-    private PigServer pig;
-    
-    @Before
-    @Override
-    protected void setUp() throws Exception {
-        pig = new PigServer(MAPREDUCE, cluster.getProperties());
-    }
-
     @Test
     public void testNumericEq() throws Throwable {
         File tmpFile = File.createTempFile("test", "txt");
@@ -61,11 +52,11 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using "+PigStorage.class.getName() +"(':');");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "' using "+PigStorage.class.getName() +"(':');");
         String query = "A = filter A by $0 == $1;";
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
@@ -91,11 +82,11 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 != $1;";
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
@@ -117,12 +108,12 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 > $1;";
 
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
@@ -140,11 +131,11 @@
             ps.println(i + "\t" + i + "\t1");            
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "';");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "';");
         String query = "A = foreach A generate ($1 >= '"+ LOOP_COUNT+"'-'10'?'1':'0');";
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         int count =0;
         while(it.hasNext()) {
@@ -168,11 +159,11 @@
             ps.println(i + "\t" + i + "\t1");            
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "';");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "';");
         String query = "A = foreach A generate ($0 < '10'?($1 >= '5' ? '2': '1') : '0');";
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         int count =0;
         while(it.hasNext()) {
@@ -199,12 +190,12 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 < $1;";
 
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
@@ -229,12 +220,12 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 >= $1;";
 
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
@@ -258,12 +249,12 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 <= $1;";
 
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestFilterOpString.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestFilterOpString.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestFilterOpString.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestFilterOpString.java Mon May  5 09:19:58 2008
@@ -34,20 +34,11 @@
 
 import junit.framework.TestCase;
 
-public class TestFilterOpString extends TestCase {
+public class TestFilterOpString extends PigExecTestCase {
 
     private final Log log = LogFactory.getLog(getClass());
 
     private static int LOOP_COUNT = 1024;
-    MiniCluster cluster = MiniCluster.buildCluster();
-    
-    private PigServer pig;
-    
-    @Before
-    @Override
-    protected void setUp() throws Exception {
-        pig = new PigServer(MAPREDUCE, cluster.getProperties());
-    }
 
     @Test
     public void testStringEq() throws Throwable {
@@ -61,12 +52,12 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 eq $1;";
 
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
@@ -88,12 +79,12 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 neq $1;";
 
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
@@ -115,12 +106,12 @@
             }
         }
         ps.close();
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 gt $1;";
 
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
@@ -147,12 +138,12 @@
         }
         ps.close();
         
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 gte $1;";
 
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
@@ -175,12 +166,12 @@
         }
         ps.close();
         
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 lt $1;";
 
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();
@@ -205,12 +196,12 @@
         }
         ps.close();
         
-        pig.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
+        pigServer.registerQuery("A=load 'file:" + tmpFile + "' using " + PigStorage.class.getName() + "(':');");
         String query = "A = filter A by $0 lte $1;";
 
         log.info(query);
-        pig.registerQuery(query);
-        Iterator it = pig.openIterator("A");
+        pigServer.registerQuery(query);
+        Iterator it = pigServer.openIterator("A");
         tmpFile.delete();
         while(it.hasNext()) {
             Tuple t = (Tuple)it.next();

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestMapReduceResultRecycling.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestMapReduceResultRecycling.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestMapReduceResultRecycling.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestMapReduceResultRecycling.java Mon May  5 09:19:58 2008
@@ -1,31 +1,24 @@
 package org.apache.pig.test;
 
-import static org.apache.pig.PigServer.ExecType ;
-
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.PrintStream;
 import java.util.Iterator;
 
-import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
 import org.junit.Test;
 
-import junit.framework.TestCase;
-
-public class TestMapReduceResultRecycling extends TestCase {
-    MiniCluster cluster = MiniCluster.buildCluster();
+public class TestMapReduceResultRecycling extends PigExecTestCase {
     
     @Test
     public void testPlanRecycling() throws Throwable {
-        PigServer pig = new PigServer(ExecType.MAPREDUCE);
         File tmpFile = this.createTempFile();
         {            
             String query = "a = load 'file:" + tmpFile + "'; " ;
             System.out.println(query);
-            pig.registerQuery(query);
-            pig.explain("a", System.out) ;
-            Iterator<Tuple> it = pig.openIterator("a");
+            pigServer.registerQuery(query);
+            pigServer.explain("a", System.out) ;
+            Iterator<Tuple> it = pigServer.openIterator("a");
             assertTrue(it.next().getAtomField(0).strval().equals("a1")) ;
             assertTrue(it.next().getAtomField(0).strval().equals("b1")) ;
             assertTrue(it.next().getAtomField(0).strval().equals("c1")) ;
@@ -35,9 +28,9 @@
         {
             String query = "b = filter a by $0 eq 'a1';" ;
             System.out.println(query);
-            pig.registerQuery(query);
-            pig.explain("b", System.out) ;
-            Iterator<Tuple> it = pig.openIterator("b");
+            pigServer.registerQuery(query);
+            pigServer.explain("b", System.out) ;
+            Iterator<Tuple> it = pigServer.openIterator("b");
             assertTrue(it.next().getAtomField(0).strval().equals("a1")) ;
             assertFalse(it.hasNext()) ;
         }
@@ -45,9 +38,9 @@
         {
             String query = "c = filter a by $0 eq 'b1';" ;
             System.out.println(query);
-            pig.registerQuery(query);
-            pig.explain("c", System.out) ;
-            Iterator<Tuple> it = pig.openIterator("c");
+            pigServer.registerQuery(query);
+            pigServer.explain("c", System.out) ;
+            Iterator<Tuple> it = pigServer.openIterator("c");
             assertTrue(it.next().getAtomField(0).strval().equals("b1")) ;
             assertFalse(it.hasNext()) ;
         }

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestOrderBy.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestOrderBy.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestOrderBy.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestOrderBy.java Mon May  5 09:19:58 2008
@@ -30,13 +30,12 @@
 import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
 import static org.apache.pig.PigServer.ExecType.MAPREDUCE;
+import static org.apache.pig.PigServer.ExecType.LOCAL;
 
-public class TestOrderBy extends TestCase {
+public class TestOrderBy extends PigExecTestCase {
     private static final int DATALEN = 1024;
     private String[][] DATA = new String[2][DATALEN];
-    MiniCluster cluster = MiniCluster.buildCluster();
-    
-    private PigServer pig;
+
     private File tmpFile;
 
     public TestOrderBy() throws Throwable {
@@ -45,7 +44,6 @@
             DATA[0][i] = myFormatter.format(i);
             DATA[1][i] = myFormatter.format(DATALEN - i - 1);
         }
-        pig = new PigServer(MAPREDUCE, cluster.getProperties());
     }
     
     protected void setUp() throws Exception {
@@ -60,10 +58,10 @@
     protected void tearDown() throws Exception {
         tmpFile.delete();
     }
-    
+
     private void verify(String query, boolean descending) throws Exception {
-        pig.registerQuery(query);
-        Iterator<Tuple> it = pig.openIterator("myid");
+        pigServer.registerQuery(query);
+        Iterator<Tuple> it = pigServer.openIterator("myid");
         int col = (descending ? 1 : 0);
         for(int i = 0; i < DATALEN; i++) {
             Tuple t = (Tuple)it.next();

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestParser.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestParser.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestParser.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestParser.java Mon May  5 09:19:58 2008
@@ -1,20 +1,18 @@
 package org.apache.pig.test;
 
-import static org.apache.pig.PigServer.ExecType.MAPREDUCE;
-
 import java.io.IOException;
 
-import junit.framework.TestCase;
-
-import org.apache.pig.PigServer;
+import org.apache.pig.PigServer.ExecType;
 import org.apache.pig.backend.executionengine.ExecException;
 
-public class TestParser extends TestCase {
+public class TestParser extends PigExecTestCase {
 
     public void testLoadingNonexistentFile() throws ExecException, IOException {
-        PigServer pig = new PigServer(MAPREDUCE);
         try {
-            pig.registerQuery("vals = load 'nonexistentfile';");
+            // FIXME : this should be tested in all modes
+            if(execType == ExecType.LOCAL)
+                return;
+            pigServer.registerQuery("vals = load 'nonexistentfile';");
             fail("Loading a  nonexistent file should throw an IOException at parse time");
         } catch (IOException io) {
         }

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestPigServer.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestPigServer.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestPigServer.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestPigServer.java Mon May  5 09:19:58 2008
@@ -18,16 +18,7 @@
 import junit.framework.TestCase;
 
 
-public class TestPigServer extends TestCase {
-    private PigServer pig = null;
-    MiniCluster cluster = MiniCluster.buildCluster();
-    
-    private void initPigServer() throws Throwable {
-        if (pig == null) {
-            pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
-        }
-    }
-    
+public class TestPigServer extends PigExecTestCase {
     
     private final static String FILE_SEPARATOR = System.getProperty("file.separator");
     
@@ -94,18 +85,17 @@
         String jarName = "BadFileNameTestJarNotPresent.jar";
         
         // jar name is not present to start with
-        initPigServer();
-        verifyStringContained(pig.getPigContext().extraJars, jarName, false);
+        verifyStringContained(pigServer.getPigContext().extraJars, jarName, false);
 
         boolean exceptionRaised = false;
         try {
-            pig.registerJar(jarName);
+            pigServer.registerJar(jarName);
         }
         catch (IOException e) {
             exceptionRaised = true;
         }        
         assertTrue(exceptionRaised);
-        verifyStringContained(pig.getPigContext().extraJars, jarName, false);
+        verifyStringContained(pigServer.getPigContext().extraJars, jarName, false);
     }
 
     /**
@@ -120,21 +110,19 @@
                               dir2 + FILE_SEPARATOR;
         String jarName = "TestRegisterJarLocal.jar";
         
-        initPigServer();
-        
         createFakeJarFile(jarLocation, jarName);
         
-        verifyStringContained(pig.getPigContext().extraJars, jarName, false);
+        verifyStringContained(pigServer.getPigContext().extraJars, jarName, false);
         
         boolean exceptionRaised = false;
         try {
-            pig.registerJar(jarLocation + jarName);
+            pigServer.registerJar(jarLocation + jarName);
         }
         catch (IOException e) {
             exceptionRaised = true;
         }        
         assertFalse(exceptionRaised);
-        verifyStringContained(pig.getPigContext().extraJars, jarName, true);
+        verifyStringContained(pigServer.getPigContext().extraJars, jarName, true);
 
         // clean-up
         assertTrue((new File(jarLocation + jarName)).delete());
@@ -156,25 +144,23 @@
         String jarLocation1 = dir + FILE_SEPARATOR + subDir1 + FILE_SEPARATOR;
         String jarLocation2 = dir + FILE_SEPARATOR + subDir2 + FILE_SEPARATOR;
         
-        initPigServer();
-        
         createFakeJarFile(jarLocation1, jarName);
         createFakeJarFile(jarLocation2, jarName);
         
-        verifyStringContained(pig.getPigContext().extraJars, jarName, false);
+        verifyStringContained(pigServer.getPigContext().extraJars, jarName, false);
         
         registerNewResource(jarLocation1);
         registerNewResource(jarLocation2);
         
         boolean exceptionRaised = false;
         try {
-            pig.registerJar(jarName);
+            pigServer.registerJar(jarName);
         }
         catch (IOException e) {
             exceptionRaised = true;
         }
         assertFalse(exceptionRaised);
-        verifyStringContained(pig.getPigContext().extraJars, jarName, true);
+        verifyStringContained(pigServer.getPigContext().extraJars, jarName, true);
 
         // clean-up
         assertTrue((new File(jarLocation1 + jarName)).delete());
@@ -197,8 +183,6 @@
         String className = "TestRegisterJar";
         String javaSrc = "package " + subDir + "; class " + className + " { }";
 
-        initPigServer();
-        
         // create dirs
         (new File(dir + FILE_SEPARATOR + subDir)).mkdirs();
 
@@ -234,7 +218,7 @@
         // load the specific resource
         boolean exceptionRaised = false;
         try {
-            pig.registerJar("sub_dir/TestRegisterJar.class");
+            pigServer.registerJar("sub_dir/TestRegisterJar.class");
         }
         catch (IOException e) {
             exceptionRaised = true;
@@ -242,7 +226,7 @@
         
         // verify proper jar file is located
         assertFalse(exceptionRaised);
-        verifyStringContained(pig.getPigContext().extraJars, jarName, true);
+        verifyStringContained(pigServer.getPigContext().extraJars, jarName, true);
 
         // clean up Jar file and test dir
         (new File(dir + FILE_SEPARATOR + jarName)).delete();

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestPigSplit.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestPigSplit.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestPigSplit.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestPigSplit.java Mon May  5 09:19:58 2008
@@ -18,46 +18,28 @@
 
 package org.apache.pig.test;
 
-import static org.apache.pig.PigServer.ExecType.MAPREDUCE;
-
 import java.io.File;
-import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.Iterator;
 
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
-import org.apache.pig.backend.executionengine.ExecException;
-
-import junit.framework.TestCase;
+import org.junit.Test;
 
-public class TestPigSplit extends TestCase {
-	PigServer pig;
-	MiniCluster cluster = MiniCluster.buildCluster();
-	
-	@Override
-	@Before
-	protected void setUp() throws Exception {
-		super.setUp();
+public class TestPigSplit extends PigExecTestCase {
 		
-		pig = new PigServer(MAPREDUCE, cluster.getProperties());
-	}
 	@Test
-	public void testLongEvalSpec() throws Exception{
+	public void notestLongEvalSpec() throws Exception{
 		File f = File.createTempFile("tmp", "");
 		
 		PrintWriter pw = new PrintWriter(f);
 		pw.println("0\ta");
 		pw.close();
 		
-		pig.registerQuery("a = load 'file:" + f + "';");
+		pigServer.registerQuery("a = load 'file:" + f + "';");
 		for (int i=0; i< 500; i++){
-			pig.registerQuery("a = filter a by $0 == '1';");
+			pigServer.registerQuery("a = filter a by $0 == '1';");
 		}
-		Iterator<Tuple> iter = pig.openIterator("a");
+		Iterator<Tuple> iter = pigServer.openIterator("a");
 		while (iter.hasNext()){
 			throw new Exception();
 		}
@@ -73,20 +55,20 @@
         pw.println("12");
         pw.println("42");
         pw.close();
-        pig.registerQuery("a = load 'file:" + f + "' as (value);");
-        pig.registerQuery("split a into b if value < 20, c if value > 10;");
-        pig.registerQuery("b1 = order b by value;");
-        pig.registerQuery("c1 = order c by value;");
+        pigServer.registerQuery("a = load 'file:" + f + "' as (value);");
+        pigServer.registerQuery("split a into b if value < 20, c if value > 10;");
+        pigServer.registerQuery("b1 = order b by value;");
+        pigServer.registerQuery("c1 = order c by value;");
 
         // order in lexicographic, so 12 comes before 2
-        Iterator<Tuple> iter = pig.openIterator("b1");
+        Iterator<Tuple> iter = pigServer.openIterator("b1");
         assertTrue("b1 has an element", iter.hasNext());
         assertEquals("first item in b1", iter.next().getAtomField(0).longVal(), 12);
         assertTrue("b1 has an element", iter.hasNext());
         assertEquals("second item in b1", iter.next().getAtomField(0).longVal(), 2);
         assertFalse("b1 is over", iter.hasNext());
 
-        iter = pig.openIterator("c1");
+        iter = pigServer.openIterator("c1");
         assertTrue("c1 has an element", iter.hasNext());
         assertEquals("first item in b1", iter.next().getAtomField(0).longVal(), 12);
         assertTrue("c1 has an element", iter.hasNext());

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestReversibleLoadStore.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestReversibleLoadStore.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestReversibleLoadStore.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestReversibleLoadStore.java Mon May  5 09:19:58 2008
@@ -20,27 +20,11 @@
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.io.BufferedPositionedInputStream;
 
-public class TestReversibleLoadStore extends TestCase {
+public class TestReversibleLoadStore extends PigExecTestCase {
 
     static List<Tuple> _storedTuples = new ArrayList<Tuple>();
-
-    public void testLocalNoReuse() throws Exception {
-        runNoReuseTest(ExecType.LOCAL) ;
-    }
-    
-    public void testMapReduceNoReuse() throws Exception {
-        runNoReuseTest(ExecType.MAPREDUCE) ;
-    }
-    
-    public void testLocalReuse() throws Exception {
-        runReuseTest(ExecType.LOCAL) ;
-    }
-    
-    public void testMapReduceReuse() throws Exception {
-        runReuseTest(ExecType.MAPREDUCE) ;
-    }
     
-    public void runNoReuseTest(ExecType runType) throws Exception {
+    public void testNoReuse() throws Exception {
         
         DummyLoadFunc.readCounterMap = null ;
         DummyStoreFunc.writeCounter = 0 ;     
@@ -76,7 +60,7 @@
         
     }
     
-    public void runReuseTest(ExecType runType) throws Exception {
+    public void testReuse() throws Exception {
         
         DummyLoadStoreFunc.readCounterMap = null ;
         DummyLoadStoreFunc.writeCounter = 0 ;     

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestStore.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestStore.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestStore.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestStore.java Mon May  5 09:19:58 2008
@@ -30,22 +30,19 @@
 
 import junit.framework.TestCase;
 
-public class TestStore extends TestCase {
+public class TestStore extends PigExecTestCase {
 
 	private int LOOP_COUNT = 1024;
-	MiniCluster cluster = MiniCluster.buildCluster();
-	
 	String fileName;
 	String tmpFile1, tmpFile2;
-	PigServer pig;
 
 	public void testSingleStore() throws Exception{
-		pig.registerQuery("A = load " + fileName + ";");
+		pigServer.registerQuery("A = load " + fileName + ";");
 		
-		pig.store("A", tmpFile1);
+		pigServer.store("A", tmpFile1);
 		
-		pig.registerQuery("B = load " + tmpFile1 + ";");
-		Iterator<Tuple> iter  = pig.openIterator("B");
+		pigServer.registerQuery("B = load " + tmpFile1 + ";");
+		Iterator<Tuple> iter  = pigServer.openIterator("B");
 		
 		int i =0;
 		while (iter.hasNext()){
@@ -57,14 +54,14 @@
 	}
 	
 	public void testMultipleStore() throws Exception{
-		pig.registerQuery("A = load " + fileName + ";");
+		pigServer.registerQuery("A = load " + fileName + ";");
 		
-		pig.store("A", tmpFile1);
+		pigServer.store("A", tmpFile1);
 		
-		pig.registerQuery("B = foreach (group A by $0) generate $0, SUM($1);");
-		pig.store("B", tmpFile2);
-		pig.registerQuery("C = load " + tmpFile2 + ";");
-		Iterator<Tuple> iter  = pig.openIterator("C");
+		pigServer.registerQuery("B = foreach (group A by $0) generate $0, SUM($1);");
+		pigServer.store("B", tmpFile2);
+		pigServer.registerQuery("C = load " + tmpFile2 + ";");
+		Iterator<Tuple> iter  = pigServer.openIterator("C");
 		
 		int i =0;
 		while (iter.hasNext()){
@@ -78,14 +75,14 @@
 	}
 	
 	public void testStoreWithMultipleMRJobs() throws Exception{
-		pig.registerQuery("A = load " + fileName + ";");		
-		pig.registerQuery("B = foreach (group A by $0) generate $0, SUM($1);");
-		pig.registerQuery("C = foreach (group B by $0) generate $0, SUM($1);");
-		pig.registerQuery("D = foreach (group C by $0) generate $0, SUM($1);");
-
-		pig.store("D", tmpFile2);
-		pig.registerQuery("E = load " + tmpFile2 + ";");
-		Iterator<Tuple> iter  = pig.openIterator("E");
+		pigServer.registerQuery("A = load " + fileName + ";");		
+		pigServer.registerQuery("B = foreach (group A by $0) generate $0, SUM($1);");
+		pigServer.registerQuery("C = foreach (group B by $0) generate $0, SUM($1);");
+		pigServer.registerQuery("D = foreach (group C by $0) generate $0, SUM($1);");
+
+		pigServer.store("D", tmpFile2);
+		pigServer.registerQuery("E = load " + tmpFile2 + ";");
+		Iterator<Tuple> iter  = pigServer.openIterator("E");
 		
 		int i =0;
 		while (iter.hasNext()){
@@ -107,18 +104,10 @@
 			pw.println(i + "\t" + i);
 		}
 		pw.close();
-		try {
-		    pig = new PigServer(MAPREDUCE, cluster.getProperties());
-		}
-		catch (ExecException e) {
-			IOException ioe = new IOException("Failed to create Pig Server");
-			ioe.initCause(e);
-		    throw ioe;
-		}
 		
-		fileName = "'" + FileLocalizer.hadoopify(f.toString(), pig.getPigContext()) + "'";
-		tmpFile1 = "'" + FileLocalizer.getTemporaryPath(null, pig.getPigContext()).toString() + "'";
-		tmpFile2 = "'" + FileLocalizer.getTemporaryPath(null, pig.getPigContext()).toString() + "'";
+		fileName = "'" + FileLocalizer.hadoopify(f.toString(), pigServer.getPigContext()) + "'";
+		tmpFile1 = "'" + FileLocalizer.getTemporaryPath(null, pigServer.getPigContext()).toString() + "'";
+		tmpFile2 = "'" + FileLocalizer.getTemporaryPath(null, pigServer.getPigContext()).toString() + "'";
 		f.delete();
 	}
 	

Modified: incubator/pig/trunk/test/org/apache/pig/test/TestStreaming.java
URL: http://svn.apache.org/viewvc/incubator/pig/trunk/test/org/apache/pig/test/TestStreaming.java?rev=653524&r1=653523&r2=653524&view=diff
==============================================================================
--- incubator/pig/trunk/test/org/apache/pig/test/TestStreaming.java (original)
+++ incubator/pig/trunk/test/org/apache/pig/test/TestStreaming.java Mon May  5 09:19:58 2008
@@ -22,24 +22,15 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.pig.PigServer;
 import org.apache.pig.PigServer.ExecType;
-import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.builtin.PigStorage;
-import org.apache.pig.data.*;
+import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.io.BufferedPositionedInputStream;
 import org.apache.pig.impl.io.FileLocalizer;
+import org.junit.Assert;
+import org.junit.Test;
 
-import static org.apache.pig.PigServer.ExecType.MAPREDUCE;
-
-import junit.framework.TestCase;
-
-public class TestStreaming extends TestCase {
-
-    MiniCluster cluster = MiniCluster.buildCluster();
+public class TestStreaming extends PigExecTestCase {
 
 	private static final String simpleEchoStreamingCommand = 
 		"perl -ne 'chomp $_; print \"$_\n\"'";
@@ -58,18 +49,8 @@
 	}
 	
 	@Test
-	public void testLocalSimpleMapSideStreaming() throws Exception {
-	    testSimpleMapSideStreaming(ExecType.LOCAL);
-	}
-	
-	@Test
-    public void testMRSimpleMapSideStreaming() throws Exception {
-        testSimpleMapSideStreaming(ExecType.MAPREDUCE);
-    }
-    
-	private void testSimpleMapSideStreaming(ExecType execType) 
+	public void testSimpleMapSideStreaming() 
 	throws Exception {
-	        PigServer pigServer = createPigServer(execType);
 		File input = Util.createInputFile("tmp", "", 
 				                          new String[] {"A,1", "B,2", "C,3", "D,2",
 				                                        "A,5", "B,5", "C,8", "A,8",
@@ -93,33 +74,10 @@
 		// Run the query and check the results
 		Util.checkQueryOutputs(pigServer.openIterator("OP"), expectedResults);
 	}
-
-    private PigServer createPigServer(ExecType execType) throws ExecException {
-        PigServer pigServer; 
-        if (execType == ExecType.MAPREDUCE) {
-            pigServer = new PigServer(execType, cluster.getProperties());
-        } else {
-            pigServer = new PigServer(execType);
-        }
-        return pigServer;
-    }
-
-	@Test
-    public void testLocalSimpleMapSideStreamingWithOutputSchema() 
-	throws Exception {
-	    testSimpleMapSideStreamingWithOutputSchema(ExecType.LOCAL);
-	}
-	
-    @Test
-    public void testMRSimpleMapSideStreamingWithOutputSchema() 
-    throws Exception {
-        testSimpleMapSideStreamingWithOutputSchema(ExecType.MAPREDUCE);
-    }
     
-	private void testSimpleMapSideStreamingWithOutputSchema(ExecType execType) 
+	@Test
+	public void testSimpleMapSideStreamingWithOutputSchema() 
 	throws Exception {
-		PigServer pigServer = createPigServer(execType);
-
 		File input = Util.createInputFile("tmp", "", 
 				                          new String[] {"A,1", "B,2", "C,3", "D,2",
 				                                        "A,5", "B,5", "C,8", "A,8",
@@ -144,21 +102,8 @@
 	}
 
 	@Test
-    public void testLocalSimpleReduceSideStreamingAfterFlatten() 
+	public void testSimpleReduceSideStreamingAfterFlatten() 
 	throws Exception {
-	    testSimpleReduceSideStreamingAfterFlatten(ExecType.LOCAL);
-	}
-	
-    @Test
-    public void testMRSimpleReduceSideStreamingAfterFlatten() 
-    throws Exception {
-        testSimpleReduceSideStreamingAfterFlatten(ExecType.MAPREDUCE);
-    }
-    
-	private void testSimpleReduceSideStreamingAfterFlatten(ExecType execType) 
-	throws Exception {
-		PigServer pigServer = createPigServer(execType);
-
 		File input = Util.createInputFile("tmp", "", 
 				                          new String[] {"A,1", "B,2", "C,3", "D,2",
 				                                        "A,5", "B,5", "C,8", "A,8",
@@ -186,22 +131,9 @@
 		Util.checkQueryOutputs(pigServer.openIterator("OP"), expectedResults);
 	}
 
-	@Test
-    public void testLocalSimpleOrderedReduceSideStreamingAfterFlatten() 
-	throws Exception {
-	    testSimpleOrderedReduceSideStreamingAfterFlatten(ExecType.LOCAL);
-	}
-	
     @Test
-    public void testMRSimpleOrderedReduceSideStreamingAfterFlatten() 
-    throws Exception {
-        testSimpleOrderedReduceSideStreamingAfterFlatten(ExecType.MAPREDUCE);
-    }
-    
-	private void testSimpleOrderedReduceSideStreamingAfterFlatten(
+	public void testSimpleOrderedReduceSideStreamingAfterFlatten(
 	        ExecType execType) throws Exception {
-	    PigServer pigServer = createPigServer(execType);
-
 		File input = Util.createInputFile("tmp", "", 
 				                          new String[] {"A,1,2,3", "B,2,4,5",
 				                                        "C,3,1,2", "D,2,5,2",
@@ -250,8 +182,9 @@
 
     @Test
     public void testInputShipSpecs() throws Exception {
-        PigServer pigServer = new PigServer(MAPREDUCE, cluster.getProperties());
-
+        // FIXME : this should be tested in all modes
+        if(execType == ExecType.LOCAL)
+            return;
         File input = Util.createInputFile("tmp", "", 
                                           new String[] {"A,1", "B,2", "C,3", 
                                                         "D,2", "A,5", "B,5", 
@@ -309,8 +242,9 @@
 
     @Test
 	public void testOutputShipSpecs() throws Exception {
-	    PigServer pigServer = new PigServer(MAPREDUCE, cluster.getProperties());
-
+        // FIXME : this should be tested in all modes
+        if(execType == ExecType.LOCAL)
+            return;
 	    File input = Util.createInputFile("tmp", "", 
 	                                      new String[] {"A,1", "B,2", "C,3", 
 	                                                    "D,2", "A,5", "B,5", 
@@ -370,8 +304,9 @@
 
     @Test
     public void testInputOutputSpecs() throws Exception {
-        PigServer pigServer = new PigServer(MAPREDUCE, cluster.getProperties());
-
+        // FIXME : this should be tested in all modes
+        if(execType == ExecType.LOCAL)
+            return;
         File input = Util.createInputFile("tmp", "", 
                                           new String[] {"A,1", "B,2", "C,3", 
                                                         "D,2", "A,5", "B,5", 
@@ -436,19 +371,8 @@
     }
 
     @Test
-    public void testLocalSimpleMapSideStreamingWithUnixPipes() 
-    throws Exception {
-        testSimpleMapSideStreamingWithUnixPipes(ExecType.LOCAL);
-    }
-    
-    @Test
-    public void testMRSimpleMapSideStreamingWithUnixPipes() throws Exception {
-        testSimpleMapSideStreamingWithUnixPipes(ExecType.MAPREDUCE);
-    }
-    
-    private void testSimpleMapSideStreamingWithUnixPipes(ExecType execType) 
+    public void testSimpleMapSideStreamingWithUnixPipes() 
     throws Exception {
-        PigServer pigServer = createPigServer(execType);
         File input = Util.createInputFile("tmp", "", 
                                           new String[] {"A,1", "B,2", "C,3", "D,2",
                                                         "A,5", "B,5", "C,8", "A,8",
@@ -484,7 +408,6 @@
     
     private void testNegativeLoadStoreOptimization(ExecType execType) 
     throws Exception {
-            PigServer pigServer = createPigServer(execType);
         File input = Util.createInputFile("tmp", "", 
                                           new String[] {"A,1", "B,2", "C,3", "D,2",
                                                         "A,5", "B,5", "C,8", "A,8",