You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ch...@apache.org on 2012/11/08 18:06:50 UTC

svn commit: r1407167 [3/10] - in /pig/trunk: ./ test/org/apache/pig/test/ test/org/apache/pig/test/pigunit/

Modified: pig/trunk/test/org/apache/pig/test/TestCombiner.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestCombiner.java?rev=1407167&r1=1407166&r2=1407167&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestCombiner.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestCombiner.java Thu Nov  8 17:06:46 2012
@@ -1,14 +1,12 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
+ * or more contributor license agreements. See the NOTICE file
  * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
+ * 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
- *
+ * 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.
@@ -32,23 +30,17 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 
-import junit.framework.Assert;
-
 import org.apache.pig.EvalFunc;
 import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
-import org.apache.pig.ResourceSchema.ResourceFieldSchema;
 import org.apache.pig.builtin.PigStorage;
 import org.apache.pig.data.DataBag;
-import org.apache.pig.data.DataType;
 import org.apache.pig.data.DefaultDataBag;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileLocalizer;
-import org.apache.pig.impl.util.Utils;
-import org.apache.pig.newplan.logical.relational.LogicalSchema;
-import org.apache.pig.newplan.logical.relational.LogicalSchema.LogicalFieldSchema;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.Test;
 
 public class TestCombiner {
@@ -59,48 +51,51 @@ public class TestCombiner {
     public static void oneTimeTearDown() throws Exception {
         cluster.shutDown();
     }
-    
-    
+
     @Test
-    public void testSuccessiveUserFuncs1() throws Exception{
+    public void testSuccessiveUserFuncs1() throws Exception {
         String query = "a = load 'students.txt' as (c1,c2,c3,c4); " +
-                       "c = group a by c2; " +
-                       "f = foreach c generate COUNT(org.apache.pig.builtin.Distinct($1.$2)); " +
-                       "store f into 'out';";
+                "c = group a by c2; " +
+                "f = foreach c generate COUNT(org.apache.pig.builtin.Distinct($1.$2)); " +
+                "store f into 'out';";
         PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         PigContext pc = pigServer.getPigContext();
-        assertTrue((Util.buildMRPlan(Util.buildPp(pigServer,query),pc).getRoots().get(0).combinePlan.isEmpty()));
+        assertTrue((Util.buildMRPlan(Util.buildPp(pigServer, query), pc).getRoots().get(0).combinePlan
+                .isEmpty()));
     }
 
     @Test
     public void testSuccessiveUserFuncs2() throws Exception {
         String dummyUDF = JiraPig1030.class.getName();
-    	String query = "a = load 'students.txt' as (c1,c2,c3,c4); " +
-                       "c = group a by c2; " +
-                       "f = foreach c generate COUNT(" + dummyUDF + "" +
-               		   "(org.apache.pig.builtin.Distinct($1.$2),"+dummyUDF+"())); " +
-               		   "store f into 'out';";
+        String query = "a = load 'students.txt' as (c1,c2,c3,c4); " +
+                "c = group a by c2; " +
+                "f = foreach c generate COUNT(" + dummyUDF + "" +
+                "(org.apache.pig.builtin.Distinct($1.$2)," + dummyUDF + "())); " +
+                "store f into 'out';";
         PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         PigContext pc = pigServer.getPigContext();
-        assertTrue((Util.buildMRPlan(Util.buildPp(pigServer,query),pc).getRoots().get(0).combinePlan.isEmpty()));
+        assertTrue((Util.buildMRPlan(Util.buildPp(pigServer, query), pc).getRoots().get(0).combinePlan
+                .isEmpty()));
     }
-    
+
     @Test
     public void testOnCluster() throws Exception {
-        // run the test on cluster        
+        // run the test on cluster
         String inputFileName = runTest(new PigServer(
                 ExecType.MAPREDUCE, cluster.getProperties()));
         Util.deleteFile(cluster, inputFileName);
 
     }
-    
-    /* (non-Javadoc)
+
+    /*
+     * (non-Javadoc)
      * @see junit.framework.TestCase#setUp()
      */
-    protected void setUp() throws Exception {
+    @Before
+    public void setUp() throws Exception {
         // cause a re initialization of FileLocalizer's
         // internal state before each test run
-        // A previous test might have been in a different 
+        // A previous test might have been in a different
         // mode than the test which is about to run. To
         // ensure each test runs correctly in it's exectype
         // mode, let's re initialize.
@@ -115,8 +110,7 @@ public class TestCombiner {
         FileLocalizer.deleteTempFiles();
     }
 
-    
-    private String  runTest(PigServer pig) throws IOException {
+    private String runTest(PigServer pig) throws IOException {
         List<String> inputLines = new ArrayList<String>();
         inputLines.add("a,b,1");
         inputLines.add("a,b,1");
@@ -130,7 +124,7 @@ public class TestCombiner {
         assertEquals("(a,b,2)", tuple.toString());
         tuple = resultIterator.next();
         assertEquals("(a,c,1)", tuple.toString());
-        
+
         return inputFileName;
     }
 
@@ -139,7 +133,7 @@ public class TestCombiner {
         File inputFile = File.createTempFile("test", "txt");
         inputFile.deleteOnExit();
         String inputFileName = inputFile.getAbsolutePath();
-        if(pig.getPigContext().getExecType() == ExecType.LOCAL) {
+        if (pig.getPigContext().getExecType() == ExecType.LOCAL) {
             PrintStream ps = new PrintStream(new FileOutputStream(inputFile));
             for (String line : inputLines) {
                 ps.println(line);
@@ -154,7 +148,7 @@ public class TestCombiner {
                 + PigStorage.class.getName() + "(',');");
         return inputFileName;
     }
-    
+
     @Test
     public void testNoCombinerUse() {
         // To simulate this, we will have two input files
@@ -162,15 +156,15 @@ public class TestCombiner {
         // in two map tasks and each would process only one record
         // hence the combiner would not get called.
     }
-    
+
     @Test
     public void testMultiCombinerUse() throws Exception {
         // test the scenario where the combiner is called multiple
         // times - this can happen when the output of the map > io.sort.mb
         // let's set the io.sort.mb to 1MB and > 1 MB map data.
-        String[] input = new String[500*1024];
-        for(int i = 0; i < input.length; i++) {
-            if(i % 2 == 0) {
+        String[] input = new String[500 * 1024];
+        for (int i = 0; i < input.length; i++) {
+            if (i % 2 == 0) {
                 input[i] = Integer.toString(1);
             } else {
                 input[i] = Integer.toString(0);
@@ -183,8 +177,8 @@ public class TestCombiner {
         pigServer.registerQuery("a = load 'MultiCombinerUseInput.txt' as (x:int);");
         pigServer.registerQuery("b = group a all;");
         pigServer.registerQuery("c = foreach b generate COUNT(a), SUM(a.$0), " +
-        		"MIN(a.$0), MAX(a.$0), AVG(a.$0), ((double)SUM(a.$0))/COUNT(a.$0)," +
-        		" COUNT(a.$0) + SUM(a.$0) +  MAX(a.$0);");
+                "MIN(a.$0), MAX(a.$0), AVG(a.$0), ((double)SUM(a.$0))/COUNT(a.$0)," +
+                " COUNT(a.$0) + SUM(a.$0) +  MAX(a.$0);");
 
         // make sure there is a combine plan in the explain output
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -201,34 +195,34 @@ public class TestCombiner {
         assertEquals(0.5, t.get(4));
         assertEquals(0.5, t.get(5));
         assertEquals(512000L + 256000L + 1, t.get(6));
-        
+
         assertFalse(it.hasNext());
         Util.deleteFile(cluster, "MultiCombinerUseInput.txt");
     }
-    
+
     @Test
     public void testDistinctAggs1() throws Exception {
         // test the use of combiner for distinct aggs:
         String input[] = {
-                "pig1\t18\t2.1",
-                "pig2\t24\t3.3",
-                "pig5\t45\t2.4",
-                "pig1\t18\t2.1",
-                "pig1\t19\t2.1",
-                "pig2\t24\t4.5",
-                "pig1\t20\t3.1" };
+                        "pig1\t18\t2.1",
+                        "pig2\t24\t3.3",
+                        "pig5\t45\t2.4",
+                        "pig1\t18\t2.1",
+                        "pig1\t19\t2.1",
+                        "pig2\t24\t4.5",
+                        "pig1\t20\t3.1" };
 
         Util.createInputFile(cluster, "distinctAggs1Input.txt", input);
         PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         pigServer.registerQuery("a = load 'distinctAggs1Input.txt' as (name:chararray, age:int, gpa:double);");
         pigServer.registerQuery("b = group a by name;");
         pigServer.registerQuery("c = foreach b  {" +
-        		"        x = distinct a.age;" +
-        		"        y = distinct a.gpa;" +
-        		"        z = distinct a;" +
-        		"        generate group, COUNT(x), SUM(x.age), SUM(y.gpa), SUM(a.age), " +
-        		"                       SUM(a.gpa), COUNT(z.age), COUNT(z), SUM(z.age);};");
-        
+                "        x = distinct a.age;" +
+                "        y = distinct a.gpa;" +
+                "        z = distinct a;" +
+                "        generate group, COUNT(x), SUM(x.age), SUM(y.gpa), SUM(a.age), " +
+                "                       SUM(a.gpa), COUNT(z.age), COUNT(z), SUM(z.age);};");
+
         // make sure there is a combine plan in the explain output
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         PrintStream ps = new PrintStream(baos);
@@ -236,11 +230,11 @@ public class TestCombiner {
         assertTrue(baos.toString().matches("(?si).*combine plan.*"));
 
         HashMap<String, Object[]> results = new HashMap<String, Object[]>();
-        results.put("pig1", new Object[] {"pig1",3L,57L,5.2,75L,9.4,3L,3L,57L});
-        results.put("pig2", new Object[] {"pig2",1L,24L,7.8,48L,7.8,2L,2L,48L});
-        results.put("pig5", new Object[] {"pig5",1L,45L,2.4,45L,2.4,1L,1L,45L});
+        results.put("pig1", new Object[] { "pig1", 3L, 57L, 5.2, 75L, 9.4, 3L, 3L, 57L });
+        results.put("pig2", new Object[] { "pig2", 1L, 24L, 7.8, 48L, 7.8, 2L, 2L, 48L });
+        results.put("pig5", new Object[] { "pig5", 1L, 45L, 2.4, 45L, 2.4, 1L, 1L, 45L });
         Iterator<Tuple> it = pigServer.openIterator("c");
-        while(it.hasNext()) {
+        while (it.hasNext()) {
             Tuple t = it.next();
             List<Object> fields = t.getAll();
             Object[] expected = results.get((String)fields.get(0));
@@ -250,27 +244,27 @@ public class TestCombiner {
             }
         }
         Util.deleteFile(cluster, "distinctAggs1Input.txt");
-        
+
     }
-    
+
     @Test
     public void testGroupElements() throws Exception {
         // test use of combiner when group elements are accessed in the foreach
         String input[] = {
-                "ABC\t1\ta\t1",
-                "ABC\t1\tb\t2",
-                "ABC\t1\ta\t3",
-                "ABC\t2\tb\t4",
-                "DEF\t1\td\t1",
-                "XYZ\t1\tx\t2"
+                        "ABC\t1\ta\t1",
+                        "ABC\t1\tb\t2",
+                        "ABC\t1\ta\t3",
+                        "ABC\t2\tb\t4",
+                        "DEF\t1\td\t1",
+                        "XYZ\t1\tx\t2"
         };
 
         Util.createInputFile(cluster, "testGroupElements.txt", input);
         PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         pigServer.registerQuery("a = load 'testGroupElements.txt' as (str:chararray, num1:int, alph : chararray, num2 : int);");
         pigServer.registerQuery("b = group a by (str, num1);");
-        
-        //check if combiner is present or not for various forms of foreach
+
+        // check if combiner is present or not for various forms of foreach
         pigServer.registerQuery("c = foreach b  generate flatten(group), COUNT(a.alph), SUM(a.num2); ");
         checkCombinerUsed(pigServer, "c", true);
 
@@ -283,100 +277,97 @@ public class TestCombiner {
 
         // projecting bag - combiner should not be used
         pigServer.registerQuery("c = foreach b  generate group, a.num2,  COUNT(a.alph), SUM(a.num2); ");
-        checkCombinerUsed(pigServer, "c", false);      
-        
+        checkCombinerUsed(pigServer, "c", false);
+
         pigServer.registerQuery("c = foreach b  generate group.$0, group.$1, COUNT(a.alph), SUM(a.num2); ");
         checkCombinerUsed(pigServer, "c", true);
 
         pigServer.registerQuery("c = foreach b  generate group.$0, group.$1 + COUNT(a.alph), SUM(a.num2); ");
         checkCombinerUsed(pigServer, "c", true);
-        
+
         pigServer.registerQuery("c = foreach b  generate group.str, group.$1, COUNT(a.alph), SUM(a.num2); ");
         checkCombinerUsed(pigServer, "c", true);
-        
-        pigServer.registerQuery("c = foreach b  generate group.str, group.$1, COUNT(a.alph), SUM(a.num2), " +
-        		" (group.num1 == 1 ? (COUNT(a.num2) + 1)  : (SUM(a.num2) + 10)) ; ");
+
+        pigServer.registerQuery("c = foreach b  generate group.str, group.$1, COUNT(a.alph), SUM(a.num2), "
+                        + " (group.num1 == 1 ? (COUNT(a.num2) + 1)  : (SUM(a.num2) + 10)) ; ");
         checkCombinerUsed(pigServer, "c", true);
 
-        List<Tuple> expectedRes = 
-            Util.getTuplesFromConstantTupleStrings(
-                    new String[] {
-                            "('ABC',1,3L,6L,4L)",
-                            "('ABC',2,1L,4L,14L)",
-                            "('DEF',1,1L,1L,2L)",
-                            "('XYZ',1,1L,2L,2L)",
-                    });
+        List<Tuple> expectedRes =
+                Util.getTuplesFromConstantTupleStrings(
+                        new String[] {
+                                        "('ABC',1,3L,6L,4L)",
+                                        "('ABC',2,1L,4L,14L)",
+                                        "('DEF',1,1L,1L,2L)",
+                                        "('XYZ',1,1L,2L,2L)",
+                        });
 
         Iterator<Tuple> it = pigServer.openIterator("c");
         Util.checkQueryOutputsAfterSort(it, expectedRes);
 
         Util.deleteFile(cluster, "distinctAggs1Input.txt");
-        
+
     }
-    
+
     @Test
     public void testGroupByLimit() throws Exception {
         // test use of combiner when group elements are accessed in the foreach
         String input[] = {
-                "ABC 1",
-                "ABC 2",
-                "DEF 1",
-                "XYZ 1",
-                "XYZ 2",
-                "XYZ 3",
+                        "ABC 1",
+                        "ABC 2",
+                        "DEF 1",
+                        "XYZ 1",
+                        "XYZ 2",
+                        "XYZ 3",
         };
 
         Util.createInputFile(cluster, "testGroupLimit.txt", input);
         PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
         pigServer.registerQuery("a = load 'testGroupLimit.txt'  using PigStorage(' ') " +
-        		"as (str:chararray, num1:int) ;");
+                "as (str:chararray, num1:int) ;");
         pigServer.registerQuery("b = group a by str;");
-        
 
         pigServer.registerQuery("c = foreach b  generate group, COUNT(a.num1) ; ");
 
-        //check if combiner is present 
+        // check if combiner is present
         pigServer.registerQuery("d = limit c 2 ; ");
         checkCombinerUsed(pigServer, "d", true);
-        
-        List<Tuple> expectedRes = 
-            Util.getTuplesFromConstantTupleStrings(
-                    new String[] {
-                            "('ABC',2L)",
-                            "('DEF',1L)",
-                    });
+
+        List<Tuple> expectedRes =
+                Util.getTuplesFromConstantTupleStrings(
+                        new String[] {
+                                        "('ABC',2L)",
+                                        "('DEF',1L)",
+                        });
 
         Iterator<Tuple> it = pigServer.openIterator("d");
         Util.checkQueryOutputsAfterSort(it, expectedRes);
-        
 
     }
 
     private void checkCombinerUsed(PigServer pigServer, String string, boolean combineExpected)
-    throws IOException {
+            throws IOException {
         // make sure there is a combine plan in the explain output
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         PrintStream ps = new PrintStream(baos);
         pigServer.explain("c", ps);
-        boolean combinerFound = baos.toString().matches("(?si).*combine plan.*"); 
+        boolean combinerFound = baos.toString().matches("(?si).*combine plan.*");
         System.out.println(baos.toString());
         assertEquals("is combiner present as expected", combineExpected, combinerFound);
     }
 
-
     @Test
     public void testDistinctNoCombiner() throws Exception {
         // test that combiner is NOT invoked when
         // one of the elements in the foreach generate
         // is a distinct() as the leaf
         String input[] = {
-                "pig1\t18\t2.1",
-                "pig2\t24\t3.3",
-                "pig5\t45\t2.4",
-                "pig1\t18\t2.1",
-                "pig1\t19\t2.1",
-                "pig2\t24\t4.5",
-                "pig1\t20\t3.1" };
+                        "pig1\t18\t2.1",
+                        "pig2\t24\t3.3",
+                        "pig5\t45\t2.4",
+                        "pig1\t18\t2.1",
+                        "pig1\t19\t2.1",
+                        "pig2\t24\t4.5",
+                        "pig1\t20\t3.1" };
 
         Util.createInputFile(cluster, "distinctNoCombinerInput.txt", input);
         PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
@@ -385,7 +376,7 @@ public class TestCombiner {
         pigServer.registerQuery("c = foreach b  {" +
                 "        z = distinct a;" +
                 "        generate group, z, SUM(a.age), SUM(a.gpa);};");
-        
+
         // make sure there is a combine plan in the explain output
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         PrintStream ps = new PrintStream(baos);
@@ -393,17 +384,17 @@ public class TestCombiner {
         assertFalse(baos.toString().matches("(?si).*combine plan.*"));
 
         HashMap<String, Object[]> results = new HashMap<String, Object[]>();
-        results.put("pig1", new Object[] {"pig1","bag-place-holder",75L,9.4});
-        results.put("pig2", new Object[] {"pig2","bag-place-holder",48L,7.8});
-        results.put("pig5", new Object[] {"pig5","bag-place-holder",45L,2.4});
+        results.put("pig1", new Object[] { "pig1", "bag-place-holder", 75L, 9.4 });
+        results.put("pig2", new Object[] { "pig2", "bag-place-holder", 48L, 7.8 });
+        results.put("pig5", new Object[] { "pig5", "bag-place-holder", 45L, 2.4 });
         Iterator<Tuple> it = pigServer.openIterator("c");
-        while(it.hasNext()) {
+        while (it.hasNext()) {
             Tuple t = it.next();
             List<Object> fields = t.getAll();
             Object[] expected = results.get((String)fields.get(0));
             int i = 0;
             for (Object field : fields) {
-                if(i == 1) {
+                if (i == 1) {
                     // ignore the second field which is a bag
                     // for comparison here
                     continue;
@@ -412,7 +403,7 @@ public class TestCombiner {
             }
         }
         Util.deleteFile(cluster, "distinctNoCombinerInput.txt");
-        
+
     }
 
     @Test
@@ -421,13 +412,13 @@ public class TestCombiner {
         // one of the elements in the foreach generate
         // has a foreach in the plan without a distinct agg
         String input[] = {
-                "pig1\t18\t2.1",
-                "pig2\t24\t3.3",
-                "pig5\t45\t2.4",
-                "pig1\t18\t2.1",
-                "pig1\t19\t2.1",
-                "pig2\t24\t4.5",
-                "pig1\t20\t3.1" };
+                        "pig1\t18\t2.1",
+                        "pig2\t24\t3.3",
+                        "pig5\t45\t2.4",
+                        "pig1\t18\t2.1",
+                        "pig1\t19\t2.1",
+                        "pig2\t24\t4.5",
+                        "pig1\t20\t3.1" };
 
         Util.createInputFile(cluster, "forEachNoCombinerInput.txt", input);
         PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
@@ -436,7 +427,7 @@ public class TestCombiner {
         pigServer.registerQuery("c = foreach b  {" +
                 "        z = a.age;" +
                 "        generate group, z, SUM(a.age), SUM(a.gpa);};");
-        
+
         // make sure there is a combine plan in the explain output
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         PrintStream ps = new PrintStream(baos);
@@ -444,17 +435,17 @@ public class TestCombiner {
         assertFalse(baos.toString().matches("(?si).*combine plan.*"));
 
         HashMap<String, Object[]> results = new HashMap<String, Object[]>();
-        results.put("pig1", new Object[] {"pig1","bag-place-holder",75L,9.4});
-        results.put("pig2", new Object[] {"pig2","bag-place-holder",48L,7.8});
-        results.put("pig5", new Object[] {"pig5","bag-place-holder",45L,2.4});
+        results.put("pig1", new Object[] { "pig1", "bag-place-holder", 75L, 9.4 });
+        results.put("pig2", new Object[] { "pig2", "bag-place-holder", 48L, 7.8 });
+        results.put("pig5", new Object[] { "pig5", "bag-place-holder", 45L, 2.4 });
         Iterator<Tuple> it = pigServer.openIterator("c");
-        while(it.hasNext()) {
+        while (it.hasNext()) {
             Tuple t = it.next();
             List<Object> fields = t.getAll();
             Object[] expected = results.get((String)fields.get(0));
             int i = 0;
             for (Object field : fields) {
-                if(i == 1) {
+                if (i == 1) {
                     // ignore the second field which is a bag
                     // for comparison here
                     continue;
@@ -463,81 +454,74 @@ public class TestCombiner {
             }
         }
         Util.deleteFile(cluster, "forEachNoCombinerInput.txt");
-        
+
     }
-    
+
     @Test
-    public void testJiraPig746() {
+    public void testJiraPig746() throws Exception {
         // test that combiner is NOT invoked when
         // one of the elements in the foreach generate
         // has a foreach in the plan without a distinct agg
         String input[] = {
-                "pig1\t18\t2.1",
-                "pig2\t24\t3.3",
-                "pig5\t45\t2.4",
-                "pig1\t18\t2.1",
-                "pig1\t19\t2.1",
-                "pig2\t24\t4.5",
-                "pig1\t20\t3.1" };
-        
+                        "pig1\t18\t2.1",
+                        "pig2\t24\t3.3",
+                        "pig5\t45\t2.4",
+                        "pig1\t18\t2.1",
+                        "pig1\t19\t2.1",
+                        "pig2\t24\t4.5",
+                        "pig1\t20\t3.1" };
+
         String expected[] = {
-                "(pig1,75,{(pig1,18,2.1),(pig1,18,2.1),(pig1,19,2.1),(pig1,20,3.1)})",
-                "(pig2,48,{(pig2,24,3.3),(pig2,24,4.5)})",
-                "(pig5,45,{(pig5,45,2.4)})"
+                        "(pig1,75,{(pig1,18,2.1),(pig1,18,2.1),(pig1,19,2.1),(pig1,20,3.1)})",
+                        "(pig2,48,{(pig2,24,3.3),(pig2,24,4.5)})",
+                        "(pig5,45,{(pig5,45,2.4)})"
         };
 
         try {
             Util.createInputFile(cluster, "forEachNoCombinerInput.txt", input);
- 
+
             PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
             pigServer.registerQuery("a = load 'forEachNoCombinerInput.txt' as (name:chararray, age:int, gpa:double);");
             pigServer.registerQuery("b = group a by name;");
             pigServer.registerQuery("c = foreach b generate group, SUM(a.age), a;");
-            
+
             // make sure there isn't a combine plan in the explain output
             ByteArrayOutputStream baos = new ByteArrayOutputStream();
             PrintStream ps = new PrintStream(baos);
             pigServer.explain("c", ps);
             assertFalse(baos.toString().matches("(?si).*combine plan.*"));
-    
+
             Iterator<Tuple> it = pigServer.openIterator("c");
-            Util.checkQueryOutputsAfterSortRecursive(it, expected, "group:chararray,age:long,b:{t:(name:chararray,age:int,gpa:double)}");
-        } catch (IOException e) {
-            e.printStackTrace();
-            Assert.fail();
+            Util.checkQueryOutputsAfterSortRecursive(it, expected,
+                    "group:chararray,age:long,b:{t:(name:chararray,age:int,gpa:double)}");
         } finally {
-            try {
-                Util.deleteFile(cluster, "forEachNoCombinerInput.txt");
-            } catch (IOException e) {
-                e.printStackTrace();
-                Assert.fail();
-            }
+            Util.deleteFile(cluster, "forEachNoCombinerInput.txt");
         }
     }
 
     public static class JiraPig1030 extends EvalFunc<DataBag> {
-        
+
         public DataBag exec(Tuple input) throws IOException {
             return new DefaultDataBag();
         }
     }
-   
+
     @Test
-    public void testJiraPig1030() {
+    public void testJiraPig1030() throws Exception {
         // test that combiner is NOT invoked when
         // one of the elements in the foreach generate
         // has a non-algebraic UDF that have multiple inputs
         // (one of them is distinct).
-        
+
         String input[] = {
-                "pig1\t18\t2.1",
-                "pig2\t24\t3.3",
-                "pig5\t45\t2.4",
-                "pig1\t18\t2.1",
-                "pig1\t19\t2.1",
-                "pig2\t24\t4.5",
-                "pig1\t20\t3.1" };
- 
+                        "pig1\t18\t2.1",
+                        "pig2\t24\t3.3",
+                        "pig5\t45\t2.4",
+                        "pig1\t18\t2.1",
+                        "pig1\t19\t2.1",
+                        "pig2\t24\t4.5",
+                        "pig1\t20\t3.1" };
+
         try {
             Util.createInputFile(cluster, "forEachNoCombinerInput.txt", input);
             PigServer pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
@@ -546,23 +530,14 @@ public class TestCombiner {
             pigServer.registerQuery("c = foreach b  {" +
                     "        d = distinct a.age;" +
                     "        generate group, " + JiraPig1030.class.getName() + "(d, 0);};");
-            
+
             // make sure there isn't a combine plan in the explain output
             ByteArrayOutputStream baos = new ByteArrayOutputStream();
             PrintStream ps = new PrintStream(baos);
             pigServer.explain("c", ps);
-            assertFalse(baos.toString().matches("(?si).*combine plan.*"));    
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
+            assertFalse(baos.toString().matches("(?si).*combine plan.*"));
         } finally {
-            try {
-                Util.deleteFile(cluster, "forEachNoCombinerInput.txt");
-            } catch (IOException e) {
-                e.printStackTrace();
-                Assert.fail();
-            }
+            Util.deleteFile(cluster, "forEachNoCombinerInput.txt");
         }
     }
-
 }

Modified: pig/trunk/test/org/apache/pig/test/TestConstExpr.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestConstExpr.java?rev=1407167&r1=1407166&r2=1407167&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestConstExpr.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestConstExpr.java Thu Nov  8 17:06:46 2012
@@ -1,14 +1,12 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
+ * or more contributor license agreements. See the NOTICE file
  * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
+ * 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
- *
+ * 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.
@@ -17,34 +15,25 @@
  */
 package org.apache.pig.test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 
 import java.util.Map;
 import java.util.Random;
 
 import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.ConstantExpression;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.ConstantExpression;
 import org.apache.pig.test.utils.GenPhyOp;
 import org.apache.pig.test.utils.GenRandomData;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 
-public class TestConstExpr extends junit.framework.TestCase {
-    Random r = new Random();
-    ConstantExpression ce = (ConstantExpression) GenPhyOp.exprConst();
-    
-    @Before
-    public void setUp() throws Exception {
-    }
-
-    @After
-    public void tearDown() throws Exception {
-    }
+public class TestConstExpr {
+    Random r = new Random(42L);
+    ConstantExpression ce = (ConstantExpression)GenPhyOp.exprConst();
 
     @Test
     public void testGetNextInteger() throws ExecException {
@@ -57,8 +46,8 @@ public class TestConstExpr extends junit
         // test with null input
         ce.setValue(null);
         resi = ce.getNext(inp);
-        ret  = (Integer)resi.result;
-        assertEquals(null, ret);
+        ret = (Integer)resi.result;
+        assertNull(ret);
     }
 
     @Test
@@ -72,8 +61,8 @@ public class TestConstExpr extends junit
         // test with null input
         ce.setValue(null);
         resl = ce.getNext(inp);
-        ret  = (Long)resl.result;
-        assertEquals(null, ret);
+        ret = (Long)resl.result;
+        assertNull(ret);
     }
 
     @Test
@@ -87,8 +76,8 @@ public class TestConstExpr extends junit
         // test with null input
         ce.setValue(null);
         resd = ce.getNext(inp);
-        ret  = (Double)resd.result;
-        assertEquals(null, ret);
+        ret = (Double)resd.result;
+        assertNull(ret);
     }
 
     @Test
@@ -102,8 +91,8 @@ public class TestConstExpr extends junit
         // test with null input
         ce.setValue(null);
         resf = ce.getNext(inp);
-        ret  = (Float)resf.result;
-        assertEquals(null, ret);
+        ret = (Float)resf.result;
+        assertNull(ret);
     }
 
     @Test
@@ -117,8 +106,8 @@ public class TestConstExpr extends junit
         // test with null input
         ce.setValue(null);
         ress = ce.getNext(inp);
-        ret  = (String)ress.result;
-        assertEquals(null, ret);
+        ret = (String)ress.result;
+        assertNull(ret);
     }
 
     @Test
@@ -132,23 +121,23 @@ public class TestConstExpr extends junit
         // test with null input
         ce.setValue(null);
         resba = ce.getNext(inp);
-        ret  = (DataByteArray)resba.result;
-        assertEquals(null, ret);
+        ret = (DataByteArray)resba.result;
+        assertNull(ret);
     }
 
     @Test
     public void testGetNextMap() throws ExecException {
-        Map<String,Object> inp = GenRandomData.genRandMap(r, 10);
+        Map<String, Object> inp = GenRandomData.genRandMap(r, 10);
         ce.setValue(inp);
         Result resm = ce.getNext(inp);
-        Map<Integer,String> ret = (Map)resm.result;
+        Map<Integer, String> ret = (Map)resm.result;
         assertEquals(inp, ret);
 
         // test with null input
         ce.setValue(null);
         resm = ce.getNext(inp);
-        ret  = (Map)resm.result;
-        assertEquals(null, ret);
+        ret = (Map)resm.result;
+        assertNull(ret);
     }
 
     @Test
@@ -162,8 +151,8 @@ public class TestConstExpr extends junit
         // test with null input
         ce.setValue(null);
         res = ce.getNext(inp);
-        ret  = (Boolean)res.result;
-        assertEquals(null, ret);
+        ret = (Boolean)res.result;
+        assertNull(ret);
     }
 
     @Test
@@ -177,8 +166,8 @@ public class TestConstExpr extends junit
         // test with null input
         ce.setValue(null);
         rest = ce.getNext(inp);
-        ret  = (Tuple)rest.result;
-        assertEquals(null, ret);
+        ret = (Tuple)rest.result;
+        assertNull(ret);
     }
 
     @Test
@@ -192,8 +181,8 @@ public class TestConstExpr extends junit
         // test with null input
         ce.setValue(null);
         res = ce.getNext(inp);
-        ret  = (DataBag)res.result;
-        assertEquals(null, ret);
+        ret = (DataBag)res.result;
+        assertNull(ret);
     }
 
 }

Modified: pig/trunk/test/org/apache/pig/test/TestDataModel.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestDataModel.java?rev=1407167&r1=1407166&r2=1407167&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestDataModel.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestDataModel.java Thu Nov  8 17:06:46 2012
@@ -1,14 +1,12 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
+ * or more contributor license agreements. See the NOTICE file
  * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
+ * 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
- *
+ * 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.
@@ -17,36 +15,43 @@
  */
 package org.apache.pig.test;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.PipedInputStream;
-import java.io.PipedOutputStream;
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.PipedInputStream;
+import java.io.PipedOutputStream;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Random;
-import java.util.LinkedHashMap;
-
-import org.junit.Test;
 
 import org.apache.pig.backend.executionengine.ExecException;
-import org.apache.pig.data.*;
+import org.apache.pig.data.BagFactory;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.InternalMap;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.junit.Test;
 
 /**
  * This class will exercise the basic Pig data model and members. It tests for proper behavior in
  * assigment and comparision, as well as function application.
- * 
- * @author dnm
  */
-public class TestDataModel extends junit.framework.TestCase {
+public class TestDataModel {
 
     @Test
     public void testDatum() throws Exception {
@@ -67,8 +72,8 @@ public class TestDataModel extends junit
         Tuple f1 = Util.loadFlatTuple(tf.newTuple(arity), input1);
         Tuple f2 = Util.loadFlatTuple(tf.newTuple(arity), input1);
         Tuple f3 = tf.newTuple(arity);
-        assertTrue(f1.size() == arity);
-        assertTrue(f1.equals(f2));
+        assertEquals(arity, f1.size());
+        assertEquals(f1, f2);
 
         // invalid equality
         f2 = Util.loadTuple(tf.newTuple(input3.length), input3);
@@ -76,20 +81,20 @@ public class TestDataModel extends junit
 
         // copy equality
         /*
-        f2.copyFrom(f1);
-        assertTrue(f1.equals(f2));
-        */
+         * f2.copyFrom(f1);
+         * assertTrue(f1.equals(f2));
+         */
 
         // append function and equality
         int[] input4 = { 1, 2, 3 };
         int[] input5 = { 4, 5 };
         /*
-        f1 = Util.loadFlatTuple(tf.newTuple(input4.length), input4);
-        f2 = Util.loadFlatTuple(tf.newTuple(input5.length), input5);
-        f3 = Util.loadFlatTuple(tf.newTuple(input1.length), input1);
-        f1.appendTuple(f2);
-        assertTrue(f3.equals(f1));
-        */
+         * f1 = Util.loadFlatTuple(tf.newTuple(input4.length), input4);
+         * f2 = Util.loadFlatTuple(tf.newTuple(input5.length), input5);
+         * f3 = Util.loadFlatTuple(tf.newTuple(input1.length), input1);
+         * f1.appendTuple(f2);
+         * assertTrue(f3.equals(f1));
+         */
 
         // arity then value comparision behavior
         f1 = Util.loadFlatTuple(tf.newTuple(input1.length), input1); // 1,2,3,4,5
@@ -124,8 +129,8 @@ public class TestDataModel extends junit
     public void testNestTuple() throws Exception {
         TupleFactory tf = TupleFactory.getInstance();
 
-        int[][] input1 = { { 1, 2, 3, 4, 5 }, { 1, 2, 3, 4, 5 }, { 1, 2, 3, 4, 5 }, { 1, 2, 3, 4, 5 },
-                { 1, 2, 3, 4, 5 } };
+        int[][] input1 = { { 1, 2, 3, 4, 5 }, { 1, 2, 3, 4, 5 }, { 1, 2, 3, 4, 5 },
+                           { 1, 2, 3, 4, 5 }, { 1, 2, 3, 4, 5 } };
         int[][] input2 = { { 1, 2 }, { 1, 2 } };
 
         Tuple n1 = Util.loadNestTuple(tf.newTuple(input1.length), input1);
@@ -192,22 +197,22 @@ public class TestDataModel extends junit
             assertTrue("isa Integer", o instanceof Integer);
             Integer ii = (Integer)o;
             assertEquals(new Integer(42), ii);
-            
+
             o = after.get(4);
             assertTrue("isa Long", o instanceof Long);
             Long l = (Long)o;
             assertEquals(new Long(5000000000L), l);
-            
+
             o = after.get(5);
             assertTrue("isa Float", o instanceof Float);
             Float f = (Float)o;
             assertEquals(new Float(3.141592654), f);
-            
+
             o = after.get(6);
             assertTrue("isa Double", o instanceof Double);
             Double d = (Double)o;
-            assertEquals(2.99792458e8, d);
-            
+            assertEquals(new Double(2.99792458e8), d);
+
             o = after.get(7);
             assertTrue("isa Boolean", o instanceof Boolean);
             Boolean bool = (Boolean)o;
@@ -222,7 +227,7 @@ public class TestDataModel extends junit
             assertTrue("isa String", o instanceof String);
             String s = (String)o;
             assertEquals("goodbye", s);
-         }
+        }
 
         file.delete();
     }
@@ -248,13 +253,13 @@ public class TestDataModel extends junit
 
         FileInputStream fis = new FileInputStream(file);
         DataInput in = new DataInputStream(fis);
- 
+
         Tuple after = tf.newTuple();
         after.readFields(in);
 
         Object o = after.get(0);
         assertTrue("isa InternalMap", o instanceof InternalMap);
-        
+
         InternalMap m = (InternalMap)o;
         assertEquals("world", (String)m.get(new Integer(1)));
         assertEquals("all", (String)m.get(new Long(3L)));
@@ -267,7 +272,10 @@ public class TestDataModel extends junit
     public void testTupleToString() throws Exception {
         Tuple t = giveMeOneOfEach();
 
-        assertEquals("toString", "((3,3.0),{(4),(mary had a little lamb)},[hello#world,goodbye#all],42,5000000000,3.1415927,2.99792458E8,true,hello,goodbye,)", t.toString());
+        assertEquals(
+                "toString",
+                "((3,3.0),{(4),(mary had a little lamb)},[hello#world,goodbye#all],42,5000000000,3.1415927,2.99792458E8,true,hello,goodbye,)",
+                t.toString());
     }
 
     @Test
@@ -288,12 +296,12 @@ public class TestDataModel extends junit
         t3.set(0, new DataByteArray("hello world"));
         t3.set(1, new Integer(1));
         t3.set(2, new Long(4));
-        assertFalse("different size", t1.hashCode() == t3.hashCode()); 
+        assertFalse("different size", t1.hashCode() == t3.hashCode());
 
         Tuple t4 = tf.newTuple(2);
         t4.set(0, new DataByteArray("hello world"));
         t4.set(1, new Integer(2));
-        assertFalse("same size, different data", t1.hashCode() == t4.hashCode()); 
+        assertFalse("same size, different data", t1.hashCode() == t4.hashCode());
 
         // Make sure we can take the hash code of all the types.
         Tuple t5 = giveMeOneOfEach();
@@ -306,7 +314,7 @@ public class TestDataModel extends junit
 
         Tuple t1 = tf.newTuple();
         Tuple t2 = tf.newTuple();
-    
+
         t1.append(new Integer(3));
         t2.append(new Integer(3));
 
@@ -334,7 +342,7 @@ public class TestDataModel extends junit
         t1.append(new Integer(3));
         t2.append(new Integer(3));
 
-        assertEquals("same data equal", 0,  t1.compareTo(t2));
+        assertEquals("same data equal", 0, t1.compareTo(t2));
 
         t2 = tf.newTuple();
         t2.append(new Integer(2));
@@ -365,7 +373,7 @@ public class TestDataModel extends junit
         t2.append(new DataByteArray("bbb"));
         t2.append(new DataByteArray("bbb"));
 
-        assertEquals("same data equal", 0,  t1.compareTo(t2));
+        assertEquals("same data equal", 0, t1.compareTo(t2));
 
         t2 = tf.newTuple();
         t2.append(new DataByteArray("aaa"));
@@ -429,7 +437,7 @@ public class TestDataModel extends junit
 
         assertEquals("same data", ba1.hashCode(), ba2.hashCode());
 
-        assertFalse("different data", ba1.hashCode() == ba3.hashCode()); 
+        assertFalse("different data", ba1.hashCode() == ba3.hashCode());
     }
 
     @Test
@@ -440,7 +448,7 @@ public class TestDataModel extends junit
 
         assertTrue("same data", ba1.equals(ba2));
 
-        assertFalse("different data", ba1.equals(ba3)); 
+        assertFalse("different data", ba1.equals(ba3));
     }
 
     @Test
@@ -452,77 +460,77 @@ public class TestDataModel extends junit
         assertTrue("same data", ba1.compareTo(ba2) == 0);
 
         assertTrue("different length lexically lower value less than",
-            ba3.compareTo(ba1) < 0);
+                ba3.compareTo(ba1) < 0);
         assertTrue("different length lexically higher value greater than",
-            ba1.compareTo(ba3) > 0);
+                ba1.compareTo(ba3) > 0);
 
         ba2 = new DataByteArray("hello worlc");
         assertTrue("same length lexically lower value less than",
-            ba2.compareTo(ba1) < 0);
+                ba2.compareTo(ba1) < 0);
         assertTrue("same length lexically higher value greater than",
-            ba1.compareTo(ba2) > 0);
+                ba1.compareTo(ba2) > 0);
 
         ba2 = new DataByteArray("hello worlds");
         assertTrue("shorter lexically same value less than",
-            ba1.compareTo(ba2) < 0);
+                ba1.compareTo(ba2) < 0);
         assertTrue("longer lexically same value greater than",
-            ba2.compareTo(ba1) > 0);
+                ba2.compareTo(ba1) > 0);
 
     }
-    
-    @Test
+
+    @Test(expected = ExecException.class)
     public void testIntegerConversionErr() throws Exception {
-    	List list = new ArrayList();
-    	try {
-    		DataType.toInteger(list);
-    		fail("Error expected.");
-    	} catch (ExecException ee) {
-    		assertTrue(ee.getErrorCode() == 1071);
-    	}
+        List list = new ArrayList();
+        try {
+            DataType.toInteger(list);
+        } catch (ExecException ee) {
+            assertEquals(1071, ee.getErrorCode());
+            throw ee;
+        }
     }
 
-    @Test
+    @Test(expected = ExecException.class)
     public void testIntegerConversionErr1() throws Exception {
-    	DataByteArray ba = new DataByteArray("hello world");
-    	try {
-    		DataType.toInteger(ba);
-    		fail("Error expected.");
-    	} catch (ExecException ee) {
-    		assertTrue(ee.getErrorCode() == 1074);
-    	}
+        DataByteArray ba = new DataByteArray("hello world");
+        try {
+            DataType.toInteger(ba);
+        } catch (ExecException ee) {
+            assertEquals(1074, ee.getErrorCode());
+            throw ee;
+        }
     }
 
-    @Test
+    @Test(expected = ExecException.class)
     public void testTupleConversionErr() throws Exception {
-    	List list = new ArrayList();
-    	try {
-    		DataType.toTuple(list);
-    		fail("Error expected.");
-    	} catch (ExecException ee) {
-    		assertTrue(ee.getErrorCode() == 1071);
-    	}
+        List list = new ArrayList();
+        try {
+            DataType.toTuple(list);
+        } catch (ExecException ee) {
+            assertEquals(1071, ee.getErrorCode());
+            throw ee;
+        }
     }
 
-    @Test
+    @Test(expected = ExecException.class)
     public void testTupleConversionErr1() throws Exception {
-    	DataByteArray ba = new DataByteArray("hello world");
-    	try {
-    		DataType.toTuple(ba);
-    		fail("Error expected.");
-    	} catch (ExecException ee) {
-    		assertTrue(ee.getErrorCode() == 1071);
-    	}
+        DataByteArray ba = new DataByteArray("hello world");
+        try {
+            DataType.toTuple(ba);
+        } catch (ExecException ee) {
+            assertEquals(1071, ee.getErrorCode());
+            throw ee;
+        }
     }
 
     @Test
     public void testByteArrayAppend() throws Exception {
-    	DataByteArray expected = new DataByteArray("hello world");
+        DataByteArray expected = new DataByteArray("hello world");
         DataByteArray db1 = new DataByteArray("hello ");
         DataByteArray db2 = new DataByteArray("world");
         db1.append(db2);
-        assertTrue("appends as expected", db1.equals(expected));
+        assertEquals("appends as expected", db1, expected);
     }
-    
+
     @Test
     public void testByteArrayAppendMore() throws Exception {
         DataByteArray expected = new DataByteArray("hello world!");
@@ -530,56 +538,53 @@ public class TestDataModel extends junit
         DataByteArray db2 = new DataByteArray("world");
         DataByteArray db3 = new DataByteArray("!");
         db1.append(db2).append(db3);
-        assertTrue("appends as expected", db1.equals(expected));
+        assertEquals("appends as expected", db1, expected);
     }
-    
+
     @Test
     public void testByteArrayAppendBytes() throws Exception {
         DataByteArray expected = new DataByteArray("hello world");
         DataByteArray db1 = new DataByteArray("hello ");
         byte[] db2 = "world".getBytes();
         db1.append(db2);
-        assertTrue("appends as expected", db1.equals(expected));
+        assertEquals("appends as expected", db1, expected);
     }
-    
+
     @Test
     public void testByteArrayAppendString() throws Exception {
         DataByteArray expected = new DataByteArray("hello world");
         DataByteArray db1 = new DataByteArray("hello ");
         db1.append("world");
-        assertTrue("appends as expected", db1.equals(expected));
+        assertEquals("appends as expected", db1, expected);
     }
 
-    @Test
+    @Test(expected = ExecException.class)
     public void testMapConversionErr() throws Exception {
-    	List list = new ArrayList();
-    	try {
-    		DataType.toMap(list);
-    		fail("Error expected.");
-    	} catch (ExecException ee) {
-    		assertTrue(ee.getErrorCode() == 1071);
-    	}
+        List list = new ArrayList();
+        try {
+            DataType.toMap(list);
+        } catch (ExecException ee) {
+            assertEquals(1071, ee.getErrorCode());
+            throw ee;
+        }
     }
 
     @Test
     public void testMapConversion() throws Exception {
-    	Map<Integer, Float> map = new HashMap<Integer, Float>();
-    	try {
-    		DataType.toMap(map);
-    	} catch (ExecException ee) {
-    		fail("Exception not expected.");
-    	}
+        Map<Integer, Float> map = new HashMap<Integer, Float>();
+        DataType.toMap(map);
     }
 
-    @Test
+    @Test(expected = ExecException.class)
     public void testDetermineFieldSchemaErr() throws Exception {
-    	List list = new ArrayList();
-    	try {
-    		DataType.determineFieldSchema(list);
-    		fail("Error expected.");
-    	} catch (ExecException ee) {
-    		assertTrue(ee.getErrorCode() == 1073);
-    	}
+        List list = new ArrayList();
+        try {
+            DataType.determineFieldSchema(list);
+            fail("Error expected.");
+        } catch (ExecException ee) {
+            assertEquals(1073, ee.getErrorCode());
+            throw ee;
+        }
     }
 
     private Tuple giveMeOneOfEach() throws Exception {
@@ -613,3 +618,4 @@ public class TestDataModel extends junit
         return t1;
     }
 }
+

Modified: pig/trunk/test/org/apache/pig/test/TestDivide.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestDivide.java?rev=1407167&r1=1407166&r2=1407167&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestDivide.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestDivide.java Thu Nov  8 17:06:46 2012
@@ -17,55 +17,53 @@
  */
 package org.apache.pig.test;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 
 import java.util.Map;
 import java.util.Random;
 
-import junit.framework.TestCase;
-
-import org.joda.time.DateTime;
-
 import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.ConstantExpression;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.Divide;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.plan.OperatorKey;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.ConstantExpression;
-import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.Divide;
 import org.apache.pig.test.utils.GenRandomData;
+import org.joda.time.DateTime;
 import org.junit.Before;
 import org.junit.Test;
 
-
-public class TestDivide extends TestCase{
-
-    Random r = new Random();
+public class TestDivide {
+    Random r = new Random(42L);
     ConstantExpression lt, rt;
     Divide op = new Divide(new OperatorKey("", r.nextLong()));
 
     @Before
     public void setUp() throws Exception {
-        lt = new ConstantExpression(new OperatorKey("",r.nextLong()));
-        rt = new ConstantExpression(new OperatorKey("",r.nextLong()));
+        lt = new ConstantExpression(new OperatorKey("", r.nextLong()));
+        rt = new ConstantExpression(new OperatorKey("", r.nextLong()));
     }
 
     @Test
-    public void testOperator() throws ExecException{
-        //int TRIALS = 10;
-        byte[] types = { DataType.BAG, DataType.BOOLEAN, DataType.BYTEARRAY, DataType.CHARARRAY, 
-                DataType.DOUBLE, DataType.FLOAT, DataType.INTEGER, DataType.LONG, DataType.DATETIME, DataType.MAP, DataType.TUPLE};
-        //Map<Byte,String> map = GenRandomData.genTypeToNameMap();
+    public void testOperator() throws ExecException {
+        // int TRIALS = 10;
+        byte[] types = { DataType.BAG, DataType.BOOLEAN, DataType.BYTEARRAY, DataType.CHARARRAY,
+                        DataType.DOUBLE, DataType.FLOAT, DataType.INTEGER, DataType.LONG,
+                        DataType.DATETIME, DataType.MAP, DataType.TUPLE };
+        // Map<Byte,String> map = GenRandomData.genTypeToNameMap();
         System.out.println("Testing DIVIDE operator");
-        for(byte type : types) {
+        for (byte type : types) {
             lt.setResultType(type);
             rt.setResultType(type);
             op.setLhs(lt);
             op.setRhs(rt);
 
-            switch(type){
+            switch (type) {
             case DataType.BAG:
                 DataBag inpdb1 = GenRandomData.genRandSmallTupDataBag(r, 10, 100);
                 DataBag inpdb2 = GenRandomData.genRandSmallTupDataBag(r, 10, 100);

Modified: pig/trunk/test/org/apache/pig/test/TestEqualTo.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestEqualTo.java?rev=1407167&r1=1407166&r2=1407167&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestEqualTo.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestEqualTo.java Thu Nov  8 17:06:46 2012
@@ -1,14 +1,12 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
+ * or more contributor license agreements. See the NOTICE file
  * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
+ * 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
- *
+ * 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.
@@ -17,40 +15,28 @@
  */
 package org.apache.pig.test;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Random;
-
-import org.joda.time.DateTime;
 
 import org.apache.pig.backend.executionengine.ExecException;
-import org.apache.pig.data.DataBag;
-import org.apache.pig.data.DataByteArray;
-import org.apache.pig.data.DataType;
-import org.apache.pig.data.Tuple;
-import org.apache.pig.data.TupleFactory;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.ConstantExpression;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.EqualToExpr;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
 import org.apache.pig.test.utils.GenPhyOp;
-import org.apache.pig.test.utils.GenRandomData;
-import org.junit.After;
-import org.junit.Before;
+import org.joda.time.DateTime;
 import org.junit.Test;
 
-import static org.junit.Assert.*;
-
-public class TestEqualTo extends junit.framework.TestCase {
-
-    @Before
-    public void setUp() throws Exception {
-    }
-
-    @After
-    public void tearDown() throws Exception {
-    }
-    
+public class TestEqualTo {
     @Test
     public void testBooleanNe() throws Exception {
         ConstantExpression lt = GenPhyOp.exprConst();
@@ -65,7 +51,7 @@ public class TestEqualTo extends junit.f
         assertEquals(POStatus.STATUS_OK, r.returnStatus);
         assertFalse((Boolean)r.result);
     }
-    
+
     @Test
     public void testBooleanEq() throws Exception {
         ConstantExpression lt = GenPhyOp.exprConst();
@@ -80,12 +66,10 @@ public class TestEqualTo extends junit.f
         assertEquals(POStatus.STATUS_OK, r.returnStatus);
         assertTrue((Boolean)r.result);
     }
-    
+
     @Test
     public void testBooleanAndNullValues() throws Exception {
-
         checkNullValues(DataType.BOOLEAN, Boolean.TRUE);
-
     }
 
     @Test
@@ -103,7 +87,6 @@ public class TestEqualTo extends junit.f
         assertFalse((Boolean)r.result);
     }
 
-
     @Test
     public void testIntegerEq() throws Exception {
         ConstantExpression lt = GenPhyOp.exprConst();
@@ -119,13 +102,10 @@ public class TestEqualTo extends junit.f
         assertTrue((Boolean)r.result);
     }
 
-	@Test
-	public void testIntegerAndNullValues() throws Exception {
-	    	
-	    checkNullValues(  DataType.INTEGER,  new Integer(1) );
-	        
-	}
-    
+    @Test
+    public void testIntegerAndNullValues() throws Exception {
+        checkNullValues(DataType.INTEGER, new Integer(1));
+    }
 
     @Test
     public void testLongNe() throws Exception {
@@ -157,14 +137,10 @@ public class TestEqualTo extends junit.f
         assertTrue((Boolean)r.result);
     }
 
-    
-	@Test
+    @Test
     public void testLongAndNullValues() throws Exception {
-		
-	    checkNullValues(  DataType.LONG,  new Long(1) );
-
+        checkNullValues(DataType.LONG, new Long(1));
     }
-    
 
     @Test
     public void testFloatNe() throws Exception {
@@ -180,7 +156,7 @@ public class TestEqualTo extends junit.f
         assertEquals(POStatus.STATUS_OK, r.returnStatus);
         assertFalse((Boolean)r.result);
     }
-    
+
     @Test
     public void testFloatEq() throws Exception {
         ConstantExpression lt = GenPhyOp.exprConst();
@@ -197,11 +173,9 @@ public class TestEqualTo extends junit.f
     }
 
     @Test
-	public void testFloatAndNullValues() throws Exception {
-	    checkNullValues(   DataType.FLOAT,  new Float(1.0f) );
-	    
-	}
-
+    public void testFloatAndNullValues() throws Exception {
+        checkNullValues(DataType.FLOAT, new Float(1.0f));
+    }
 
     @Test
     public void testDoubleNe() throws Exception {
@@ -217,8 +191,6 @@ public class TestEqualTo extends junit.f
         assertEquals(POStatus.STATUS_OK, r.returnStatus);
         assertFalse((Boolean)r.result);
     }
-    
- 
 
     @Test
     public void testDoubleEq() throws Exception {
@@ -234,12 +206,11 @@ public class TestEqualTo extends junit.f
         assertEquals(POStatus.STATUS_OK, r.returnStatus);
         assertTrue((Boolean)r.result);
     }
-    
+
     @Test
-	public void testDoubleAndNullValues() throws Exception {
-	    checkNullValues(   DataType.DOUBLE,  new Double(1.0) );
-	    
-	}
+    public void testDoubleAndNullValues() throws Exception {
+        checkNullValues(DataType.DOUBLE, new Double(1.0));
+    }
 
     @Test
     public void testDateTimeNe() throws Exception {
@@ -271,12 +242,9 @@ public class TestEqualTo extends junit.f
         assertTrue((Boolean)r.result);
     }
 
-    
     @Test
     public void testDateTimeAndNullValues() throws Exception {
-        
-        checkNullValues(  DataType.DATETIME,  new DateTime(1L) );
-
+        checkNullValues(DataType.DATETIME, new DateTime(1L));
     }
 
     @Test
@@ -294,7 +262,6 @@ public class TestEqualTo extends junit.f
         assertFalse((Boolean)r.result);
     }
 
-    
     @Test
     public void testStringEq() throws Exception {
         ConstantExpression lt = GenPhyOp.exprConst();
@@ -310,15 +277,12 @@ public class TestEqualTo extends junit.f
         assertTrue((Boolean)r.result);
     }
 
-    
-	@Test
-	public void testStringAndNullValues() throws Exception {
-	    checkNullValues(   DataType.CHARARRAY,  new String("b") );
-		    
-	}
-
+    @Test
+    public void testStringAndNullValues() throws Exception {
+        checkNullValues(DataType.CHARARRAY, new String("b"));
+    }
 
-	@Test
+    @Test
     public void testDataByteArrayNe() throws Exception {
         ConstantExpression lt = GenPhyOp.exprConst();
         lt.setValue(new DataByteArray("a"));
@@ -333,7 +297,6 @@ public class TestEqualTo extends junit.f
         assertFalse((Boolean)r.result);
     }
 
-    
     @Test
     public void testDataByteArrayEq() throws Exception {
         ConstantExpression lt = GenPhyOp.exprConst();
@@ -348,11 +311,11 @@ public class TestEqualTo extends junit.f
         assertEquals(POStatus.STATUS_OK, r.returnStatus);
         assertTrue((Boolean)r.result);
     }
-    
+
     @Test
-    public void testTupleEq() throws ExecException{
-        Tuple tuple_1=TupleFactory.getInstance().newTuple("item_1");
-        Tuple tuple_2=TupleFactory.getInstance().newTuple("item_1");
+    public void testTupleEq() throws ExecException {
+        Tuple tuple_1 = TupleFactory.getInstance().newTuple("item_1");
+        Tuple tuple_2 = TupleFactory.getInstance().newTuple("item_1");
         ConstantExpression lt = GenPhyOp.exprConst();
         lt.setValue(tuple_1);
         ConstantExpression rt = GenPhyOp.exprConst();
@@ -365,11 +328,11 @@ public class TestEqualTo extends junit.f
         assertEquals(POStatus.STATUS_OK, r.returnStatus);
         assertTrue((Boolean)r.result);
     }
-    
+
     @Test
-    public void testTupleNe() throws ExecException{
-        Tuple tuple_1=TupleFactory.getInstance().newTuple("item_1");
-        Tuple tuple_2=TupleFactory.getInstance().newTuple("item_2");
+    public void testTupleNe() throws ExecException {
+        Tuple tuple_1 = TupleFactory.getInstance().newTuple("item_1");
+        Tuple tuple_2 = TupleFactory.getInstance().newTuple("item_2");
         ConstantExpression lt = GenPhyOp.exprConst();
         lt.setValue(tuple_1);
         ConstantExpression rt = GenPhyOp.exprConst();
@@ -382,14 +345,14 @@ public class TestEqualTo extends junit.f
         assertEquals(POStatus.STATUS_OK, r.returnStatus);
         assertFalse((Boolean)r.result);
     }
-    
+
     @Test
-    public void testMapEq() throws ExecException{
-        Map map_1=new HashMap();
+    public void testMapEq() throws ExecException {
+        Map map_1 = new HashMap();
         map_1.put("key_1", "value_1");
-        Map map_2=new HashMap();
+        Map map_2 = new HashMap();
         map_2.put("key_1", "value_1");
-        
+
         ConstantExpression lt = GenPhyOp.exprConst();
         lt.setValue(map_1);
         ConstantExpression rt = GenPhyOp.exprConst();
@@ -402,15 +365,15 @@ public class TestEqualTo extends junit.f
         assertEquals(POStatus.STATUS_OK, r.returnStatus);
         assertTrue((Boolean)r.result);
     }
-    
+
     @Test
-    public void testMapNe() throws ExecException{
-        Map map_1=new HashMap();
+    public void testMapNe() throws ExecException {
+        Map map_1 = new HashMap();
         map_1.put("key_1", "value_1");
-        Map map_2=new HashMap();
+        Map map_2 = new HashMap();
         map_2.put("key_1", "value_2");
-        
-        Tuple tuple_2=TupleFactory.getInstance().newTuple("item_2");
+
+        Tuple tuple_2 = TupleFactory.getInstance().newTuple("item_2");
         ConstantExpression lt = GenPhyOp.exprConst();
         lt.setValue(map_1);
         ConstantExpression rt = GenPhyOp.exprConst();
@@ -423,15 +386,14 @@ public class TestEqualTo extends junit.f
         assertEquals(POStatus.STATUS_OK, r.returnStatus);
         assertFalse((Boolean)r.result);
     }
-    
-	@Test
-	public void testDataByteArrayAndNullValues() throws Exception {
-	    checkNullValues(   DataType.BYTEARRAY,  new DataByteArray("b") );
-		    
-	}
-    
-	public <U> void checkNullValues( byte operandType, U value ) throws Exception {
-		
+
+    @Test
+    public void testDataByteArrayAndNullValues() throws Exception {
+        checkNullValues(DataType.BYTEARRAY, new DataByteArray("b"));
+    }
+
+    public <U> void checkNullValues(byte operandType, U value) throws Exception {
+
         ConstantExpression lt = GenPhyOp.exprConst();
         ConstantExpression rt = GenPhyOp.exprConst();
         EqualToExpr g = GenPhyOp.compEqualToExpr();
@@ -439,37 +401,34 @@ public class TestEqualTo extends junit.f
         // test with null in lhs
         g.setOperandType(operandType);
         lt.setValue(null);
-        rt.setValue( value );
+        rt.setValue(value);
         g.setLhs(lt);
         g.setRhs(rt);
-       
+
         Result r = g.getNext(new Boolean(true));
         assertEquals(POStatus.STATUS_NULL, r.returnStatus);
-        assertEquals(null, (Boolean)r.result);
-        
+        assertNull(r.result);
+
         // test with null in rhs
         g.setOperandType(operandType);
-        lt.setValue( value );
+        lt.setValue(value);
         rt.setValue(null);
         g.setLhs(lt);
         g.setRhs(rt);
-       
+
         r = g.getNext(new Boolean(true));
         assertEquals(POStatus.STATUS_NULL, r.returnStatus);
-        assertEquals(null, (Boolean)r.result);
-   
-        
+        assertNull(r.result);
+
         // test with null in lhs and rhs
         g.setOperandType(operandType);
         lt.setValue(null);
         rt.setValue(null);
         g.setLhs(lt);
         g.setRhs(rt);
-       
+
         r = g.getNext(new Boolean(true));
         assertEquals(POStatus.STATUS_NULL, r.returnStatus);
-        assertEquals(null, (Boolean)r.result);
- 
-
+        assertNull(r.result);
     }
 }