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 2014/12/11 19:18:54 UTC

svn commit: r1644713 - in /pig/trunk: ./ test/ test/org/apache/pig/test/

Author: daijy
Date: Thu Dec 11 18:18:54 2014
New Revision: 1644713

URL: http://svn.apache.org/r1644713
Log:
PIG-4354: Port local mode tests to Tez - part3

Added:
    pig/trunk/test/org/apache/pig/test/TestPigServerWithMacrosRemote.java
Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/test/org/apache/pig/test/TestAutoLocalMode.java
    pig/trunk/test/org/apache/pig/test/TestEvalPipelineLocal.java
    pig/trunk/test/org/apache/pig/test/TestFetch.java
    pig/trunk/test/org/apache/pig/test/TestPigRunner.java
    pig/trunk/test/org/apache/pig/test/TestPigServerWithMacros.java
    pig/trunk/test/tez-local-tests
    pig/trunk/test/tez-tests

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1644713&r1=1644712&r2=1644713&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Thu Dec 11 18:18:54 2014
@@ -28,6 +28,8 @@ PIG-4333: Split BigData tests into multi
  
 BUG FIXES
 
+PIG-4354: Port local mode tests to Tez - part3 (daijy)
+
 PIG-4338: Fix test failures with JDK8 (rohini)
 
 PIG-4351: TestPigRunner.simpleTest2 fail on trunk (daijy)

Modified: pig/trunk/test/org/apache/pig/test/TestAutoLocalMode.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestAutoLocalMode.java?rev=1644713&r1=1644712&r2=1644713&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestAutoLocalMode.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestAutoLocalMode.java Thu Dec 11 18:18:54 2014
@@ -33,7 +33,6 @@ import org.apache.log4j.FileAppender;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.log4j.SimpleLayout;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigConfiguration;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler;
@@ -94,7 +93,7 @@ public class TestAutoLocalMode {
 
     @Before
     public void setUp() throws Exception{
-        pigServer = new PigServer(ExecType.MAPREDUCE, cluster.getProperties());
+        pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
         pigServer.getPigContext().getExecutionEngine().setProperty(PigConfiguration.PIG_OPT_FETCH, "false");
         pigServer.getPigContext().getExecutionEngine().setProperty(PigConfiguration.PIG_AUTO_LOCAL_ENABLED, String.valueOf("true"));
         pigServer.getPigContext().getExecutionEngine().setProperty(PigConfiguration.PIG_AUTO_LOCAL_INPUT_MAXBYTES, "200");

Modified: pig/trunk/test/org/apache/pig/test/TestEvalPipelineLocal.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestEvalPipelineLocal.java?rev=1644713&r1=1644712&r2=1644713&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestEvalPipelineLocal.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestEvalPipelineLocal.java Thu Dec 11 18:18:54 2014
@@ -39,7 +39,6 @@ import junit.framework.Assert;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.pig.ComparisonFunc;
 import org.apache.pig.EvalFunc;
-import org.apache.pig.ExecType;
 import org.apache.pig.FuncSpec;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
@@ -60,6 +59,7 @@ import org.apache.pig.impl.util.Pair;
 import org.apache.pig.impl.util.UDFContext;
 import org.apache.pig.impl.util.Utils;
 import org.apache.pig.test.utils.Identity;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -73,7 +73,7 @@ public class TestEvalPipelineLocal {
     
     @Before
     public void setUp() throws Exception{
-        pigServer = new PigServer(ExecType.LOCAL);
+        pigServer = new PigServer(Util.getLocalTestMode());
     }
     
     static public class MyBagFunction extends EvalFunc<DataBag>{
@@ -1030,6 +1030,8 @@ public class TestEvalPipelineLocal {
     
     @Test
     public void testExplainInDotGraph() throws Exception{
+        Assume.assumeTrue("Skip this test for TEZ since TEZ does not support explain in dot format",
+                !Util.getLocalTestMode().toString().startsWith("TEZ"));
         pigServer.registerQuery("a = load 'voter' using " + PigStorage.class.getName() + "(',') as (name, age, registration, contributions);");
         pigServer.registerQuery("b = filter a by age < 50;");
         pigServer.registerQuery("c = group b by registration;");
@@ -1139,10 +1141,16 @@ public class TestEvalPipelineLocal {
         Schema expectedSchema = Utils.getSchemaFromString(
                     "group: bytearray");
         Assert.assertEquals(expectedSchema, dumpedSchema);
+        TupleFactory tf = TupleFactory.getInstance();
+        List<Tuple> expected = new ArrayList<Tuple>();
+        Tuple t = tf.newTuple(1);
+        t.set(0, new DataByteArray("NYSE".getBytes()));
+        expected.add(t);
+        t = tf.newTuple(1);
+        t.set(0, new DataByteArray("NASDAQ".getBytes()));
+        expected.add(t);
         Iterator<Tuple> iter = pigServer.openIterator("zzz");
-        Assert.assertTrue(iter.next().toString().equals("(NYSE)"));
-        Assert.assertTrue(iter.next().toString().equals("(NASDAQ)"));
-        Assert.assertFalse(iter.hasNext());
+        Util.checkQueryOutputsAfterSort(iter, expected);
     }
     
     // Self cross, see PIG-3292

Modified: pig/trunk/test/org/apache/pig/test/TestFetch.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestFetch.java?rev=1644713&r1=1644712&r2=1644713&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestFetch.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestFetch.java Thu Dec 11 18:18:54 2014
@@ -33,12 +33,11 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.pig.ExecType;
 import org.apache.pig.PigConfiguration;
 import org.apache.pig.PigServer;
+import org.apache.pig.backend.hadoop.executionengine.HExecutionEngine;
 import org.apache.pig.backend.hadoop.executionengine.fetch.FetchLauncher;
 import org.apache.pig.backend.hadoop.executionengine.fetch.FetchOptimizer;
-import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRExecutionEngine;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLimit;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLoad;
@@ -94,7 +93,7 @@ public class TestFetch {
 
     @Before
     public void setUp() throws Exception{
-        pigServer = new PigServer(ExecType.LOCAL, new Properties());
+        pigServer = new PigServer(Util.getLocalTestMode(), new Properties());
         // force direct fetch mode
         pigServer.getPigContext().getProperties().setProperty(PigConfiguration.PIG_OPT_FETCH, "true");
     }
@@ -123,7 +122,7 @@ public class TestFetch {
 
         LogicalPlan lp = ParserTestingUtils.generateLogicalPlan(query);
 
-        PhysicalPlan pp = ((MRExecutionEngine) pigServer.getPigContext().getExecutionEngine())
+        PhysicalPlan pp = ((HExecutionEngine) pigServer.getPigContext().getExecutionEngine())
                 .compile(lp, null);
 
         boolean planFetchable = FetchOptimizer.isPlanFetchable(pigServer.getPigContext(), pp);
@@ -162,7 +161,7 @@ public class TestFetch {
 
         LogicalPlan lp = ParserTestingUtils.generateLogicalPlan(query);
 
-        PhysicalPlan pp = ((MRExecutionEngine) pigServer.getPigContext().getExecutionEngine())
+        PhysicalPlan pp = ((HExecutionEngine) pigServer.getPigContext().getExecutionEngine())
                 .compile(lp, null);
 
         boolean planFetchable = FetchOptimizer.isPlanFetchable(pigServer.getPigContext(), pp);
@@ -228,7 +227,7 @@ public class TestFetch {
             pigServer.setBatchOn();
 
             LogicalPlan lp = TestPigStats.getLogicalPlan(pigServer);
-            PhysicalPlan pp = ((MRExecutionEngine)
+            PhysicalPlan pp = ((HExecutionEngine)
                     pigServer.getPigContext().getExecutionEngine()).compile(lp, null);
             boolean planFetchable = FetchOptimizer.isPlanFetchable(pigServer.getPigContext(), pp);
             assertFalse(planFetchable);
@@ -257,7 +256,7 @@ public class TestFetch {
             pigServer.setBatchOn();
 
             LogicalPlan lp = TestPigStats.getLogicalPlan(pigServer);
-            PhysicalPlan pp = ((MRExecutionEngine)
+            PhysicalPlan pp = ((HExecutionEngine)
                     pigServer.getPigContext().getExecutionEngine()).compile(lp, null);
             boolean planFetchable = FetchOptimizer.isPlanFetchable(pigServer.getPigContext(), pp);
             assertFalse(planFetchable);

Modified: pig/trunk/test/org/apache/pig/test/TestPigRunner.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPigRunner.java?rev=1644713&r1=1644712&r2=1644713&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigRunner.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPigRunner.java Thu Dec 11 18:18:54 2014
@@ -517,7 +517,7 @@ public class TestPigRunner {
 
     @Test
     public void testIsTempFile() throws Exception {
-        PigContext context = new PigContext(ExecType.LOCAL, new Properties());
+        PigContext context = new PigContext(Util.getLocalTestMode(), new Properties());
         context.connect();
         for (int i=0; i<100; i++) {
             String file = FileLocalizer.getTemporaryPath(context).toString();

Modified: 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=1644713&r1=1644712&r2=1644713&view=diff
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigServerWithMacros.java (original)
+++ pig/trunk/test/org/apache/pig/test/TestPigServerWithMacros.java Thu Dec 11 18:18:54 2014
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertTru
 
 import java.io.File;
 import java.io.FileOutputStream;
-import java.io.FileWriter;
 import java.io.PrintWriter;
 import java.util.Iterator;
 import java.util.jar.Attributes;
@@ -32,61 +31,16 @@ import java.util.jar.JarEntry;
 import java.util.jar.JarOutputStream;
 import java.util.jar.Manifest;
 
-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.AfterClass;
-import org.junit.Before;
 import org.junit.Test;
 
 public class TestPigServerWithMacros {
-    private static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
-
-    @Before
-    public void setup() throws Exception {
-        Util.resetStateForExecModeSwitch();
-    }
-
-    @AfterClass
-    public static void tearDown() throws Exception {
-        cluster.shutDown();
-    }
-
-    @Test
-    public void testRegisterRemoteMacro() throws Throwable {
-        PigServer pig = new PigServer(cluster.getExecType(), cluster.getProperties());
-
-        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");
-
-        assertEquals(2L, ((Long)iter.next().get(0)).longValue());
-
-        pig.shutdown();
-    }
 
     @Test
     public void testInlineMacro() throws Throwable {
-        PigServer pig = new PigServer(ExecType.LOCAL);
+        PigServer pig = new PigServer(Util.getLocalTestMode());
 
         Storage.Data data = resetData(pig);
         data.set("some_path", "(l:chararray)", tuple("first row"), tuple("second row"));
@@ -103,7 +57,7 @@ public class TestPigServerWithMacros {
 
     @Test
     public void testRegisterResourceMacro() throws Throwable {
-        PigServer pig = new PigServer(ExecType.LOCAL);
+        PigServer pig = new PigServer(Util.getLocalTestMode());
 
         String macrosFile = "test/pig/macros.pig";
         File macrosJarFile = File.createTempFile("macros", ".jar");

Added: pig/trunk/test/org/apache/pig/test/TestPigServerWithMacrosRemote.java
URL: http://svn.apache.org/viewvc/pig/trunk/test/org/apache/pig/test/TestPigServerWithMacrosRemote.java?rev=1644713&view=auto
==============================================================================
--- pig/trunk/test/org/apache/pig/test/TestPigServerWithMacrosRemote.java (added)
+++ pig/trunk/test/org/apache/pig/test/TestPigServerWithMacrosRemote.java Thu Dec 11 18:18:54 2014
@@ -0,0 +1,76 @@
+/*
+ * 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 java.io.File;
+import java.io.FileWriter;
+import java.io.PrintWriter;
+import java.util.Iterator;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.pig.PigServer;
+import org.apache.pig.data.Tuple;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestPigServerWithMacrosRemote {
+    private static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
+
+    @Before
+    public void setup() throws Exception {
+        Util.resetStateForExecModeSwitch();
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        cluster.shutDown();
+    }
+
+    @Test
+    public void testRegisterRemoteMacro() throws Throwable {
+        PigServer pig = new PigServer(cluster.getExecType(), cluster.getProperties());
+
+        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");
+
+        assertEquals(2L, ((Long)iter.next().get(0)).longValue());
+
+        pig.shutdown();
+    }
+}

Modified: pig/trunk/test/tez-local-tests
URL: http://svn.apache.org/viewvc/pig/trunk/test/tez-local-tests?rev=1644713&r1=1644712&r2=1644713&view=diff
==============================================================================
--- pig/trunk/test/tez-local-tests (original)
+++ pig/trunk/test/tez-local-tests Thu Dec 11 18:18:54 2014
@@ -64,3 +64,8 @@
 **/TestUnion.java
 **/TestUTF8.java
 **/TestGruntParser.java
+**/TestAutoLocalMode.java
+**/TestEvalPipelineLocal.java
+**/TestFetch.java
+**/TestPigRunner.java
+**/TestPigServerWithMacros.java

Modified: pig/trunk/test/tez-tests
URL: http://svn.apache.org/viewvc/pig/trunk/test/tez-tests?rev=1644713&r1=1644712&r2=1644713&view=diff
==============================================================================
--- pig/trunk/test/tez-tests (original)
+++ pig/trunk/test/tez-tests Thu Dec 11 18:18:54 2014
@@ -67,3 +67,4 @@
 **/TestTezCompiler.java
 **/TestTezJobControlCompiler.java
 **/TestTezLauncher.java
+**/TestPigServerWithMacrosRemote.java