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 2013/03/04 20:31:13 UTC

svn commit: r1452460 - in /pig/trunk: ./ src/org/apache/pig/newplan/logical/relational/ test/org/apache/pig/test/

Author: cheolsoo
Date: Mon Mar  4 19:31:13 2013
New Revision: 1452460

URL: http://svn.apache.org/r1452460
Log:
PIG-3144: Erroneous map entry alias resolution leading to "Duplicate schema alias" errors (jcoveney via cheolsoo)

Added:
    pig/trunk/test/org/apache/pig/test/TestMapProjectionDuplicate.java
Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/newplan/logical/relational/LOGenerate.java
    pig/trunk/src/org/apache/pig/newplan/logical/relational/LOJoin.java
    pig/trunk/src/org/apache/pig/newplan/logical/relational/LogicalRelationalOperator.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1452460&r1=1452459&r2=1452460&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Mon Mar  4 19:31:13 2013
@@ -223,6 +223,8 @@ OPTIMIZATIONS
 
 BUG FIXES
 
+PIG-3144: Erroneous map entry alias resolution leading to "Duplicate schema alias" errors (jcoveney via cheolsoo)
+
 PIGG-3212: Race Conditions in POSort and (Internal)SortedBag during Proactive Spill (kadeng via dvryaboy)
 
 PIG-3206: HBaseStorage does not work with Oozie pig action and secure HBase (rohini)

Modified: pig/trunk/src/org/apache/pig/newplan/logical/relational/LOGenerate.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/newplan/logical/relational/LOGenerate.java?rev=1452460&r1=1452459&r2=1452460&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/newplan/logical/relational/LOGenerate.java (original)
+++ pig/trunk/src/org/apache/pig/newplan/logical/relational/LOGenerate.java Mon Mar  4 19:31:13 2013
@@ -62,8 +62,9 @@ public class LOGenerate extends LogicalR
         
         if (uidOnlySchemas == null) {
             uidOnlySchemas = new ArrayList<LogicalSchema>();
-            for (int i=0;i<outputPlans.size();i++)
+            for (int i=0;i<outputPlans.size();i++) {
                 uidOnlySchemas.add(null);
+            }
         }
         
         schema = new LogicalSchema();
@@ -191,6 +192,11 @@ public class LOGenerate extends LogicalR
             schema = null;
             outputPlanSchemas = null;
         }
+        
+        if (schema != null) {
+            LogicalRelationalOperator.fixDuplicateUids(schema.getFields());
+        }
+        
         return schema;
     }
 

Modified: pig/trunk/src/org/apache/pig/newplan/logical/relational/LOJoin.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/newplan/logical/relational/LOJoin.java?rev=1452460&r1=1452459&r2=1452460&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/newplan/logical/relational/LOJoin.java (original)
+++ pig/trunk/src/org/apache/pig/newplan/logical/relational/LOJoin.java Mon Mar  4 19:31:13 2013
@@ -21,17 +21,13 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.util.MultiMap;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.PlanVisitor;
-import org.apache.pig.newplan.logical.expression.LogicalExpression;
 import org.apache.pig.newplan.logical.expression.LogicalExpressionPlan;
-import org.apache.pig.newplan.logical.relational.LogicalSchema.LogicalFieldSchema;
 
-import com.google.common.collect.Sets;
 
 
 public class LOJoin extends LogicalRelationalOperator {
@@ -160,7 +156,7 @@ public class LOJoin extends LogicalRelat
             }
         }
 
-        fixDuplicateUids(fss);
+        LogicalRelationalOperator.fixDuplicateUids(fss);
 
         schema = new LogicalSchema();
         for(LogicalSchema.LogicalFieldSchema fieldSchema: fss) {
@@ -170,31 +166,6 @@ public class LOJoin extends LogicalRelat
         return schema;
     }
 
-    /**
-     * In the case of a join it is possible for multiple columns to have been derived from the same
-     * column and thus have duplicate UID's. This detects that case and resets the uid.
-     * See PIG-3022 and PIG-3093 for more information.
-     * @param fss a list of LogicalFieldSchemas to check the uids of
-     */
-    private void fixDuplicateUids(List<LogicalFieldSchema> fss) {
-        Set<Long> uids = Sets.newHashSet();
-        for (LogicalFieldSchema lfs : fss) {
-            addFieldSchemaUidsToSet(uids, lfs);
-        }
-    }
-
-    private void addFieldSchemaUidsToSet(Set<Long> uids, LogicalFieldSchema lfs) {
-        while (!uids.add(lfs.uid)) {
-            lfs.uid = LogicalExpression.getNextUid();
-        }
-        LogicalSchema ls = lfs.schema;
-        if (ls != null) {
-            for (LogicalFieldSchema lfs2 : ls.getFields()) {
-                addFieldSchemaUidsToSet(uids, lfs2);
-            }
-        }
-    }
-
     @Override
     public void accept(PlanVisitor v) throws FrontendException {
         if (!(v instanceof LogicalRelationalNodesVisitor)) {

Modified: pig/trunk/src/org/apache/pig/newplan/logical/relational/LogicalRelationalOperator.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/newplan/logical/relational/LogicalRelationalOperator.java?rev=1452460&r1=1452459&r2=1452460&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/newplan/logical/relational/LogicalRelationalOperator.java (original)
+++ pig/trunk/src/org/apache/pig/newplan/logical/relational/LogicalRelationalOperator.java Mon Mar  4 19:31:13 2013
@@ -19,11 +19,17 @@
 package org.apache.pig.newplan.logical.relational;
 
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.newplan.Operator;
 import org.apache.pig.newplan.OperatorPlan;
+import org.apache.pig.newplan.logical.expression.LogicalExpression;
+import org.apache.pig.newplan.logical.relational.LogicalSchema.LogicalFieldSchema;
+
+import com.google.common.collect.Sets;
 
 /**
  * Logical representation of relational operators.  Relational operators have
@@ -205,4 +211,30 @@ abstract public class LogicalRelationalO
     public boolean isPinnedOption(Integer opt) {
         return mPinnedOptions.contains(opt);
     }
+
+    private static void addFieldSchemaUidsToSet(Set<Long> uids, LogicalFieldSchema lfs) {
+        while (!uids.add(lfs.uid)) {
+            lfs.uid = LogicalExpression.getNextUid();
+        }
+        LogicalSchema ls = lfs.schema;
+        if (ls != null) {
+            for (LogicalFieldSchema lfs2 : ls.getFields()) {
+                addFieldSchemaUidsToSet(uids, lfs2);
+            }
+        }
+    }
+
+    /**
+     * In the case of an operation which manipualtes columns (such as a foreach or a join)
+     * it is possible for multiple columns to have been derived from the same
+     * column and thus have duplicate UID's. This detects that case and resets the uid.
+     * See PIG-3020 and PIG-3093 for more information.
+     * @param fss a list of LogicalFieldSchemas to check the uids of
+     */
+    public static void fixDuplicateUids(List<LogicalFieldSchema> fss) {
+        Set<Long> uids = Sets.newHashSet();
+        for (LogicalFieldSchema lfs : fss) {
+            LogicalRelationalOperator.addFieldSchemaUidsToSet(uids, lfs);
+        }
+    }
 }

Added: pig/trunk/test/org/apache/pig/test/TestMapProjectionDuplicate.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestMapProjectionDuplicate.java?rev=1452460&view=auto
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestMapProjectionDuplicate.java (added)
+++ pig/trunk/test/org/apache/pig/test/TestMapProjectionDuplicate.java Mon Mar  4 19:31:13 2013
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.pig.test;
+
+import static org.apache.pig.builtin.mock.Storage.resetData;
+import static org.apache.pig.builtin.mock.Storage.tuple;
+import static org.junit.Assert.assertEquals;
+
+import java.util.Map;
+
+import org.apache.pig.ExecType;
+import org.apache.pig.PigServer;
+import org.apache.pig.builtin.mock.Storage.Data;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.impl.util.Utils;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableMap;
+
+public class TestMapProjectionDuplicate {
+    private static PigServer pigServer;
+    
+    @Before
+    public void setUp() throws Exception {
+        pigServer = new PigServer(ExecType.LOCAL);
+    }
+    
+    @Test
+    public void testDuplicate() throws Exception {
+        Data data = resetData(pigServer);
+
+        Map<String,String> m1 = ImmutableMap.of("name", "jon", "age", "26");
+        
+        data.set("foo", Utils.getSchemaFromString("a:[chararray]"), tuple(m1));
+        
+        pigServer.registerQuery("a = load 'foo' using mock.Storage() as (a:map[chararray]);");
+        pigServer.registerQuery("b = foreach @ generate a#'name' as name, a#'age' as age;");
+        pigServer.registerQuery("c = foreach @ generate *;");
+        Schema s = pigServer.dumpSchema("c");
+        assertEquals("name", s.getField(0).alias);
+        assertEquals("age", s.getField(1).alias);
+    }
+}