You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by th...@apache.org on 2011/05/16 20:56:47 UTC

svn commit: r1103841 - in /pig/branches/branch-0.9: ./ src/org/apache/pig/newplan/logical/relational/ test/org/apache/pig/parser/ test/org/apache/pig/test/

Author: thejas
Date: Mon May 16 18:56:47 2011
New Revision: 1103841

URL: http://svn.apache.org/viewvc?rev=1103841&view=rev
Log:
PIG-2071: casting numeric type to chararray during schema merge for union
 is inconsistent with other schema merge cases 

Modified:
    pig/branches/branch-0.9/CHANGES.txt
    pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LogicalSchema.java
    pig/branches/branch-0.9/test/org/apache/pig/parser/TestUnionOnSchemaSetter.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestSchema.java
    pig/branches/branch-0.9/test/org/apache/pig/test/TestUnionOnSchema.java

Modified: pig/branches/branch-0.9/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/CHANGES.txt?rev=1103841&r1=1103840&r2=1103841&view=diff
==============================================================================
--- pig/branches/branch-0.9/CHANGES.txt (original)
+++ pig/branches/branch-0.9/CHANGES.txt Mon May 16 18:56:47 2011
@@ -182,6 +182,9 @@ PIG-1696: Performance: Use System.arrayc
 
 BUG FIXES
 
+PIG-2071: casting numeric type to chararray during schema merge for union
+ is inconsistent with other schema merge cases (thejas)
+
 PIG-2070: "Unknown" appears in error message for an error case (thejas)
 
 PIG-2069: LoadFunc jar does not ship to backend in MultiQuery case (rding)

Modified: pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LogicalSchema.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LogicalSchema.java?rev=1103841&r1=1103840&r2=1103841&view=diff
==============================================================================
--- pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LogicalSchema.java (original)
+++ pig/branches/branch-0.9/src/org/apache/pig/newplan/logical/relational/LogicalSchema.java Mon May 16 18:56:47 2011
@@ -385,14 +385,8 @@ public class LogicalSchema {
                     // Take the more specific type
                     mergedType = DataType.mergeType(fs1.type, fs2.type);
                     if (mergedType == DataType.ERROR) {
-                        // Another possibility is one side is numeric, the other side is string, in this case, we take string
-                        if (DataType.isNumberType(fs1.type) && fs2.type==DataType.CHARARRAY ||
-                                DataType.isNumberType(fs2.type) && fs1.type==DataType.CHARARRAY)
-                            mergedType = DataType.CHARARRAY;
-                        else {
-                            // True incompatible, set to bytearray
-                            mergedType = DataType.BYTEARRAY;
-                        }
+                        // True incompatible, set to bytearray
+                        mergedType = DataType.BYTEARRAY;
                     }
                 }
             }

Modified: pig/branches/branch-0.9/test/org/apache/pig/parser/TestUnionOnSchemaSetter.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/test/org/apache/pig/parser/TestUnionOnSchemaSetter.java?rev=1103841&r1=1103840&r2=1103841&view=diff
==============================================================================
--- pig/branches/branch-0.9/test/org/apache/pig/parser/TestUnionOnSchemaSetter.java (original)
+++ pig/branches/branch-0.9/test/org/apache/pig/parser/TestUnionOnSchemaSetter.java Mon May 16 18:56:47 2011
@@ -76,7 +76,7 @@ public class TestUnionOnSchemaSetter {
 
     @Test
     public void testMergeCompatibleSchema() throws FrontendException {
-        String query = "A = load 'x' as ( u:int, v:long, w:chararray); " + 
+        String query = "A = load 'x' as ( u:int, v:long, w:int); " + 
                        "B = load 'y' as ( u:int, v:long, w:long); " +
                        "C = union onschema A, B; " +
                        "D = store C into 'output';";

Modified: pig/branches/branch-0.9/test/org/apache/pig/test/TestSchema.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/test/org/apache/pig/test/TestSchema.java?rev=1103841&r1=1103840&r2=1103841&view=diff
==============================================================================
--- pig/branches/branch-0.9/test/org/apache/pig/test/TestSchema.java (original)
+++ pig/branches/branch-0.9/test/org/apache/pig/test/TestSchema.java Mon May 16 18:56:47 2011
@@ -730,7 +730,7 @@ public class TestSchema {
         
         LogicalSchema mergedSchema = LogicalSchema.merge(a, b, LogicalSchema.MergeMode.Union);
         LogicalSchema expected = org.apache.pig.newplan.logical.Util.translateSchema(Utils.getSchemaFromString(
-            "a1:(a11:chararray, a12:float), b1:(), c1:chararray"));
+            "a1:(a11:chararray, a12:float), b1:(), c1:bytearray"));
         expected.getField(1).schema = new LogicalSchema();
         Assert.assertTrue(LogicalSchema.equals(mergedSchema, expected, false, false));
         

Modified: pig/branches/branch-0.9/test/org/apache/pig/test/TestUnionOnSchema.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.9/test/org/apache/pig/test/TestUnionOnSchema.java?rev=1103841&r1=1103840&r2=1103841&view=diff
==============================================================================
--- pig/branches/branch-0.9/test/org/apache/pig/test/TestUnionOnSchema.java (original)
+++ pig/branches/branch-0.9/test/org/apache/pig/test/TestUnionOnSchema.java Mon May 16 18:56:47 2011
@@ -21,16 +21,12 @@ import static org.junit.Assert.assertEqu
 import static org.junit.Assert.fail;
 
 import java.io.File;
-import java.io.FileWriter;
 import java.io.IOException;
-import java.io.PrintWriter;
 import java.util.Iterator;
 import java.util.List;
 
 import junit.framework.Assert;
 
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.pig.EvalFunc;
 import org.apache.pig.ExecType;
 import org.apache.pig.PigException;
@@ -52,8 +48,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestUnionOnSchema  {
-    static MiniCluster cluster ;
-    private static final String EMPTY_DIR = "emptydir";
     private static final String INP_FILE_2NUMS = "TestUnionOnSchemaInput1";
     private static final String INP_FILE_2NUM_1CHAR_1BAG = "TestUnionOnSchemaInput2";
     private static final String INP_FILE_EMPTY= "TestUnionOnSchemaInput3";
@@ -70,39 +64,26 @@ public class TestUnionOnSchema  {
     
     @BeforeClass
     public static void oneTimeSetup() throws IOException, Exception {
-        cluster = MiniCluster.buildCluster();
-        FileSystem fs = cluster.getFileSystem();
-        if (!fs.mkdirs(new Path(EMPTY_DIR))) {
-            throw new Exception("failed to create empty dir");
-        }
         // first input file
-        PrintWriter w = new PrintWriter(new FileWriter(INP_FILE_2NUMS));
-        w.println("1\t2");
-        w.println("5\t3");
-        w.close();
-        Util.copyFromLocalToCluster(cluster, INP_FILE_2NUMS, INP_FILE_2NUMS);
+        String[] input1 = {"1\t2","5\t3"};
+        Util.createLocalInputFile(INP_FILE_2NUMS, input1);
         
         // 2nd input file
-        w = new PrintWriter(new FileWriter(INP_FILE_2NUM_1CHAR_1BAG));
-        w.println("1\tabc\t2\t{(1,a),(1,b)}\t(1,c)");
-        w.println("5\tdef\t3\t{(2,a),(2,b)}\t(2,c)");
-        w.close();
-        Util.copyFromLocalToCluster(cluster, INP_FILE_2NUM_1CHAR_1BAG, INP_FILE_2NUM_1CHAR_1BAG);
+        String[] input2 = {
+                "1\tabc\t2\t{(1,a),(1,b)}\t(1,c)",
+                "5\tdef\t3\t{(2,a),(2,b)}\t(2,c)"
+        };
+        Util.createLocalInputFile(INP_FILE_2NUM_1CHAR_1BAG, input2);
 
         //3rd input - empty file
-        w = new PrintWriter(new FileWriter(INP_FILE_EMPTY));
-        w.close();
-        Util.copyFromLocalToCluster(cluster, INP_FILE_EMPTY, INP_FILE_EMPTY);
-        
+        Util.createLocalInputFile(INP_FILE_EMPTY, new String[0]);
     }
     
     @AfterClass
     public static void oneTimeTearDown() throws Exception {
-
         new File(INP_FILE_2NUMS).delete();
         new File(INP_FILE_2NUM_1CHAR_1BAG).delete();
         new File(INP_FILE_EMPTY).delete();
-        cluster.shutDown();
     }
  
 
@@ -441,8 +422,7 @@ public class TestUnionOnSchema  {
      */
     @Test
     public void testUnionOnSchemaAdditionalColumn() throws IOException, ParserException {
-        PigServer pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
-        //PigServer pig = new PigServer(ExecType.LOCAL);
+        PigServer pig = new PigServer(ExecType.LOCAL);
         String query =
             "  l1 = load '" + INP_FILE_2NUMS + "' as (i : int, j : int);"
             + "l2 = load '" + INP_FILE_2NUM_1CHAR_1BAG + "' as " 
@@ -613,7 +593,7 @@ public class TestUnionOnSchema  {
     
 
     /**
-     * negative test - test error on incompatible types in schema
+     * test union with incompatible types in schema
      * @throws IOException
      * @throws ParserException
      */
@@ -624,7 +604,7 @@ public class TestUnionOnSchema  {
             + "l2 = load '" + INP_FILE_2NUMS + "' as (x : long, y : float);"
             + "u = union onschema l1, l2;";
 
-        checkSchemaEquals(query, "x : long, y : chararray");
+        checkSchemaEquals(query, "x : long, y : bytearray");
 
 
         
@@ -724,8 +704,7 @@ public class TestUnionOnSchema  {
      */
     @Test
     public void testUnionOnSchemaUdfTypeEvolution() throws IOException, ParserException {
-        PigServer pig = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
-        //PigServer pig = new PigServer(ExecType.LOCAL);
+        PigServer pig = new PigServer(ExecType.LOCAL);
         String query_prefix =
             "  l1 = load '" + INP_FILE_2NUM_1CHAR_1BAG + "' as " 
             + "  (i : int, c : chararray, j : int "