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

svn commit: r1377360 - in /pig/branches/branch-0.10: ./ shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/shims/ shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/ src/org/apache/pig/backend/hadoop/executionengine/ma...

Author: dvryaboy
Date: Sat Aug 25 22:01:54 2012
New Revision: 1377360

URL: http://svn.apache.org/viewvc?rev=1377360&view=rev
Log:
PIG-2890: Revert PIG-2578

Modified:
    pig/branches/branch-0.10/CHANGES.txt
    pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java
    pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java
    pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
    pig/branches/branch-0.10/test/org/apache/pig/test/TestMultiQueryLocal.java

Modified: pig/branches/branch-0.10/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/CHANGES.txt?rev=1377360&r1=1377359&r2=1377360&view=diff
==============================================================================
--- pig/branches/branch-0.10/CHANGES.txt (original)
+++ pig/branches/branch-0.10/CHANGES.txt Sat Aug 25 22:01:54 2012
@@ -28,6 +28,8 @@ PIG-2727: PigStorage Source tagging does
 
 BUG FIXES
 
+PIG-2890: Revert PIG-2578 (dvryaboy)
+
 PIG-2859: Fix few e2e test failures (rohini via daijy)
 
 PIG-2729: Macro expansion does not use pig.import.search.path - UnitTest borked (johannesch via daijy)

Modified: pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java?rev=1377360&r1=1377359&r2=1377360&view=diff
==============================================================================
--- pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java (original)
+++ pig/branches/branch-0.10/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java Sat Aug 25 22:01:54 2012
@@ -49,7 +49,7 @@ public class HadoopShims {
     
     static public TaskAttemptContext createTaskAttemptContext(Configuration conf, 
                                 TaskAttemptID taskId) {
-        TaskAttemptContext newContext = new TaskAttemptContext(new Configuration(conf),
+        TaskAttemptContext newContext = new TaskAttemptContext(conf,
             taskId);
         return newContext;
     }
@@ -57,7 +57,7 @@ public class HadoopShims {
     static public JobContext createJobContext(Configuration conf, 
             JobID jobId) {
         JobContext newJobContext = new JobContext(
-                new Configuration(conf), jobId);
+                conf, jobId);
         return newJobContext;
     }
 

Modified: pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java?rev=1377360&r1=1377359&r2=1377360&view=diff
==============================================================================
--- pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java (original)
+++ pig/branches/branch-0.10/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java Sat Aug 25 22:01:54 2012
@@ -44,13 +44,13 @@ public class HadoopShims {
     
     static public TaskAttemptContext createTaskAttemptContext(Configuration conf, 
                                 TaskAttemptID taskId) {
-        TaskAttemptContext newContext = new TaskAttemptContextImpl(new Configuration(conf), taskId);
+        TaskAttemptContext newContext = new TaskAttemptContextImpl(conf, taskId);
         return newContext;
     }
     
     static public JobContext createJobContext(Configuration conf, 
             JobID jobId) {
-        JobContext newContext = new JobContextImpl(new Configuration(conf), jobId);
+        JobContext newContext = new JobContextImpl(conf, jobId);
         return newContext;
     }
 

Modified: pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java?rev=1377360&r1=1377359&r2=1377360&view=diff
==============================================================================
--- pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java (original)
+++ pig/branches/branch-0.10/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java Sat Aug 25 22:01:54 2012
@@ -462,13 +462,13 @@ public class JobControlCompiler{
             for (POStore st: mapStores) {
                 storeLocations.add(st);
                 StoreFuncInterface sFunc = st.getStoreFunc();
-                sFunc.setStoreLocation(st.getSFile().getFileName(), new org.apache.hadoop.mapreduce.Job(nwJob.getConfiguration()));
+                sFunc.setStoreLocation(st.getSFile().getFileName(), nwJob);
             }
 
             for (POStore st: reduceStores) {
                 storeLocations.add(st);
                 StoreFuncInterface sFunc = st.getStoreFunc();
-                sFunc.setStoreLocation(st.getSFile().getFileName(), new org.apache.hadoop.mapreduce.Job(nwJob.getConfiguration()));
+                sFunc.setStoreLocation(st.getSFile().getFileName(), nwJob);
             }
 
             // the OutputFormat we report to Hadoop is always PigOutputFormat

Modified: pig/branches/branch-0.10/test/org/apache/pig/test/TestMultiQueryLocal.java
URL: http://svn.apache.org/viewvc/pig/branches/branch-0.10/test/org/apache/pig/test/TestMultiQueryLocal.java?rev=1377360&r1=1377359&r2=1377360&view=diff
==============================================================================
--- pig/branches/branch-0.10/test/org/apache/pig/test/TestMultiQueryLocal.java (original)
+++ pig/branches/branch-0.10/test/org/apache/pig/test/TestMultiQueryLocal.java Sat Aug 25 22:01:54 2012
@@ -17,8 +17,6 @@
  */
 package org.apache.pig.test;
 
-import java.io.BufferedReader;
-import java.io.FileReader;
 import java.io.StringReader;
 import java.io.IOException;
 import java.io.File;
@@ -31,16 +29,12 @@ import java.util.Properties;
 import junit.framework.Assert;
 import junit.framework.TestCase;
 
-import org.apache.hadoop.mapreduce.Job;
 import org.apache.pig.ExecType;
 import org.apache.pig.PigException;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecJob;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
-import org.apache.pig.builtin.PigStorage;
-import org.apache.pig.data.Tuple;
-import org.apache.pig.data.TupleFactory;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.plan.Operator;
 import org.apache.pig.impl.plan.OperatorPlan;
@@ -341,69 +335,6 @@ public class TestMultiQueryLocal {
             Assert.fail();
         }
     }
-    
-    public static class PigStorageWithSuffix extends PigStorage {
-
-        private String suffix;
-        public PigStorageWithSuffix(String s) {
-            this.suffix = s;
-        }
-        static private final String key="test.key";
-        @Override
-        public void setStoreLocation(String location, Job job) throws IOException {
-            super.setStoreLocation(location, job);
-            if (job.getConfiguration().get(key)==null) {
-                job.getConfiguration().set(key, suffix);
-            }
-            suffix = job.getConfiguration().get(key);
-        }
-        
-        @Override
-        public void putNext(Tuple f) throws IOException {
-            try {
-                Tuple t = TupleFactory.getInstance().newTuple();
-                for (Object obj : f.getAll()) {
-                    t.append(obj);
-                }
-                t.append(suffix);
-                writer.write(null, t);
-            } catch (InterruptedException e) {
-                throw new IOException(e);
-            }
-        }
-    }
-    
-    // See PIG-2578
-    @Test
-    public void testMultiStoreWithConfig() {
-
-        System.out.println("===== test multi-query with competing config =====");
-
-        try {
-            myPig.setBatchOn();
-
-            myPig.registerQuery("a = load 'test/org/apache/pig/test/data/passwd' " +
-                                "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int,gid:int);");
-            myPig.registerQuery("store a into '/tmp/Pig-TestMultiQueryLocal1' using " + PigStorageWithSuffix.class.getName() + "('a');");
-            myPig.registerQuery("store a into '/tmp/Pig-TestMultiQueryLocal2' using " + PigStorageWithSuffix.class.getName() + "('b');");
-
-            myPig.executeBatch();
-            myPig.discardBatch();
-            BufferedReader reader = new BufferedReader(new FileReader("/tmp/Pig-TestMultiQueryLocal1/part-m-00000"));
-            String line;
-            while ((line = reader.readLine())!=null) {
-                Assert.assertTrue(line.endsWith("a"));
-            }
-            reader = new BufferedReader(new FileReader("/tmp/Pig-TestMultiQueryLocal2/part-m-00000"));
-            while ((line = reader.readLine())!=null) {
-                Assert.assertTrue(line.endsWith("b"));
-            }
-
-        } catch (Exception e) {
-            e.printStackTrace();
-            Assert.fail();
-        }
-    }
 
     @Test
     public void testMultiQueryWithExplain() {