You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by ju...@apache.org on 2012/02/16 19:17:51 UTC

svn commit: r1245112 - in /pig/trunk: CHANGES.txt src/org/apache/pig/scripting/jython/JythonFunction.java test/org/apache/pig/test/TestScriptingLanguagePython.java

Author: julien
Date: Thu Feb 16 18:17:50 2012
New Revision: 1245112

URL: http://svn.apache.org/viewvc?rev=1245112&view=rev
Log:
PIG-2322: varargs functions do not get passed the arguments in Python embedding

Added:
    pig/trunk/test/org/apache/pig/test/TestScriptingLanguagePython.java
Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/scripting/jython/JythonFunction.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1245112&r1=1245111&r2=1245112&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Thu Feb 16 18:17:50 2012
@@ -98,6 +98,8 @@ OPTIMIZATIONS
 
 BUG FIXES
 
+PIG-2322: varargs functions do not get passed the arguments in Python embedding (julien)
+
 PIG-2491: Pig docs still mention hadoop-site.xml (daijy)
 
 PIG-2504: Incorrect sample provided for REGEX_EXTRACT (prkommireddi via daijy)

Modified: pig/trunk/src/org/apache/pig/scripting/jython/JythonFunction.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/scripting/jython/JythonFunction.java?rev=1245112&r1=1245111&r2=1245112&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/scripting/jython/JythonFunction.java (original)
+++ pig/trunk/src/org/apache/pig/scripting/jython/JythonFunction.java Thu Feb 16 18:17:50 2012
@@ -32,6 +32,7 @@ import org.python.core.PyBaseCode;
 import org.python.core.PyException;
 import org.python.core.PyFunction;
 import org.python.core.PyObject;
+import org.python.core.PyTableCode;
 
 /**
  * Python implementation of a Pig UDF Performs mappings between Python & Pig
@@ -103,7 +104,7 @@ public class JythonFunction extends Eval
     @Override
     public Object exec(Tuple tuple) throws IOException {
         try {
-            if (tuple == null || num_parameters == 0) {
+            if (tuple == null || (num_parameters == 0 && !((PyTableCode)function.func_code).varargs)) {
                 // ignore input tuple
                 PyObject out = function.__call__();
                 return JythonUtils.pythonToPig(out);

Added: pig/trunk/test/org/apache/pig/test/TestScriptingLanguagePython.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestScriptingLanguagePython.java?rev=1245112&view=auto
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestScriptingLanguagePython.java (added)
+++ pig/trunk/test/org/apache/pig/test/TestScriptingLanguagePython.java Thu Feb 16 18:17:50 2012
@@ -0,0 +1,83 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.pig.ExecType;
+import org.apache.pig.PigServer;
+import org.apache.pig.scripting.ScriptEngine;
+import org.apache.pig.tools.pigstats.PigStats;
+import org.junit.Test;
+
+public class TestScriptingLanguagePython {
+
+  @Test
+  public void varargTest() throws Exception {
+    System.setProperty("python.cachedir", "/Users/julien/tmp/python");
+    PigServer pigServer = new PigServer(ExecType.LOCAL);
+    String[] script = {
+        "#!/usr/bin/python",
+
+        "from org.apache.pig.scripting import *",
+
+        "@outputSchema(\"s:chararray\")",
+        "def firstNonempty(*args):",
+        "\tfor v in args:",
+        "\t\tif len(v) != 0:",
+        "\t\t\treturn v",
+        "\treturn ''",
+
+        "if __name__ == \"__main__\":",
+        "\tPig.compile(\"\"\"",
+        "data = load 'simple_table' AS (string1:chararray, string2:chararray);",
+        "data = foreach data generate firstNonempty(string1, string2) as id, string1, string2;",
+        "store data into 'simple_out';",
+        "\"\"\").bind().runSingle()"
+    };
+    String[] input = {
+        "1\t3",
+        "2\t4",
+        "3\t5"
+    };
+
+    Util.deleteFile(pigServer.getPigContext(), "simple_table");
+    Util.deleteFile(pigServer.getPigContext(), "simple_out");
+    Util.createInputFile(pigServer.getPigContext(), "simple_table", input);
+    Util.createLocalInputFile( "testScript.py", script);
+
+    ScriptEngine scriptEngine = ScriptEngine.getInstance("jython");
+    Map<String, List<PigStats>> statsMap = scriptEngine.run(pigServer.getPigContext(), "testScript.py");
+    assertEquals(1, statsMap.size());
+    Iterator<List<PigStats>> it = statsMap.values().iterator();
+    PigStats stats = it.next().get(0);
+    assertTrue(stats.isSuccessful());
+
+    String[] output = Util.readOutput(pigServer.getPigContext(), "simple_out");
+    assertEquals(3, output.length);
+    assertEquals(output[0], "1\t1\t3");
+    assertEquals(output[1], "2\t2\t4");
+    assertEquals(output[2], "3\t3\t5");
+  }
+
+}