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/02/27 01:52:08 UTC

svn commit: r631443 [10/10] - in /incubator/pig/branches/types: ./ lib-src/bzip2/org/apache/tools/bzip2r/ lib-src/shock/org/apache/pig/shock/ lib/ scripts/ src/org/apache/pig/ src/org/apache/pig/backend/ src/org/apache/pig/backend/datastorage/ src/org/...

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java?rev=631443&r1=631442&r2=631443&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestMapReduce.java Tue Feb 26 16:51:49 2008
@@ -29,7 +29,6 @@
 
 import junit.framework.TestCase;
 
-import org.apache.hadoop.fs.Path;
 import org.junit.Test;
 
 import org.apache.pig.EvalFunc;
@@ -46,13 +45,15 @@
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.io.BufferedPositionedInputStream;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.backend.datastorage.ElementDescriptor;
 
 public class TestMapReduce extends TestCase {
 
-	private String initString = "mapreduce";
-	
-	@Test
-    public void testBigGroupAll() throws Exception {
+    private String initString = "mapreduce";
+    MiniCluster cluster = MiniCluster.buildCluster();
+
+    @Test
+    public void testBigGroupAll() throws Throwable {
         int LOOP_COUNT = 4*1024;
         PigServer pig = new PigServer(initString);
         File tmpFile = File.createTempFile("test", "txt");
@@ -72,13 +73,13 @@
     }
     
     static public class MyApply extends EvalFunc<DataBag> {
-    	String field0 = "Got";
-    	public MyApply() {}
-    	public MyApply(String field0) {
-    		this.field0 = field0;
-    	}
+        String field0 = "Got";
+        public MyApply() {}
+        public MyApply(String field0) {
+            this.field0 = field0;
+        }
         @Override
-		public DataBag exec(Tuple input) throws IOException {
+        public DataBag exec(Tuple input) throws IOException {
             DataBag output = BagFactory.getInstance().newDefaultBag();
             Iterator<Tuple> it = (DataType.toBag(input.get(0))).iterator();
             while(it.hasNext()) {
@@ -103,9 +104,9 @@
     static public class MyStorage implements LoadFunc, StoreFunc {
         final static int COUNT = 10;
         int count = 0;
-		public void bindTo(String fileName, BufferedPositionedInputStream is, long offset, long end) throws IOException {
+        public void bindTo(String fileName, BufferedPositionedInputStream is, long offset, long end) throws IOException {
         }
-		public Tuple getNext() throws IOException {
+        public Tuple getNext() throws IOException {
             if (count < COUNT) {
                 Tuple t = TupleFactory.getInstance().newTuple(Integer.toString(count++));
                 return t;
@@ -113,19 +114,19 @@
             return null;
         }
         OutputStream os;
-		public void bindTo(OutputStream os) throws IOException {
-			this.os = os;
-		}
-		public void finish() throws IOException {
-			
-		}
-		public void putNext(Tuple f) throws IOException {
-			os.write((f.toDelimitedString("-")+"\n").getBytes());			
-		}
+        public void bindTo(OutputStream os) throws IOException {
+            this.os = os;
+        }
+        public void finish() throws IOException {
+            
+        }
+        public void putNext(Tuple f) throws IOException {
+            os.write((f.toDelimitedString("-")+"\n").getBytes());            
+        }
     }
     @Test
-    public void testStoreFunction() throws IOException {
-    	PigServer pig = new PigServer(initString);
+    public void testStoreFunction() throws Throwable {
+        PigServer pig = new PigServer(initString);
         File tmpFile = File.createTempFile("test", ".txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         for(int i = 0; i < 10; i++) {
@@ -136,7 +137,7 @@
         System.out.println(query);
         pig.registerQuery("asdf_id = " + query);
         try {
-        	pig.deleteFile("frog");
+            pig.deleteFile("frog");
         } catch(Exception e) {}
         pig.store("asdf_id", "frog", MyStorage.class.getName()+"()");
         InputStream is = FileLocalizer.open("frog", pig.getPigContext());
@@ -144,14 +145,14 @@
         String line;
         int i = 0;
         while((line = br.readLine()) != null) {
-        	assertEquals(line, Integer.toString(i) + '-' + Integer.toString(i));
-        	i++;
+            assertEquals(line, Integer.toString(i) + '-' + Integer.toString(i));
+            i++;
         }
         br.close();
         pig.deleteFile("frog");
     }
     @Test
-    public void testQualifiedFuncions() throws IOException {
+    public void testQualifiedFuncions() throws Throwable {
         PigServer pig = new PigServer(initString);
         File tmpFile = File.createTempFile("test", ".txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
@@ -176,7 +177,7 @@
     }
     
     @Test
-    public void testDefinedFunctions() throws IOException {
+    public void testDefinedFunctions() throws Throwable {
         PigServer pig = new PigServer(initString);
         File tmpFile = File.createTempFile("test", ".txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
@@ -200,9 +201,9 @@
         }
         assertEquals(count, MyStorage.COUNT);
     }
-    
+
     @Test
-    public void testPigServer() throws IOException {
+    public void testPigServer() throws Throwable {
         System.out.println("creating pig server");
         PigContext pigContext = new PigContext(ExecType.MAPREDUCE);
         PigServer pig = new PigServer(pigContext);
@@ -211,24 +212,12 @@
         assertTrue(capacity > 0);
         String sampleFileName = "/tmp/fileTest";
         if (!pig.existsFile(sampleFileName)) {
-            OutputStream os = pigContext.getDfs().create(new Path(sampleFileName));
+            ElementDescriptor path = pigContext.getDfs().asElement(sampleFileName);
+            OutputStream os = path.create();
             os.write("Ben was here!".getBytes());
             os.close();
         }
         long length = pig.fileSize(sampleFileName);
         assertTrue(length > 0);
     }
-    
-    @Test
-    public void testCreateNewRelation() throws IOException {
-        System.out.println("creating pig server");
-        PigServer pig = new PigServer(initString);
-		pig.deleteFile("/tmp/test_createNewRelation");
-        System.out.println("testing create new relation");
-        pig.newRelation("new_rel");
-        pig.insertTuple("new_rel", TupleFactory.getInstance().newTuple("hello"));
-        pig.store("new_rel", "/tmp/test_createNewRelation");
-        assertTrue(pig.existsFile("/tmp/test_createNewRelation"));
-    }
-    
 }

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestOrderBy.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestOrderBy.java?rev=631443&r1=631442&r2=631443&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestOrderBy.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestOrderBy.java Tue Feb 26 16:51:49 2008
@@ -35,11 +35,12 @@
     private String initString = "mapreduce";
     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 Exception {
+    public TestOrderBy() throws Throwable {
         DecimalFormat myFormatter = new DecimalFormat("0000000");
         for (int i = 0; i < DATALEN; i++) {
             DATA[0][i] = myFormatter.format(i);
@@ -68,7 +69,6 @@
         for(int i = 0; i < DATALEN; i++) {
             Tuple t = (Tuple)it.next();
             int value = DataType.toInteger(t.get(1));
-//            System.out.println("" + i + "," + DATA[0][i] + "," + DATA[1][i] + "," + value);
             assertEquals(Integer.parseInt(DATA[col][i]), value);
         }
         assertFalse(it.hasNext());

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestPi.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPi.java?rev=631443&r1=631442&r2=631443&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestPi.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestPi.java Tue Feb 26 16:51:49 2008
@@ -25,43 +25,51 @@
 
 import org.apache.pig.impl.io.FileLocalizer;
 
+import org.apache.pig.backend.executionengine.ExecException;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import junit.framework.TestCase;
 
 import java.io.File;
+import java.io.IOException;
 import java.io.FileOutputStream;
 import java.util.Iterator;
 import java.util.Random;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 /*
  * Testcase aimed at testing pig with large file sizes and filter and group functions
 */
 public class TestPi extends TestCase {
-	
-	File datFile;
-	private long defaultBlockSize = (new Configuration()).getLong("dfs.block.size", 0);
-	
-	private long total = ((defaultBlockSize >> 20) / 10) << 20;
-	private int inCircle = 0;
-	private String initString = "mapreduce";
-
-	private long totalLength = 0, totalLengthTest = 0;
-
-	
-	
-	PigServer pig;
-	String fileName, tmpFile1;
-	
-	@Override
-	@Before
+    
+    private final Log log = LogFactory.getLog(getClass());
+
+    File datFile;
+    private long defaultBlockSize = (new Configuration()).getLong("dfs.block.size", 0);
+    
+    private long total = ((defaultBlockSize >> 20) / 10) << 20;
+    private int inCircle = 0;
+    private String initString = "mapreduce";
+    MiniCluster cluster = MiniCluster.buildCluster();
+
+    private long totalLength = 0, totalLengthTest = 0;
+
+    
+    
+    PigServer pig;
+    String fileName, tmpFile1;
+    
+    @Override
+    @Before
     protected void setUp() throws Exception{
 
-        System.out.println("Generating test data...");
-        System.out.println("Default block size = " + defaultBlockSize);
-        System.out.println("Total no. of iterations to run for test data = " + total);
+        log.info("Generating test data...");
+        log.info("Default block size = " + defaultBlockSize);
+        log.info("Total no. of iterations to run for test data = " + total);
         datFile = File.createTempFile("PiTest", ".dat");
         
         FileOutputStream dat = new FileOutputStream(datFile);
@@ -77,7 +85,7 @@
             double y1 = y.doubleValue() - 0.5;
             double sq_dist = (x1*x1) + (y1*y1); 
             if(sq_dist <= 0.25) {
-            	inCircle ++;
+                inCircle ++;
 
             }
 
@@ -91,63 +99,71 @@
         
         dat.close();
         
-        pig = new PigServer(initString);
-		fileName = "'" + FileLocalizer.hadoopify(datFile.toString(), pig.getPigContext()) + "'";
-		tmpFile1 = "'" + FileLocalizer.getTemporaryPath(null, pig.getPigContext()).toString() + "'";
+        try {
+            pig = new PigServer(initString);
+        }
+        catch (ExecException e) {
+            IOException ioe = new IOException("Failed to create Pig Server");
+            ioe.initCause(e);
+            throw ioe;
+        }
+        
+        fileName = "'" + FileLocalizer.hadoopify(datFile.toString(), pig.getPigContext()) + "'";
+        tmpFile1 = "'" + FileLocalizer.getTemporaryPath(null, pig.getPigContext()).toString() + "'";
 
         datFile.delete();
     }
-	
-	@Override
-	@After
-	protected void tearDown() throws Exception {
-		
-	}
-	
-	@Test
-	public void testPi () throws Exception {
-		
-		pig.registerQuery("A = load " + fileName + " using PigStorage(':');");
-		pig.registerQuery("B = foreach A generate $0 - '0.5' as d1, $1 - '0.5' as d2;");
-		pig.registerQuery("C = foreach B generate $0 * $0 as m1, $1 * $1 as m2;");
-		pig.registerQuery("D = foreach C generate $0 + $1 as s1;");
-		pig.registerQuery("D = foreach D generate $0, ARITY($0);");
-		pig.store("D", tmpFile1);
-
-		pig.registerQuery("E = filter D by $0 <= '0.25';");
-
-		pig.registerQuery("F = group D by $1;");
-		pig.registerQuery("G = group E by $1;");
-
-		pig.registerQuery("J = foreach F generate COUNT($1);");
-		pig.registerQuery("K = foreach G generate COUNT($1);");
-		
-
-		Iterator <Tuple> Total = pig.openIterator("J");
-		Iterator <Tuple> InCircle = pig.openIterator("K");
-
-		
-		int totalPoints = DataType.toInteger(Total.next().get(0));
-		int inCirclePoints = DataType.toInteger(InCircle.next().get(0));
-
-		System.out.println("Value of PI = " + 4 * (double)inCircle / (double)total);
-		System.out.println("Value of PI (From Test data) = " + 4 * (double)inCirclePoints / (double)totalPoints);
-		
-		
-		Iterator <Tuple> lengthTest = pig.openIterator("D");
-		
-		while(lengthTest.hasNext()) {
-			Tuple temp = lengthTest.next();
-			totalLengthTest += temp.get(0).toString().length();
-		}
-		
-		assertEquals(totalPoints, total);
-		assertEquals(inCirclePoints, inCircle);
-		assertEquals(totalLengthTest, totalLength);
+    
+    @Override
+    @After
+    protected void tearDown() throws Exception {
+        
+    }
+    
+    @Test
+    public void testPi () throws Exception {
+        
+        pig.registerQuery("A = load " + fileName + " using PigStorage(':');");
+        pig.registerQuery("B = foreach A generate $0 - '0.5' as d1, $1 - '0.5' as d2;");
+        pig.registerQuery("C = foreach B generate $0 * $0 as m1, $1 * $1 as m2;");
+        pig.registerQuery("D = foreach C generate $0 + $1 as s1;");
+        pig.registerQuery("D = foreach D generate $0, ARITY($0);");
+        pig.store("D", tmpFile1);
+
+        pig.registerQuery("E = filter D by $0 <= '0.25';");
+
+        pig.registerQuery("F = group D by $1;");
+        pig.registerQuery("G = group E by $1;");
+
+        pig.registerQuery("J = foreach F generate COUNT($1);");
+        pig.registerQuery("K = foreach G generate COUNT($1);");
+        
 
-		
+        Iterator <Tuple> Total = pig.openIterator("J");
+        Iterator <Tuple> InCircle = pig.openIterator("K");
 
-			
-	}
+        
+        int totalPoints = DataType.toInteger(Total.next().get(0));
+        int inCirclePoints = DataType.toInteger(InCircle.next().get(0));
+
+        log.info("Value of PI = " + 4 * (double)inCircle / (double)total);
+        log.info("Value of PI (From Test data) = " + 4 * (double)inCirclePoints / (double)totalPoints);
+        
+        
+        Iterator <Tuple> lengthTest = pig.openIterator("D");
+        
+        while(lengthTest.hasNext()) {
+            Tuple temp = lengthTest.next();
+            totalLengthTest += temp.get(0).toString().length();
+        }
+        
+        assertEquals(totalPoints, total);
+        assertEquals(inCirclePoints, inCircle);
+        assertEquals(totalLengthTest, totalLength);
+
+        
+
+            
+    }
 
 }

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestPigFile.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPigFile.java?rev=631443&r1=631442&r2=631443&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestPigFile.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestPigFile.java Tue Feb 26 16:51:49 2008
@@ -32,6 +32,8 @@
 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.BinStorage;
 import org.apache.pig.builtin.PigStorage;
@@ -42,14 +44,16 @@
 
 public class TestPigFile extends TestCase {
 
+    private final Log log = LogFactory.getLog(getClass());
+
     DataBag bag          = BagFactory.getInstance().newDefaultBag();
     Random rand = new Random();
     
     @Override
-	@Before
+    @Before
     protected void setUp() throws Exception {
 
-        System.out.println("Generating PigFile test data...");
+        log.info("Generating PigFile test data...");
 
         Random rand = new Random();
         byte[] r = new byte[10];
@@ -65,11 +69,11 @@
             t.set(j, new DataByteArray(r));
 
         }
-        System.out.println("Done.");
+        log.info("Done.");
     }
 
     @Override
-	@After
+    @After
     protected void tearDown() throws Exception {
     }
 
@@ -77,16 +81,16 @@
     public void testStoreAndLoadText() throws IOException {
         PigContext pigContext = new PigContext(ExecType.LOCAL);
         
-        System.out.println("Running Store...");
+        log.info("Running Store...");
         String initialdata = File.createTempFile("pig-tmp", "").getAbsolutePath();
         PigFile store = new PigFile(initialdata);
         store.store(bag, new PigStorage(), pigContext);
-        System.out.println("Done.");
+        log.info("Done.");
 
-        System.out.println("Running Load...");
+        log.info("Running Load...");
         PigFile load = new PigFile(initialdata);
         DataBag loaded = load.load(new PigStorage(), pigContext);
-        System.out.println("Done.");
+        log.info("Done.");
 
         assertTrue(bag.size() == loaded.size());
 
@@ -102,17 +106,17 @@
     }
 
     private Object getRandomDatum(int nestingLevel) throws IOException{
-    	if (nestingLevel>3)
-    		return getRandomDataAtom();
-    	int i = rand.nextInt(4);
-    	switch(i){
-    	case 0: return getRandomDataAtom();
-    	case 1: return getRandomTuple(nestingLevel);
-    	case 2: return getRandomBag(20,nestingLevel);
-    	case 3: return getRandomMap(nestingLevel);
-    	}
-    	
-    	throw new RuntimeException("Shouldn't reach here.");
+        if (nestingLevel>3)
+            return getRandomDataAtom();
+        int i = rand.nextInt(4);
+        switch(i){
+        case 0: return getRandomDataAtom();
+        case 1: return getRandomTuple(nestingLevel);
+        case 2: return getRandomBag(20,nestingLevel);
+        case 3: return getRandomMap(nestingLevel);
+        }
+        
+        throw new RuntimeException("Shouldn't reach here.");
     }
 
     private char[] letters = {'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j',
@@ -122,9 +126,9 @@
     private DataByteArray getRandomDataAtom(){
         /*
         byte[] bytes = new byte[10];
-    	rand.nextBytes(bytes);
-    	//return new DataByteArray(bytes);
-    	return new DataByteArray("Abc");
+        rand.nextBytes(bytes);
+        //return new DataByteArray(bytes);
+        return new DataByteArray("Abc");
         */
         String s = new String();
         for (int i = 0; i < 10; i++) {
@@ -135,52 +139,52 @@
     }
     
     private Tuple getRandomTuple(int nestingLevel) throws IOException{
-    	
-    	int cardinality = rand.nextInt(2)+1;
-    	Tuple t = TupleFactory.getInstance().newTuple(cardinality);
-    	for (int i=0; i<cardinality; i++)
-    		t.set(i, getRandomDatum(nestingLevel+1));
-    	return t;
+        
+        int cardinality = rand.nextInt(2)+1;
+        Tuple t = TupleFactory.getInstance().newTuple(cardinality);
+        for (int i=0; i<cardinality; i++)
+            t.set(i, getRandomDatum(nestingLevel+1));
+        return t;
     }
     
     private DataBag getRandomBag(int maxCardinality, int nestingLevel) throws IOException{
-    	int cardinality = rand.nextInt(maxCardinality)+1;
-    	DataBag b = BagFactory.getInstance().newDefaultBag();
-    	for (int i=0; i<cardinality; i++){
-    		Tuple t = getRandomTuple(nestingLevel+1); 
-    		b.add(t);
-    	}
-    	return b;
-    	
+        int cardinality = rand.nextInt(maxCardinality)+1;
+        DataBag b = BagFactory.getInstance().newDefaultBag();
+        for (int i=0; i<cardinality; i++){
+            Tuple t = getRandomTuple(nestingLevel+1); 
+            b.add(t);
+        }
+        return b;
+        
     }
     
     private Map<Object, Object> getRandomMap(int nestingLevel) throws IOException{
-    	int cardinality = rand.nextInt(2)+1;
-    	Map<Object, Object> m = new HashMap<Object, Object>();
-    	for (int i=0; i<cardinality; i++){
-    		m.put(getRandomDataAtom().toString(),getRandomDatum(nestingLevel+1));
-    	}
-    	return m;
+        int cardinality = rand.nextInt(2)+1;
+        Map<Object, Object> m = new HashMap<Object, Object>();
+        for (int i=0; i<cardinality; i++){
+            m.put(getRandomDataAtom().toString(),getRandomDatum(nestingLevel+1));
+        }
+        return m;
     }
 
     @Test
     public void testStoreAndLoadBin() throws IOException {
-        System.out.println("Generating Data ...");
+        log.info("Generating Data ...");
         bag = getRandomBag(5000,0);
-        System.out.println("Done.");
+        log.info("Done.");
         
         PigContext pigContext = new PigContext(ExecType.LOCAL);
         
-        System.out.println("Running Store...");
+        log.info("Running Store...");
         String storeFile = File.createTempFile("pig-tmp", "").getAbsolutePath();
         PigFile store = new PigFile(storeFile);
         store.store(bag, new BinStorage(), pigContext);
-        System.out.println("Done.");
+        log.info("Done.");
 
-        System.out.println("Running Load...");
+        log.info("Running Load...");
         PigFile load = new PigFile(storeFile);
         DataBag loaded = load.load(new BinStorage(), pigContext);
-        System.out.println("Done.");
+        log.info("Done.");
 
         assertTrue(bag.size() == loaded.size());
 
@@ -196,8 +200,8 @@
     }
 
 
-    public void testLocalStore() throws Exception{
-    	PigServer pig = new PigServer("local");
+    public void testLocalStore() throws Throwable {
+        PigServer pig = new PigServer("local");
         File tmpFile = File.createTempFile("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
         for(int i = 0; i < 10; i++) {
@@ -212,7 +216,7 @@
         tmpFile.delete();
         tmpFile = new File("/tmp/abc");
         tmpFile.delete();
-    	
+        
     }
     
 

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestPigServer.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPigServer.java?rev=631443&r1=631442&r2=631443&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestPigServer.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestPigServer.java Tue Feb 26 16:51:49 2008
@@ -18,8 +18,16 @@
 
 
 public class TestPigServer extends TestCase {
-    private PigServer pig = new PigServer();
-
+    private PigServer pig = null;
+    MiniCluster cluster = MiniCluster.buildCluster();
+    
+    private void initPigServer() throws Throwable {
+        if (pig == null) {
+            pig = new PigServer();
+        }
+    }
+    
+    
     private final static String FILE_SEPARATOR = System.getProperty("file.separator");
     
     // make sure that name is included or not (depending on flag "included") 
@@ -79,12 +87,13 @@
      * The jar file to register is not present
      */
     @Test
-    public void testRegisterJarFileNotPresent() throws Exception {
+    public void testRegisterJarFileNotPresent() throws Throwable {
         // resister a jar file that does not exist
         
         String jarName = "BadFileNameTestJarNotPresent.jar";
         
         // jar name is not present to start with
+        initPigServer();
         verifyStringContained(pig.getPigContext().extraJars, jarName, false);
 
         boolean exceptionRaised = false;
@@ -103,13 +112,15 @@
      * in this case name of jar file is relative to current working dir
      */
     @Test
-    public void testRegisterJarLocalDir() throws Exception {
+    public void testRegisterJarLocalDir() throws Throwable {
         String dir1 = "test1_register_jar_local";
         String dir2 = "test2_register_jar_local";
         String jarLocation = dir1 + FILE_SEPARATOR +
                               dir2 + FILE_SEPARATOR;
         String jarName = "TestRegisterJarLocal.jar";
         
+        initPigServer();
+        
         createFakeJarFile(jarLocation, jarName);
         
         verifyStringContained(pig.getPigContext().extraJars, jarName, false);
@@ -136,7 +147,7 @@
      * only one of them is registered.
      */
     @Test
-    public void testRegisterJarFromResources () throws Exception {
+    public void testRegisterJarFromResources () throws Throwable {
         String dir = "test_register_jar_res_dir";
         String subDir1 = "test_register_jar_res_sub_dir1";
         String subDir2 = "test_register_jar_res_sub_dir2";
@@ -144,6 +155,8 @@
         String jarLocation1 = dir + FILE_SEPARATOR + subDir1 + FILE_SEPARATOR;
         String jarLocation2 = dir + FILE_SEPARATOR + subDir2 + FILE_SEPARATOR;
         
+        initPigServer();
+        
         createFakeJarFile(jarLocation1, jarName);
         createFakeJarFile(jarLocation2, jarName);
         
@@ -176,13 +189,15 @@
      * @throws Exception
      */
     @Test
-    public void testRegisterJarResourceInJar() throws Exception {
+    public void testRegisterJarResourceInJar() throws Throwable {
         String dir = "test_register_jar_res_in_jar";
         String subDir = "sub_dir";
         String jarName = "TestRegisterJarNonEmpty.jar";
         String className = "TestRegisterJar";
         String javaSrc = "package " + subDir + "; class " + className + " { }";
 
+        initPigServer();
+        
         // create dirs
         (new File(dir + FILE_SEPARATOR + subDir)).mkdirs();
 

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestPigSplit.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestPigSplit.java?rev=631443&r1=631442&r2=631443&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestPigSplit.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestPigSplit.java Tue Feb 26 16:51:49 2008
@@ -20,6 +20,7 @@
 
 import java.io.BufferedWriter;
 import java.io.File;
+import java.io.IOException;
 import java.io.FileWriter;
 import java.io.PrintWriter;
 import java.util.Iterator;
@@ -28,37 +29,44 @@
 
 import org.apache.pig.PigServer;
 import org.apache.pig.data.Tuple;
-import org.apache.pig.impl.util.PigLogger;
+import org.apache.pig.backend.executionengine.ExecException;
 
 import junit.framework.TestCase;
 
 public class TestPigSplit extends TestCase {
-	PigServer pig;
-	
-	@Override
-	protected void setUp() throws Exception {
-		super.setUp();
-		PigLogger.setAppenderForJunit();
-		pig = new PigServer();
-	}
-	
-	@Test
-	public void testLongEvalSpec() throws Exception{
-		File f = File.createTempFile("tmp", "");
-		
-		PrintWriter pw = new PrintWriter(f);
-		pw.println("0\ta");
-		pw.close();
-		
-		pig.registerQuery("a = load 'file:" + f + "';");
-		for (int i=0; i< 500; i++){
-			pig.registerQuery("a = filter a by $0 == '1';");
-		}
-		Iterator<Tuple> iter = pig.openIterator("a");
-		while (iter.hasNext()){
-			throw new Exception();
-		}
-		f.delete();
-	}
-	
+    PigServer pig;
+    MiniCluster cluster = MiniCluster.buildCluster();
+    
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+
+        try {
+            pig = new PigServer();
+        }
+        catch (ExecException e) {
+            IOException ioe = new IOException("Failed to create Pig Server");
+            ioe.initCause(e);
+            throw ioe;
+        }
+    }
+    @Test
+    public void testLongEvalSpec() throws Exception{
+        File f = File.createTempFile("tmp", "");
+        
+        PrintWriter pw = new PrintWriter(f);
+        pw.println("0\ta");
+        pw.close();
+        
+        pig.registerQuery("a = load 'file:" + f + "';");
+        for (int i=0; i< 500; i++){
+            pig.registerQuery("a = filter a by $0 == '1';");
+        }
+        Iterator<Tuple> iter = pig.openIterator("a");
+        while (iter.hasNext()){
+            throw new Exception();
+        }
+        f.delete();
+    }
+    
 }

Modified: incubator/pig/branches/types/test/org/apache/pig/test/TestStore.java
URL: http://svn.apache.org/viewvc/incubator/pig/branches/types/test/org/apache/pig/test/TestStore.java?rev=631443&r1=631442&r2=631443&view=diff
==============================================================================
--- incubator/pig/branches/types/test/org/apache/pig/test/TestStore.java (original)
+++ incubator/pig/branches/types/test/org/apache/pig/test/TestStore.java Tue Feb 26 16:51:49 2008
@@ -31,6 +31,7 @@
 public class TestStore extends TestCase {
 
     private String initString = "mapreduce";
+    MiniCluster cluster = MiniCluster.buildCluster();
     private int LOOP_COUNT = 1024;
     
     String fileName;