You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by bi...@apache.org on 2012/08/14 01:48:46 UTC

svn commit: r1372669 - in /pig/trunk: CHANGES.txt src/org/apache/pig/parser/QueryParserDriver.java test/org/apache/pig/test/TestPigServer.java test/org/apache/pig/test/TestPigServerWithMacros.java

Author: billgraham
Date: Mon Aug 13 23:48:46 2012
New Revision: 1372669

URL: http://svn.apache.org/viewvc?rev=1372669&view=rev
Log:
PIG-2866: PigServer fails with macros without a script file (billgraham)

Added:
    pig/trunk/test/org/apache/pig/test/TestPigServerWithMacros.java
Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java
    pig/trunk/test/org/apache/pig/test/TestPigServer.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1372669&r1=1372668&r2=1372669&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Mon Aug 13 23:48:46 2012
@@ -232,6 +232,8 @@ OPTIMIZATIONS
 
 BUG FIXES
 
+PIG-2866: PigServer fails with macros without a script file (billgraham)
+
 PIG-2860: [piggybank] TestAvroStorageUtils.testGetConcretePathFromGlob fails on some version of hadoop (cheolsoo via jcoveney)
 
 PIG-2861: PlanHelper imports org.python.google.common.collect.Lists instead of org.google.common.collect.Lists (jcoveney)

Modified: pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java?rev=1372669&r1=1372668&r2=1372669&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java (original)
+++ pig/trunk/src/org/apache/pig/parser/QueryParserDriver.java Mon Aug 13 23:48:46 2012
@@ -407,10 +407,15 @@ public class QueryParserDriver {
         String body = bodyNode.getChild(0).getText();
 
         body = body.substring(1, body.length() - 1);
-        
-        FetchFileRet localFileRet = getMacroFile(fname);
 
-        PigMacro pm = new PigMacro(mn, localFileRet.file.getAbsolutePath(), params, returns, body, seen);
+        // sometimes the script has no filename, like when a string is passed to PigServer for
+        // example. See PIG-2866.
+        if (fname != null) {
+            FetchFileRet localFileRet = getMacroFile(fname);
+            fname = localFileRet.file.getAbsolutePath();
+        }
+
+        PigMacro pm = new PigMacro(mn, fname, params, returns, body, seen);
         
         try {
             pm.validate();

Modified: pig/trunk/test/org/apache/pig/test/TestPigServer.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPigServer.java?rev=1372669&r1=1372668&r2=1372669&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigServer.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPigServer.java Mon Aug 13 23:48:46 2012
@@ -45,6 +45,7 @@ import java.util.Properties;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.pig.builtin.mock.Storage;
 import org.apache.pig.impl.PigContext;
 
 
@@ -54,7 +55,6 @@ import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.logicalLayer.FrontendException;
 import org.apache.pig.impl.logicalLayer.schema.Schema;
-import org.apache.pig.impl.util.LogUtils;
 import org.apache.pig.impl.util.PropertiesUtil;
 import org.apache.pig.impl.util.Utils;
 import org.junit.After;
@@ -398,57 +398,6 @@ public class TestPigServer {
         // clean-up
         Assert.assertTrue(fs.delete(new Path(jarLocation), true));
     }
-    
-    @Test
-    public void testRegisterRemoteMacro() throws Throwable {
-        String macroName = "util.pig";
-        File macroFile = File.createTempFile("tmp", "");
-        PrintWriter pw = new PrintWriter(new FileWriter(macroFile));
-        pw.println("DEFINE row_count(X) RETURNS Z { Y = group $X all; $Z = foreach Y generate COUNT($X); };");
-        pw.close();
-        
-        FileSystem fs = cluster.getFileSystem();
-        fs.copyFromLocalFile(new Path(macroFile.getAbsolutePath()), new Path(macroName));
-        
-        // find the absolute path for the directory so that it does not
-        // depend on configuration
-        String absPath = fs.getFileStatus(new Path(macroName)).getPath().toString();
-        
-        Util.createInputFile(cluster, "testRegisterRemoteMacro_input", new String[]{"1", "2"});
-        
-        pig.registerQuery("import '" + absPath + "';");
-        pig.registerQuery("a = load 'testRegisterRemoteMacro_input';");
-        pig.registerQuery("b = row_count(a);");
-        Iterator<Tuple> iter = pig.openIterator("b");
-        
-        Assert.assertTrue(((Long)iter.next().get(0))==2);
-    }
-    
-    @Test
-    public void testRegisterRemoteScript() throws Throwable {
-        String scriptName = "script.py";
-        File scriptFile = File.createTempFile("tmp", "");
-        PrintWriter pw = new PrintWriter(new FileWriter(scriptFile));
-        pw.println("@outputSchema(\"word:chararray\")\ndef helloworld():\n    return 'Hello, World'");
-        pw.close();
-        
-        FileSystem fs = cluster.getFileSystem();
-        fs.copyFromLocalFile(new Path(scriptFile.getAbsolutePath()), new Path(scriptName));
-        
-        // find the absolute path for the directory so that it does not
-        // depend on configuration
-        String absPath = fs.getFileStatus(new Path(scriptName)).getPath().toString();
-        
-        Util.createInputFile(cluster, "testRegisterRemoteScript_input", new String[]{"1", "2"});
-        pig.registerCode(absPath, "jython", "pig");
-        pig.registerQuery("a = load 'testRegisterRemoteScript_input';");
-        pig.registerQuery("b = foreach a generate pig.helloworld($0);");
-        Iterator<Tuple> iter = pig.openIterator("b");
-        
-        Assert.assertTrue(iter.next().get(0).equals("Hello, World"));
-        Assert.assertTrue(iter.next().get(0).equals("Hello, World"));
-        Assert.assertFalse(iter.hasNext());
-    }
 
     @Test
     public void testDescribeLoad() throws Throwable {

Added: pig/trunk/test/org/apache/pig/test/TestPigServerWithMacros.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPigServerWithMacros.java?rev=1372669&view=auto
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigServerWithMacros.java (added)
+++ pig/trunk/test/org/apache/pig/test/TestPigServerWithMacros.java Mon Aug 13 23:48:46 2012
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.pig.ExecType;
+import org.apache.pig.PigServer;
+import org.apache.pig.builtin.mock.Storage;
+import org.apache.pig.data.Tuple;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.PrintWriter;
+import java.util.Iterator;
+
+import static org.apache.pig.builtin.mock.Storage.resetData;
+import static org.apache.pig.builtin.mock.Storage.tuple;
+
+public class TestPigServerWithMacros {
+    private PigServer pig = null;
+
+    @Before
+    public void setUp() throws Exception{
+        pig = new PigServer(ExecType.LOCAL);
+    }
+
+    @After
+    public void tearDown() throws Exception{
+        pig = null;
+    }
+
+    @Test
+    public void testRegisterRemoteMacro() throws Throwable {
+        String macroName = "util.pig";
+        File macroFile = File.createTempFile("tmp", "");
+        PrintWriter pw = new PrintWriter(new FileWriter(macroFile));
+        pw.println("DEFINE row_count(X) RETURNS Z { Y = group $X all; $Z = foreach Y generate COUNT($X); };");
+        pw.close();
+
+        Path macroPath = new Path(macroName);
+        FileSystem fs = macroPath.getFileSystem(new Configuration());
+
+        fs.copyFromLocalFile(new Path(macroFile.getAbsolutePath()), macroPath);
+
+        // find the absolute path for the directory so that it does not
+        // depend on configuration
+        String absPath = fs.getFileStatus(new Path(macroName)).getPath().toString();
+
+        pig = new PigServer(ExecType.LOCAL);
+        Storage.Data data = resetData(pig);
+        data.set("some_path", "(l:chararray)", tuple("first row"), tuple("second row"));
+
+        pig.registerQuery("import '" + absPath + "';");
+        pig.registerQuery("a = load 'some_path' USING mock.Storage();");
+        pig.registerQuery("b = row_count(a);");
+        Iterator<Tuple> iter = pig.openIterator("b");
+
+        Assert.assertEquals(2L, iter.next().get(0));
+    }
+
+    @Test
+    public void testInlineMacro() throws Throwable {
+        Storage.Data data = resetData(pig);
+        data.set("some_path", "(l:chararray)", tuple("first row"), tuple("second row"));
+
+        pig.registerQuery("DEFINE row_count(X) RETURNS Z { Y = group $X all; $Z = foreach Y generate COUNT($X); };");
+        pig.registerQuery("a = load 'some_path' USING mock.Storage();");
+        pig.registerQuery("b = row_count(a);");
+        Iterator<Tuple> iter = pig.openIterator("b");
+
+        Assert.assertEquals(2L, iter.next().get(0));
+    }
+
+    @Test
+    public void testRegisterRemoteScript() throws Throwable {
+        String scriptName = "script.py";
+        File scriptFile = File.createTempFile("tmp", "");
+        PrintWriter pw = new PrintWriter(new FileWriter(scriptFile));
+        pw.println("@outputSchema(\"word:chararray\")\ndef helloworld():\n    return 'Hello, World'");
+        pw.close();
+
+        Path scriptPath = new Path(scriptName);
+        FileSystem fs = scriptPath.getFileSystem(new Configuration());
+        fs.copyFromLocalFile(new Path(scriptFile.getAbsolutePath()), scriptPath);
+
+        // find the absolute path for the directory so that it does not
+        // depend on configuration
+        String absPath = fs.getFileStatus(scriptPath).getPath().toString();
+
+        Storage.Data data = resetData(pig);
+        data.set("some_path", "(l:chararray)", tuple(tuple("first row")), tuple(tuple("second row")));
+
+        pig.registerCode(absPath, "jython", "pig");
+        pig.registerQuery("a = load 'some_path' USING mock.Storage();");
+        pig.registerQuery("b = foreach a generate pig.helloworld($0);");
+        Iterator<Tuple> iter = pig.openIterator("b");
+
+        Assert.assertTrue(iter.next().get(0).equals("Hello, World"));
+        Assert.assertTrue(iter.next().get(0).equals("Hello, World"));
+        Assert.assertFalse(iter.hasNext());
+    }
+}