You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by rd...@apache.org on 2011/05/04 02:11:51 UTC

svn commit: r1099299 [2/2] - in /pig/trunk: CHANGES.txt test/org/apache/pig/test/TestMultiQuery.java test/org/apache/pig/test/TestMultiQueryBasic.java test/org/apache/pig/test/Util.java

Modified: pig/trunk/test/org/apache/pig/test/TestMultiQueryBasic.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestMultiQueryBasic.java?rev=1099299&r1=1099298&r2=1099299&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestMultiQueryBasic.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestMultiQueryBasic.java Wed May  4 00:11:51 2011
@@ -22,14 +22,12 @@ import static org.junit.Assert.assertFal
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
-import java.io.FileWriter;
 import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringReader;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 
 import junit.framework.Assert;
 
@@ -50,8 +48,7 @@ import org.apache.pig.StoreFunc;
 import org.apache.pig.backend.executionengine.ExecJob;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.Tuple;
-import org.apache.pig.impl.io.FileLocalizer;
-import org.apache.pig.tools.grunt.GruntParser;
+import org.apache.pig.impl.PigContext;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -59,452 +56,380 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
+
 @RunWith(JUnit4.class)
 public class TestMultiQueryBasic {
 
-    private static final MiniCluster cluster = MiniCluster.buildCluster();
-
-    private PigServer myPig;
+    private static PigServer myPig;
 
     @BeforeClass
-    public static void setUpBeforeClass() throws IOException {
-        Util.copyFromLocalToCluster(cluster,
+    public static void setUpBeforeClass() throws Exception {
+        Util.copyFromLocalToLocal(
                 "test/org/apache/pig/test/data/passwd", "passwd");
-        Util.copyFromLocalToCluster(cluster,
+        Util.copyFromLocalToLocal(
                 "test/org/apache/pig/test/data/passwd2", "passwd2");
+        Properties props = new Properties();
+        props.setProperty("opt.multiquery", ""+true);
+        myPig = new PigServer(ExecType.LOCAL, props);
     }
     
     @AfterClass
-    public static void tearDownAfterClass() throws IOException {
-        Util.deleteFile(cluster, "passwd");
-        Util.deleteFile(cluster, "passwd2");
-        cluster.shutDown();
+    public static void tearDownAfterClass() throws Exception {
+        Util.deleteFile(new PigContext(ExecType.LOCAL, new Properties()), "passwd");
+        Util.deleteFile(new PigContext(ExecType.LOCAL, new Properties()), "passwd2");
+        deleteOutputFiles();
     }
     
     @Before
     public void setUp() throws Exception {
-        cluster.setProperty("opt.multiquery", ""+true);
-        myPig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         deleteOutputFiles();
     }
 
     @After
     public void tearDown() throws Exception {
-        myPig = null;
+        
     }
     
  
     @Test
-    public void testMultiQueryWithTwoStores2() {
+    public void testMultiQueryWithTwoStores2() throws Exception {
 
         System.out.println("===== multi-query with 2 stores (2) =====");
 
-        try {
-            myPig.setBatchOn();
-
-            myPig.registerQuery("a = load 'passwd' " +
-                                "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int,gid:int);");
-            myPig.registerQuery("b = filter a by uid > 5;");
-            myPig.registerQuery("store b into '/tmp/output1';");
-            myPig.registerQuery("c = group b by gid;");
-            myPig.registerQuery("store c into '/tmp/output2';");
+        myPig.setBatchOn();
 
-            List<ExecJob> jobs = myPig.executeBatch();
-            assertTrue(jobs.size() == 2);
+        myPig.registerQuery("a = load 'passwd' " +
+                            "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int,gid:int);");
+        myPig.registerQuery("b = filter a by uid > 5;");
+        myPig.registerQuery("store b into 'output1';");
+        myPig.registerQuery("c = group b by gid;");
+        myPig.registerQuery("store c into 'output2';");
 
-            for (ExecJob job : jobs) {
-                assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
-            }
+        List<ExecJob> jobs = myPig.executeBatch();
+        assertTrue(jobs.size() == 2);
 
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        for (ExecJob job : jobs) {
+            assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
+        }
     }
 
     @Test
-    public void testMultiQueryWithTwoLoads2() {
+    public void testMultiQueryWithTwoLoads2() throws Exception {
 
         System.out.println("===== multi-query with two loads (2) =====");
 
-        try {
-            myPig.setBatchOn();
+        myPig.setBatchOn();
+
+        myPig.registerQuery("a = load 'passwd' " +
+                            "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int,gid:int);");
+        myPig.registerQuery("b = load 'passwd2' " +
+                            "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int,gid:int);");
+        myPig.registerQuery("c = filter a by uid > 5;");
+        myPig.registerQuery("d = filter b by uid > 10;");
+        myPig.registerQuery("store c into 'output1';");
+        myPig.registerQuery("store d into 'output2';");
+        myPig.registerQuery("e = cogroup c by uid, d by uid;");
+        myPig.registerQuery("store e into 'output3';");
 
-            myPig.registerQuery("a = load 'passwd' " +
-                                "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int,gid:int);");
-            myPig.registerQuery("b = load 'passwd2' " +
-                                "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int,gid:int);");
-            myPig.registerQuery("c = filter a by uid > 5;");
-            myPig.registerQuery("d = filter b by uid > 10;");
-            myPig.registerQuery("store c into '/tmp/output1';");
-            myPig.registerQuery("store d into '/tmp/output2';");
-            myPig.registerQuery("e = cogroup c by uid, d by uid;");
-            myPig.registerQuery("store e into '/tmp/output3';");
-
-            myPig.executeBatch();
-            myPig.discardBatch();
-
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        myPig.executeBatch();
+        myPig.discardBatch();
     }       
     
     @Test
-    public void testMultiQueryPhase3BaseCase2() {
+    public void testMultiQueryPhase3BaseCase2() throws Exception {
 
         System.out.println("===== multi-query phase 3 base case (2) =====");
 
-        try {
-            myPig.setBatchOn();
+        myPig.setBatchOn();
 
-            myPig.registerQuery("a = load 'passwd' " +
-                                 "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
-            myPig.registerQuery("b = filter a by uid < 5;");
-            myPig.registerQuery("c = filter a by uid >= 5 and uid < 10;");
-            myPig.registerQuery("d = filter a by uid >= 10;");
-            myPig.registerQuery("b1 = group b by gid;");
-            myPig.registerQuery("b2 = foreach b1 generate group, COUNT(b.uid);");
-            myPig.registerQuery("b3 = filter b2 by $1 > 5;");
-            myPig.registerQuery("store b3 into '/tmp/output1';");
-            myPig.registerQuery("c1 = group c by gid;");
-            myPig.registerQuery("c2 = foreach c1 generate group, SUM(c.uid);");
-            myPig.registerQuery("store c2 into '/tmp/output2';");
-            myPig.registerQuery("d1 = group d by gid;");
-            myPig.registerQuery("d2 = foreach d1 generate group, AVG(d.uid);");            
-            myPig.registerQuery("store d2 into '/tmp/output3';");
-             
-            List<ExecJob> jobs = myPig.executeBatch();
-            
-            for (ExecJob job : jobs) {
-                assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
-            }
-            
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        myPig.registerQuery("a = load 'passwd' " +
+                             "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
+        myPig.registerQuery("b = filter a by uid < 5;");
+        myPig.registerQuery("c = filter a by uid >= 5 and uid < 10;");
+        myPig.registerQuery("d = filter a by uid >= 10;");
+        myPig.registerQuery("b1 = group b by gid;");
+        myPig.registerQuery("b2 = foreach b1 generate group, COUNT(b.uid);");
+        myPig.registerQuery("b3 = filter b2 by $1 > 5;");
+        myPig.registerQuery("store b3 into 'output1';");
+        myPig.registerQuery("c1 = group c by gid;");
+        myPig.registerQuery("c2 = foreach c1 generate group, SUM(c.uid);");
+        myPig.registerQuery("store c2 into 'output2';");
+        myPig.registerQuery("d1 = group d by gid;");
+        myPig.registerQuery("d2 = foreach d1 generate group, AVG(d.uid);");            
+        myPig.registerQuery("store d2 into 'output3';");
+         
+        List<ExecJob> jobs = myPig.executeBatch();
+        
+        for (ExecJob job : jobs) {
+            assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
+        }
     }         
     
     @Test
-    public void testMultiQueryPhase3WithoutCombiner2() {
+    public void testMultiQueryPhase3WithoutCombiner2() throws Exception {
 
         System.out.println("===== multi-query phase 3 without combiner (2) =====");
 
-        try {
-            myPig.setBatchOn();
+        myPig.setBatchOn();
 
-            myPig.registerQuery("a = load 'passwd' " +
-                                 "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
-            myPig.registerQuery("b = filter a by uid < 5;");
-            myPig.registerQuery("c = filter a by uid >= 5 and uid < 10;");
-            myPig.registerQuery("d = filter a by uid >= 10;");
-            myPig.registerQuery("b1 = group b by gid;");
-            myPig.registerQuery("b2 = foreach b1 generate group, COUNT(b.uid) + SUM(b.uid);");
-            myPig.registerQuery("b3 = filter b2 by $1 > 5;");
-            myPig.registerQuery("store b3 into '/tmp/output1';");
-            myPig.registerQuery("c1 = group c by gid;");
-            myPig.registerQuery("c2 = foreach c1 generate group, SUM(c.uid) - COUNT(c.uid);");
-            myPig.registerQuery("store c2 into '/tmp/output2';");
-            myPig.registerQuery("d1 = group d by gid;");           
-            myPig.registerQuery("d2 = foreach d1 generate group, MAX(d.uid) - MIN(d.uid);");
-            myPig.registerQuery("store d2 into '/tmp/output3';");
-             
-            List<ExecJob> jobs = myPig.executeBatch();
+        myPig.registerQuery("a = load 'passwd' " +
+                             "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
+        myPig.registerQuery("b = filter a by uid < 5;");
+        myPig.registerQuery("c = filter a by uid >= 5 and uid < 10;");
+        myPig.registerQuery("d = filter a by uid >= 10;");
+        myPig.registerQuery("b1 = group b by gid;");
+        myPig.registerQuery("b2 = foreach b1 generate group, COUNT(b.uid) + SUM(b.uid);");
+        myPig.registerQuery("b3 = filter b2 by $1 > 5;");
+        myPig.registerQuery("store b3 into 'output1';");
+        myPig.registerQuery("c1 = group c by gid;");
+        myPig.registerQuery("c2 = foreach c1 generate group, SUM(c.uid) - COUNT(c.uid);");
+        myPig.registerQuery("store c2 into 'output2';");
+        myPig.registerQuery("d1 = group d by gid;");           
+        myPig.registerQuery("d2 = foreach d1 generate group, MAX(d.uid) - MIN(d.uid);");
+        myPig.registerQuery("store d2 into 'output3';");
+         
+        List<ExecJob> jobs = myPig.executeBatch();
 
-            for (ExecJob job : jobs) {
-                assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
-            }
-  
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        for (ExecJob job : jobs) {
+            assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
+        }
     }     
     
     @Test
-    public void testMultiQueryPhase3WithMixedCombiner2() {
+    public void testMultiQueryPhase3WithMixedCombiner2() throws Exception {
 
         System.out.println("===== multi-query phase 3 with mixed combiner (2) =====");
 
-        try {
-            myPig.setBatchOn();
+        myPig.setBatchOn();
 
-            myPig.registerQuery("a = load 'passwd' " +
-                                 "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
-            myPig.registerQuery("b = filter a by uid < 5;");
-            myPig.registerQuery("c = filter a by uid >= 5 and uid < 10;");
-            myPig.registerQuery("d = filter a by uid >= 10;");
-            myPig.registerQuery("b1 = group b by gid;");
-            myPig.registerQuery("b2 = foreach b1 generate group, COUNT(b.uid);");
-            myPig.registerQuery("b3 = filter b2 by $1 > 5;");
-            myPig.registerQuery("store b3 into '/tmp/output1';");
-            myPig.registerQuery("c1 = group c by gid;");
-            myPig.registerQuery("c2 = foreach c1 generate group, SUM(c.uid);");
-            myPig.registerQuery("store c2 into '/tmp/output2';");
-            myPig.registerQuery("d1 = group d by gid;");            
-            myPig.registerQuery("d2 = foreach d1 generate group, MAX(d.uid) - MIN(d.uid);");
-            myPig.registerQuery("store d2 into '/tmp/output3';");
-             
-            List<ExecJob> jobs = myPig.executeBatch();
-            assertEquals(3, jobs.size());
+        myPig.registerQuery("a = load 'passwd' " +
+                             "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
+        myPig.registerQuery("b = filter a by uid < 5;");
+        myPig.registerQuery("c = filter a by uid >= 5 and uid < 10;");
+        myPig.registerQuery("d = filter a by uid >= 10;");
+        myPig.registerQuery("b1 = group b by gid;");
+        myPig.registerQuery("b2 = foreach b1 generate group, COUNT(b.uid);");
+        myPig.registerQuery("b3 = filter b2 by $1 > 5;");
+        myPig.registerQuery("store b3 into 'output1';");
+        myPig.registerQuery("c1 = group c by gid;");
+        myPig.registerQuery("c2 = foreach c1 generate group, SUM(c.uid);");
+        myPig.registerQuery("store c2 into 'output2';");
+        myPig.registerQuery("d1 = group d by gid;");            
+        myPig.registerQuery("d2 = foreach d1 generate group, MAX(d.uid) - MIN(d.uid);");
+        myPig.registerQuery("store d2 into 'output3';");
+         
+        List<ExecJob> jobs = myPig.executeBatch();
+        assertEquals(3, jobs.size());
 
-            for (ExecJob job : jobs) {
-                assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
-            }
-            
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        for (ExecJob job : jobs) {
+            assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
+        }
     }         
 
     @Test
-    public void testMultiQueryPhase3WithDifferentMapDataTypes2() {
+    public void testMultiQueryPhase3WithDifferentMapDataTypes2() throws Exception {
 
         System.out.println("===== multi-query phase 3 with different map datatypes (2) =====");
 
-        try {
-            myPig.setBatchOn();
+        myPig.setBatchOn();
 
-            myPig.registerQuery("a = load 'passwd' " +
-                                 "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
-            myPig.registerQuery("b = filter a by uid < 5;");
-            myPig.registerQuery("c = filter a by uid >= 5 and uid < 10;");
-            myPig.registerQuery("d = filter a by uid >= 10;");
-            myPig.registerQuery("b1 = group b by gid;");
-            myPig.registerQuery("b2 = foreach b1 generate group, COUNT(b.uid);");
-            myPig.registerQuery("b3 = filter b2 by $1 > 5;");
-            myPig.registerQuery("store b3 into '/tmp/output1';");
-            myPig.registerQuery("c1 = group c by $1;");
-            myPig.registerQuery("c2 = foreach c1 generate group, SUM(c.uid);");
-            myPig.registerQuery("store c2 into '/tmp/output2';");
-            myPig.registerQuery("d1 = group d by $1;");
-            myPig.registerQuery("d2 = foreach d1 generate group, COUNT(d.uid);");
-            myPig.registerQuery("store d2 into '/tmp/output3';");
-             
-            List<ExecJob> jobs = myPig.executeBatch();
-            assertEquals(3, jobs.size());
+        myPig.registerQuery("a = load 'passwd' " +
+                             "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
+        myPig.registerQuery("b = filter a by uid < 5;");
+        myPig.registerQuery("c = filter a by uid >= 5 and uid < 10;");
+        myPig.registerQuery("d = filter a by uid >= 10;");
+        myPig.registerQuery("b1 = group b by gid;");
+        myPig.registerQuery("b2 = foreach b1 generate group, COUNT(b.uid);");
+        myPig.registerQuery("b3 = filter b2 by $1 > 5;");
+        myPig.registerQuery("store b3 into 'output1';");
+        myPig.registerQuery("c1 = group c by $1;");
+        myPig.registerQuery("c2 = foreach c1 generate group, SUM(c.uid);");
+        myPig.registerQuery("store c2 into 'output2';");
+        myPig.registerQuery("d1 = group d by $1;");
+        myPig.registerQuery("d2 = foreach d1 generate group, COUNT(d.uid);");
+        myPig.registerQuery("store d2 into 'output3';");
+         
+        List<ExecJob> jobs = myPig.executeBatch();
+        assertEquals(3, jobs.size());
 
-            for (ExecJob job : jobs) {
-                assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
-            }
-            
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        for (ExecJob job : jobs) {
+            assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
+        }
     }         
     
     @Test
-    public void testMultiQueryPhase3WithDifferentMapDataTypes3() {
+    public void testMultiQueryPhase3WithDifferentMapDataTypes3() throws Exception {
 
         System.out.println("===== multi-query phase 3 with different map datatypes (3) =====");
 
-        try {
-            myPig.setBatchOn();
-            String[] inputData = {"john\t20\t3.4",
-            		"john\t25\t3.4" ,
-            		"henry\t23\t3.9" ,
-            		"adam\t54\t2.9" ,
-            		"henry\t21\t3.9"};
-            Util.createInputFile(cluster, "queryInput.txt", inputData);
-
-            myPig.registerQuery("a = load 'queryInput.txt' " +
-                                 "as (name:chararray, age:int, gpa:double);");
-            myPig.registerQuery("b = group a all;");
-            myPig.registerQuery("c = foreach b generate group, COUNT(a);");
-            myPig.registerQuery("store c into 'foo';");
-            myPig.registerQuery("d = group a by (name, gpa);");
-            myPig.registerQuery("e = foreach d generate flatten(group), MIN(a.age);");
-            myPig.registerQuery("store e into 'bar';");
-             
-            myPig.executeBatch();
-            
-            myPig.registerQuery("a = load 'foo' as (grp:chararray, cnt:long) ;");
-            Iterator<Tuple> it = myPig.openIterator("a");
-            assertEquals(Util.getPigConstant("('all', 5l)"), it.next());
-            assertFalse(it.hasNext());
-            
-            myPig.registerQuery("a = load 'bar' as (name:chararray, gpa:double, age:int);");
-            it = myPig.openIterator("a");
-            int i = 0;
-            Map<String, Tuple> expectedResults = new HashMap<String, Tuple>();
-            expectedResults.put("john", (Tuple) Util.getPigConstant("('john',3.4,20)"));
-            expectedResults.put("adam", (Tuple) Util.getPigConstant("('adam',2.9,54)"));
-            expectedResults.put("henry", (Tuple) Util.getPigConstant("('henry',3.9,21)"));
-            while(it.hasNext()) {
-                Tuple t = it.next();
-                i++;
-                assertEquals(expectedResults.get(t.get(0)), t);
-            }
-            assertEquals(3, i);
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        myPig.setBatchOn();
+        String[] inputData = {"john\t20\t3.4",
+        		"john\t25\t3.4" ,
+        		"henry\t23\t3.9" ,
+        		"adam\t54\t2.9" ,
+        		"henry\t21\t3.9"};
+        Util.createLocalInputFile("queryInput.txt", inputData);
+
+        myPig.registerQuery("a = load 'queryInput.txt' " +
+                             "as (name:chararray, age:int, gpa:double);");
+        myPig.registerQuery("b = group a all;");
+        myPig.registerQuery("c = foreach b generate group, COUNT(a);");
+        myPig.registerQuery("store c into 'output1';");
+        myPig.registerQuery("d = group a by (name, gpa);");
+        myPig.registerQuery("e = foreach d generate flatten(group), MIN(a.age);");
+        myPig.registerQuery("store e into 'output2';");
+         
+        myPig.executeBatch();
+        
+        myPig.registerQuery("a = load 'output1' as (grp:chararray, cnt:long) ;");
+        Iterator<Tuple> it = myPig.openIterator("a");
+        assertEquals(Util.getPigConstant("('all', 5l)"), it.next());
+        assertFalse(it.hasNext());
+        
+        myPig.registerQuery("a = load 'output2' as (name:chararray, gpa:double, age:int);");
+        it = myPig.openIterator("a");
+        int i = 0;
+        Map<String, Tuple> expectedResults = new HashMap<String, Tuple>();
+        expectedResults.put("john", (Tuple) Util.getPigConstant("('john',3.4,20)"));
+        expectedResults.put("adam", (Tuple) Util.getPigConstant("('adam',2.9,54)"));
+        expectedResults.put("henry", (Tuple) Util.getPigConstant("('henry',3.9,21)"));
+        while(it.hasNext()) {
+            Tuple t = it.next();
+            i++;
+            assertEquals(expectedResults.get(t.get(0)), t);
+        }
+        assertEquals(3, i);
     }         
  
     @Test
-    public void testMultiQueryPhase3StreamingInReducer2() {
+    public void testMultiQueryPhase3StreamingInReducer2() throws Exception {
 
         System.out.println("===== multi-query phase 3 with streaming in reducer (2) =====");
 
-        try {
-            myPig.setBatchOn();
+        myPig.setBatchOn();
 
-            myPig.registerQuery("A = load 'passwd';");
-            myPig.registerQuery("Split A into A1 if $2 > 5, A2 if $2 >= 5;");
-            myPig.registerQuery("Split A1 into A3 if $0 > 'm', A4 if $0 >= 'm';");
-            myPig.registerQuery("B = group A3 by $2;");
-            myPig.registerQuery("C = foreach B generate flatten(A3);");
-            myPig.registerQuery("D = stream B through `cat`;");
-            myPig.registerQuery("store D into '/tmp/output1';");
-            myPig.registerQuery("E = group A4 by $2;");
-            myPig.registerQuery("F = foreach E generate group, COUNT(A4);");
-            myPig.registerQuery("store F into '/tmp/output2';");            
-            myPig.registerQuery("G = group A1 by $2;");
-            myPig.registerQuery("H = foreach G generate group, COUNT(A1);");          
-            myPig.registerQuery("store H into '/tmp/output3';");
-             
-            List<ExecJob> jobs = myPig.executeBatch();
-            assertEquals(3, jobs.size());
+        myPig.registerQuery("A = load 'passwd';");
+        myPig.registerQuery("Split A into A1 if $2 > 5, A2 if $2 >= 5;");
+        myPig.registerQuery("Split A1 into A3 if $0 > 'm', A4 if $0 >= 'm';");
+        myPig.registerQuery("B = group A3 by $2;");
+        myPig.registerQuery("C = foreach B generate flatten(A3);");
+        myPig.registerQuery("D = stream B through `cat`;");
+        myPig.registerQuery("store D into 'output1';");
+        myPig.registerQuery("E = group A4 by $2;");
+        myPig.registerQuery("F = foreach E generate group, COUNT(A4);");
+        myPig.registerQuery("store F into 'output2';");            
+        myPig.registerQuery("G = group A1 by $2;");
+        myPig.registerQuery("H = foreach G generate group, COUNT(A1);");          
+        myPig.registerQuery("store H into 'output3';");
+         
+        List<ExecJob> jobs = myPig.executeBatch();
+        assertEquals(3, jobs.size());
 
-            for (ExecJob job : jobs) {
-                assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
-            }
-            
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        for (ExecJob job : jobs) {
+            assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
+        }
     }       
     
     @Test
-    public void testMultiQueryWithPigMixL12_2() {
+    public void testMultiQueryWithPigMixL12_2() throws Exception {
 
         System.out.println("===== multi-query with PigMix L12 (2) =====");
 
-        try {
-            myPig.setBatchOn();
-
-            myPig.registerQuery("a = load 'passwd' " +
-                                "using PigStorage(':') as (uname, passwd, uid, gid);");
-            myPig.registerQuery("b = foreach a generate uname, passwd, uid, gid;");
-            myPig.registerQuery("split b into c1 if uid > 5, c2 if uid <= 5 ;"); 
-            myPig.registerQuery("split c1 into d1 if gid < 5, d2 if gid >= 5;");
-            myPig.registerQuery("e = group d1 by uname;");
-            myPig.registerQuery("e1 = foreach e generate group, MAX(d1.uid);");
-            myPig.registerQuery("store e1 into '/tmp/output1';");
-            myPig.registerQuery("f = group c2 by uname;");
-            myPig.registerQuery("f1 = foreach f generate group, SUM(c2.gid);");
-            myPig.registerQuery("store f1 into '/tmp/output2';");
-            myPig.registerQuery("g = group d2 by uname;");
-            myPig.registerQuery("g1 = foreach g generate group, COUNT(d2);");
-            myPig.registerQuery("store g1 into '/tmp/output3';");
+        myPig.setBatchOn();
 
-            List<ExecJob> jobs = myPig.executeBatch();
-            assertEquals(3, jobs.size());
+        myPig.registerQuery("a = load 'passwd' " +
+                            "using PigStorage(':') as (uname, passwd, uid, gid);");
+        myPig.registerQuery("b = foreach a generate uname, passwd, uid, gid;");
+        myPig.registerQuery("split b into c1 if uid > 5, c2 if uid <= 5 ;"); 
+        myPig.registerQuery("split c1 into d1 if gid < 5, d2 if gid >= 5;");
+        myPig.registerQuery("e = group d1 by uname;");
+        myPig.registerQuery("e1 = foreach e generate group, MAX(d1.uid);");
+        myPig.registerQuery("store e1 into 'output1';");
+        myPig.registerQuery("f = group c2 by uname;");
+        myPig.registerQuery("f1 = foreach f generate group, SUM(c2.gid);");
+        myPig.registerQuery("store f1 into 'output2';");
+        myPig.registerQuery("g = group d2 by uname;");
+        myPig.registerQuery("g1 = foreach g generate group, COUNT(d2);");
+        myPig.registerQuery("store g1 into 'output3';");
+
+        List<ExecJob> jobs = myPig.executeBatch();
+        assertEquals(3, jobs.size());
 
-            for (ExecJob job : jobs) {
-                assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
-            }
-            
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        for (ExecJob job : jobs) {
+            assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
+        }
     }
     
     @Test
-    public void testMultiQueryWithCoGroup_2() {
+    public void testMultiQueryWithCoGroup_2() throws Exception {
 
         System.out.println("===== multi-query with CoGroup (2) =====");
+        
+        myPig.setBatchOn();
 
-        try {
-            myPig.setBatchOn();
-
-            myPig.registerQuery("a = load 'passwd' " +
-                                "using PigStorage(':') as (uname, passwd, uid, gid);");
-            myPig.registerQuery("store a into '/tmp/output1' using BinStorage();");
-            myPig.registerQuery("b = load '/tmp/output1' using BinStorage() as (uname, passwd, uid, gid);"); 
-            myPig.registerQuery("c = load 'passwd2' " +
-                                "using PigStorage(':') as (uname, passwd, uid, gid);");
-            myPig.registerQuery("d = cogroup b by (uname, uid) inner, c by (uname, uid) inner;");
-            myPig.registerQuery("e = foreach d generate flatten(b), flatten(c);");
-            myPig.registerQuery("store e into '/tmp/output2';");
-
-            List<ExecJob> jobs = myPig.executeBatch();
-            assertTrue(jobs.size() == 2);
+        myPig.registerQuery("a = load 'passwd' " +
+                            "using PigStorage(':') as (uname, passwd, uid, gid);");
+        myPig.registerQuery("store a into 'output1' using BinStorage();");
+        myPig.registerQuery("b = load 'output1' using BinStorage() as (uname, passwd, uid, gid);"); 
+        myPig.registerQuery("c = load 'passwd2' " +
+                            "using PigStorage(':') as (uname, passwd, uid, gid);");
+        myPig.registerQuery("d = cogroup b by (uname, uid) inner, c by (uname, uid) inner;");
+        myPig.registerQuery("e = foreach d generate flatten(b), flatten(c);");
+        myPig.registerQuery("store e into 'output2';");
 
-            for (ExecJob job : jobs) {
-                assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
-            }
+        List<ExecJob> jobs = myPig.executeBatch();
+        assertTrue(jobs.size() == 2);
 
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        for (ExecJob job : jobs) {
+            assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
+        }
     }
  
     @Test
-    public void testMultiQueryWithFJ_2() {
+    public void testMultiQueryWithFJ_2() throws Exception {
 
         System.out.println("===== multi-query with FJ (2) =====");
 
-        try {
-            myPig.setBatchOn();
-
-            myPig.registerQuery("a = load 'passwd' " +
-                                "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
-            myPig.registerQuery("b = load 'passwd' " +
-                                "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
-            myPig.registerQuery("c = filter a by uid > 5;");
-            myPig.registerQuery("store c into '/tmp/output1';");
-            myPig.registerQuery("d = filter b by gid > 10;");
-            myPig.registerQuery("store d into '/tmp/output2';");
-            myPig.registerQuery("e = join c by gid, d by gid using \'repl\';");
-            myPig.registerQuery("store e into '/tmp/output3';");
+        myPig.setBatchOn();
 
-            List<ExecJob> jobs = myPig.executeBatch();
-            assertEquals(3, jobs.size());
+        myPig.registerQuery("a = load 'passwd' " +
+                            "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
+        myPig.registerQuery("b = load 'passwd' " +
+                            "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
+        myPig.registerQuery("c = filter a by uid > 5;");
+        myPig.registerQuery("store c into 'output1';");
+        myPig.registerQuery("d = filter b by gid > 10;");
+        myPig.registerQuery("store d into 'output2';");
+        myPig.registerQuery("e = join c by gid, d by gid using \'repl\';");
+        myPig.registerQuery("store e into 'output3';");
 
-            for (ExecJob job : jobs) {
-                assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
-            }
+        List<ExecJob> jobs = myPig.executeBatch();
+        assertEquals(3, jobs.size());
 
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        for (ExecJob job : jobs) {
+            assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
+        }
     } 
  
     @Test
-    public void testMultiQueryWithIntermediateStores_2() {
+    public void testMultiQueryWithIntermediateStores_2() throws Exception {
 
         System.out.println("===== multi-query with intermediate stores (2) =====");
 
-        try {
-            
-            myPig.setBatchOn();
-            
-            myPig.registerQuery("a = load 'passwd' " +
-                                "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
-            myPig.registerQuery("store a into '/tmp/output1';");
-            myPig.registerQuery("b = load '/tmp/output1' using PigStorage(':'); ");
-            myPig.registerQuery("store b into '/tmp/output2';");
-
-            List<ExecJob> jobs = myPig.executeBatch();
-            assertTrue(jobs.size() == 2);
+        myPig.setBatchOn();
+        
+        myPig.registerQuery("a = load 'passwd' " +
+                            "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
+        myPig.registerQuery("store a into 'output1';");
+        myPig.registerQuery("b = load 'output1' using PigStorage(':'); ");
+        myPig.registerQuery("store b into 'output2';");
 
-            for (ExecJob job : jobs) {
-                assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
-            }
+        List<ExecJob> jobs = myPig.executeBatch();
+        assertTrue(jobs.size() == 2);
 
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        for (ExecJob job : jobs) {
+            assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
+        }
     }         
 
     @Test
@@ -513,7 +438,7 @@ public class TestMultiQueryBasic {
         // clean up any existing dirs/files
         String[] toClean = {"tmwsimam-input.txt", "foo1", "foo2", "foo3", "foo4" };
         for (int j = 0; j < toClean.length; j++) {
-            Util.deleteFile(cluster, toClean[j]);    
+            Util.deleteFile(new PigContext(ExecType.LOCAL, new Properties()), toClean[j]);    
         }
         
         // the data below is tab delimited
@@ -522,7 +447,7 @@ public class TestMultiQueryBasic {
         "2	a	b	e	f	i	j	m	n",
         "3	c	d	g	h	k	l	o	p",
         "4	c	d	g	h	k	l	o	p" };
-        Util.createInputFile(cluster, "tmwsimam-input.txt", inputData);
+        Util.createLocalInputFile("tmwsimam-input.txt", inputData);
         String query = 
         "A = LOAD 'tmwsimam-input.txt' " +
         "as (f0:chararray, f1:chararray, f2:chararray, f3:chararray, " +
@@ -579,61 +504,49 @@ public class TestMultiQueryBasic {
         }
         // cleanup
         for (int j = 0; j < toClean.length; j++) {
-            Util.deleteFile(cluster, toClean[j]);    
+            Util.deleteFile(new PigContext(ExecType.LOCAL, new Properties()), toClean[j]);    
         }
         
     }
        
     @Test
-    public void testMultiQueryWithTwoStores2Execs() {
+    public void testMultiQueryWithTwoStores2Execs() throws Exception {
 
         System.out.println("===== multi-query with 2 stores execs =====");
 
-        try {
-            myPig.setBatchOn();
-
-            myPig.registerQuery("a = load 'passwd' " +
-                                "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int,gid:int);");
-            myPig.registerQuery("b = filter a by uid > 5;");
-            myPig.executeBatch();
-            myPig.registerQuery("store b into '/tmp/output1';");
-            myPig.executeBatch();
-            myPig.registerQuery("c = group b by gid;");
-            myPig.registerQuery("store c into '/tmp/output2';");
+        myPig.setBatchOn();
 
-            myPig.executeBatch();
-            myPig.discardBatch();
+        myPig.registerQuery("a = load 'passwd' " +
+                            "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int,gid:int);");
+        myPig.registerQuery("b = filter a by uid > 5;");
+        myPig.executeBatch();
+        myPig.registerQuery("store b into 'output1';");
+        myPig.executeBatch();
+        myPig.registerQuery("c = group b by gid;");
+        myPig.registerQuery("store c into 'output2';");
 
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        myPig.executeBatch();
+        myPig.discardBatch();
     }
 
     @Test
-    public void testMultiQueryWithThreeStores2() {
+    public void testMultiQueryWithThreeStores2() throws Exception {
 
         System.out.println("===== multi-query with 3 stores (2) =====");
 
-        try {
-            myPig.setBatchOn();
+        myPig.setBatchOn();
 
-            myPig.registerQuery("a = load 'passwd' " +
-                                "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int,gid:int);");
-            myPig.registerQuery("b = filter a by uid > 5;");
-            myPig.registerQuery("store b into '/tmp/output1';");
-            myPig.registerQuery("c = filter b by uid > 10;");
-            myPig.registerQuery("store c into '/tmp/output2';");
-            myPig.registerQuery("d = filter c by uid > 15;");
-            myPig.registerQuery("store d into '/tmp/output3';");
-
-            myPig.executeBatch();
-            myPig.discardBatch();
-
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        } 
+        myPig.registerQuery("a = load 'passwd' " +
+                            "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int,gid:int);");
+        myPig.registerQuery("b = filter a by uid > 5;");
+        myPig.registerQuery("store b into 'output1';");
+        myPig.registerQuery("c = filter b by uid > 10;");
+        myPig.registerQuery("store c into 'output2';");
+        myPig.registerQuery("d = filter c by uid > 15;");
+        myPig.registerQuery("store d into 'output3';");
+
+        myPig.executeBatch();
+        myPig.discardBatch();
     }
  
     /**
@@ -644,7 +557,7 @@ public class TestMultiQueryBasic {
      */
     @Test
     public void testMultiStoreWithOutputFormat() throws IOException {
-        Util.createInputFile(cluster, "input.txt", new String[] {"hello", "bye"});
+        Util.createLocalInputFile("input.txt", new String[] {"hello", "bye"});
         String query = "a = load 'input.txt';" +
         		"b = filter a by $0 < 10;" +
         		"store b into 'output1' using "+DUMMY_STORE_WITH_OUTPUTFORMAT_CLASS+"();" +
@@ -657,12 +570,12 @@ public class TestMultiQueryBasic {
         
         // check that files were created as a result of the
         // checkOutputSpecs() method of the OutputFormat being called
-        FileSystem fs = cluster.getFileSystem();
+        FileSystem fs = FileSystem.getLocal(new Configuration());
         assertEquals(true, fs.exists(new Path("output1_checkOutputSpec_test")));
         assertEquals(true, fs.exists(new Path("output2_checkOutputSpec_test")));
-        Util.deleteFile(cluster, "input.txt");
-        Util.deleteFile(cluster, "output1_checkOutputSpec_test");
-        Util.deleteFile(cluster, "output2_checkOutputSpec_test");
+ 
+        Util.deleteFile(new PigContext(ExecType.LOCAL, new Properties()), "output1_checkOutputSpec_test");
+        Util.deleteFile(new PigContext(ExecType.LOCAL, new Properties()), "output2_checkOutputSpec_test");
     }
         
     private static final String DUMMY_STORE_WITH_OUTPUTFORMAT_CLASS
@@ -749,16 +662,11 @@ public class TestMultiQueryBasic {
     // --------------------------------------------------------------------------
     // Helper methods
 
-    private void deleteOutputFiles() {
-        try {
-            FileLocalizer.delete("/tmp/output1", myPig.getPigContext());
-            FileLocalizer.delete("/tmp/output2", myPig.getPigContext());
-            FileLocalizer.delete("/tmp/output3", myPig.getPigContext());
-            FileLocalizer.delete("/tmp/output4", myPig.getPigContext());
-            FileLocalizer.delete("/tmp/output5", myPig.getPigContext());
-        } catch (IOException e) {
-            e.printStackTrace();
-            Assert.fail();
-        }
+    private static void deleteOutputFiles() {
+        Util.deleteDirectory(new File("output1"));
+        Util.deleteDirectory(new File("output2"));
+        Util.deleteDirectory(new File("output3"));
+        Util.deleteDirectory(new File("output4"));
+        Util.deleteDirectory(new File("output5"));
     }
 }

Modified: pig/trunk/test/org/apache/pig/test/Util.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/Util.java?rev=1099299&r1=1099298&r2=1099299&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/Util.java (original)
+++ pig/trunk/test/org/apache/pig/test/Util.java Wed May  4 00:11:51 2011
@@ -43,6 +43,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 
 import junit.framework.Assert;
 
@@ -533,6 +534,24 @@ public class Util {
         }
 	}
 	
+    static public void copyFromLocalToLocal(String fromLocalFileName,
+            String toLocalFileName) throws IOException {
+        PigServer ps = new PigServer(ExecType.LOCAL, new Properties());
+        String script = "fs -cp " + fromLocalFileName + " " + toLocalFileName;
+
+        new File(toLocalFileName).deleteOnExit();
+        
+        GruntParser parser = new GruntParser(new StringReader(script));
+        parser.setInteractive(false);
+        parser.setParams(ps);
+        try {
+            parser.parseStopOnError();
+        } catch (org.apache.pig.tools.pigscript.parser.ParseException e) {
+            throw new IOException(e);
+        }
+        
+    }
+	
 	static public void copyFromClusterToLocal(MiniCluster cluster, String fileNameOnCluster, String localFileName) throws IOException {
 	    PrintWriter writer = new PrintWriter(new FileWriter(localFileName));