You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by da...@apache.org on 2015/08/05 19:41:44 UTC

svn commit: r1694271 - in /pig/trunk: CHANGES.txt src/org/apache/pig/builtin/TOMAP.java test/org/apache/pig/builtin/TestTOMAP.java

Author: daijy
Date: Wed Aug  5 17:41:44 2015
New Revision: 1694271

URL: http://svn.apache.org/r1694271
Log:
PIG-4638: Allow TOMAP to accept dynamically sized input

Added:
    pig/trunk/test/org/apache/pig/builtin/TestTOMAP.java
Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/builtin/TOMAP.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1694271&r1=1694270&r2=1694271&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Wed Aug  5 17:41:44 2015
@@ -30,10 +30,12 @@ PIG-4365: TOP udf should implement Accum
 
 PIG-4570: Allow AvroStorage to use a class for the schema (pmazak via daijy)
 
-BUG FIXES
-
 PIG-4405: Adding 'map[]' support to mock/Storage (nielsbasjes via daijy)
 
+PIG-4638: Allow TOMAP to accept dynamically sized input (nielsbasjes via daijy)
+
+BUG FIXES
+
 PIG-4636: Occurred spelled incorrectly in error message for Launcher and POMergeCogroup (stevenmz via daijy)
 
 PIG-4624: Error on ORC empty file without schema (daijy)

Modified: pig/trunk/src/org/apache/pig/builtin/TOMAP.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/builtin/TOMAP.java?rev=1694271&r1=1694270&r2=1694271&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/builtin/TOMAP.java (original)
+++ pig/trunk/src/org/apache/pig/builtin/TOMAP.java Wed Aug  5 17:41:44 2015
@@ -18,10 +18,12 @@
 package org.apache.pig.builtin;
 
 import java.io.IOException;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.HashMap;
 
 import org.apache.pig.EvalFunc;
+import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
@@ -30,27 +32,53 @@ import org.apache.pig.impl.logicalLayer.
  * This class makes a map out of the parameters passed to it
  * T = foreach U generate TOMAP($0, $1, $2, $3);
  * It generates a map $0->1, $2->$3
+ *
+ * This UDF also accepts a bag with 'pair' tuples (i.e. tuples with a 'key' and a 'value').
+ *
  */
 public class TOMAP extends EvalFunc<Map> {
 
     @Override
     public Map exec(Tuple input) throws IOException {
-	if (input == null || input.size() < 2)
-		return null;
+        if (input == null || input.size() == 0) {
+            return null;
+        }
+
+        Map<String, Object> output = new HashMap<String, Object>();
+
         try {
-	    Map<String, Object> output = new HashMap<String, Object>();
+            // Is this a single bag with all the values?
+            if (input.size() == 1) {
+                if (input.get(0) instanceof DataBag) {
+                    DataBag bagOfPairs = (DataBag)input.get(0);
+                    if (bagOfPairs.size() == 0) {
+                        return output;
+                    }
+
+                    for (Tuple tuple: bagOfPairs) {
+                        if (tuple.size() != 2) {
+                            throw new RuntimeException("All input tuples in the bag MUST have exactly 2 fields");
+                        }
+                        String key = (String)tuple.get(0);
+                        Object val = tuple.get(1);
+                        output.put(key, val);
+                    }
+                    return output;
+                } else {
+                    return null; // If only 1 value then it must be a bag
+                }
+            }
 
             for (int i = 0; i < input.size(); i=i+2) {
-		String key = (String)input.get(i);
-		Object val = input.get(i+1);	
-		output.put(key, val);
+                String key = (String)input.get(i);
+                Object val = input.get(i+1);
+                output.put(key, val);
             }
-
-	    return output;
-	} catch (ClassCastException e){
-		throw new RuntimeException("Map key must be a String");
-	} catch (ArrayIndexOutOfBoundsException e){
-		throw new RuntimeException("Function input must have even number of parameters");
+            return output;
+        } catch (ClassCastException e){
+            throw new RuntimeException("Map key must be a String");
+        } catch (ArrayIndexOutOfBoundsException e){
+            throw new RuntimeException("Function input must have even number of parameters");
         } catch (Exception e) {
             throw new RuntimeException("Error while creating a map", e);
         }

Added: pig/trunk/test/org/apache/pig/builtin/TestTOMAP.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/builtin/TestTOMAP.java?rev=1694271&view=auto
==============================================================================
--- pig/trunk/test/org/apache/pig/builtin/TestTOMAP.java (added)
+++ pig/trunk/test/org/apache/pig/builtin/TestTOMAP.java Wed Aug  5 17:41:44 2015
@@ -0,0 +1,112 @@
+/*
+ * 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.builtin;
+
+import org.apache.pig.PigServer;
+import org.apache.pig.builtin.mock.Storage;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.test.Util;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.apache.pig.builtin.mock.Storage.map;
+import static org.apache.pig.builtin.mock.Storage.tuple;
+import static org.apache.pig.builtin.mock.Storage.resetData;
+
+public class TestTOMAP {
+
+    static PigServer pigServer;
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        pigServer = new PigServer(Util.getLocalTestMode());
+    }
+
+    @Test
+    public void testTOMAP_Tuple() throws Exception {
+        Storage.Data data = resetData(pigServer);
+
+        data.set("foo",
+                tuple("a", "b", "c", "d")
+        );
+
+        pigServer.registerQuery("A = LOAD 'foo' USING mock.Storage();");
+        pigServer.registerQuery("B = FOREACH A GENERATE TOMAP($0, $1, $2, $3);");
+        pigServer.registerQuery("STORE B INTO 'bar' USING mock.Storage();");
+
+        List<Tuple> out = data.get("bar");
+        assertEquals(tuple(map("c", "d", "a", "b")), out.get(0));
+    }
+
+    @Test
+    public void testTOMAP_Bad_Tuple() throws Exception {
+        Storage.Data data = resetData(pigServer);
+
+        data.set("foo",
+                tuple("a", "b", "c")
+        );
+
+        pigServer.registerQuery("A = LOAD 'foo' USING mock.Storage();");
+        pigServer.registerQuery("B = FOREACH A GENERATE TOMAP($0, $1, $2);");
+        pigServer.registerQuery("STORE B INTO 'bar' USING mock.Storage();");
+
+        List<Tuple> out = data.get("bar");
+        assertEquals(0, out.size()); // Error should occur --> no results
+    }
+
+    @Test
+    public void testTOMAP_BagOfPairs() throws Exception {
+        Storage.Data data = resetData(pigServer);
+
+        data.set("foo",
+                tuple("a", "b"),
+                tuple("c", "d")
+        );
+
+        pigServer.registerQuery("A = LOAD 'foo' USING mock.Storage();");
+        pigServer.registerQuery("B = GROUP A ALL;");
+        pigServer.registerQuery("C = FOREACH B GENERATE TOMAP(A);");
+        pigServer.registerQuery("STORE C INTO 'bar' USING mock.Storage();");
+
+        List<Tuple> out = data.get("bar");
+        assertEquals(tuple(map("a", "b", "c", "d")), out.get(0));
+    }
+
+    @Test
+    public void testTOMAP_Bad_BagOfPairs() throws Exception {
+        Storage.Data data = resetData(pigServer);
+
+        data.set("foo",
+                tuple("a", "b"),
+                tuple("c")
+        );
+
+        pigServer.registerQuery("A = LOAD 'foo' USING mock.Storage();");
+        pigServer.registerQuery("B = GROUP A ALL;");
+        pigServer.registerQuery("C = FOREACH B GENERATE TOMAP(A);");
+        pigServer.registerQuery("STORE C INTO 'bar' USING mock.Storage();");
+
+        List<Tuple> out = data.get("bar");
+        assertEquals(0, out.size()); // Error should occur --> no results
+    }
+
+}