You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by gd...@apache.org on 2012/10/18 20:44:51 UTC

svn commit: r1399775 - in /pig/trunk: ./ 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/mapReduceLayer/

Author: gdfm
Date: Thu Oct 18 18:44:51 2012
New Revision: 1399775

URL: http://svn.apache.org/viewvc?rev=1399775&view=rev
Log:
PIG-2985: TestRank1,2,3 fail with hadoop-2.0.x (rohini via azaroth)

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java
    pig/trunk/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1399775&r1=1399774&r2=1399775&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Thu Oct 18 18:44:51 2012
@@ -321,6 +321,8 @@ OPTIMIZATIONS
 
 BUG FIXES
 
+PIG-2985: TestRank1,2,3 fail with hadoop-2.0.x (rohini via azaroth)
+
 PIG-2971: Add new parameter to specify the streaming environment (jcoveney)
 
 PIG-2963: Illustrate command and POPackageLite (cheolsoo via jcoveney)

Modified: pig/trunk/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java
URL: http://svn.apache.org/viewvc/pig/trunk/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java?rev=1399775&r1=1399774&r2=1399775&view=diff
==============================================================================
--- pig/trunk/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java (original)
+++ pig/trunk/shims/src/hadoop20/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java Thu Oct 18 18:44:51 2012
@@ -22,6 +22,8 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.jobcontrol.Job;
 import org.apache.hadoop.mapred.jobcontrol.JobControl;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -32,7 +34,6 @@ import org.apache.hadoop.mapreduce.TaskA
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputCommitter;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
 import org.apache.pig.backend.hadoop20.PigJobControl;
-import org.apache.pig.impl.PigContext;
 
 /**
  * We need to make Pig work with both hadoop 20 and hadoop 23 (PIG-2125). However,
@@ -99,4 +100,9 @@ public class HadoopShims {
     public static long getDefaultBlockSize(FileSystem fs, Path path) {
         return fs.getDefaultBlockSize();
     }
+
+    public static Counters getCounters(Job job) throws IOException {
+        JobClient jobClient = job.getJobClient();
+        return jobClient.getJob(job.getAssignedJobID()).getCounters();
+    }
 }

Modified: pig/trunk/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java
URL: http://svn.apache.org/viewvc/pig/trunk/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java?rev=1399775&r1=1399774&r2=1399775&view=diff
==============================================================================
--- pig/trunk/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java (original)
+++ pig/trunk/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java Thu Oct 18 18:44:51 2012
@@ -22,6 +22,7 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.Counters;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.jobcontrol.Job;
 import org.apache.hadoop.mapred.jobcontrol.JobControl;
@@ -36,7 +37,6 @@ import org.apache.hadoop.mapreduce.task.
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
 import org.apache.pig.backend.hadoop23.PigJobControl;
-import org.apache.pig.impl.PigContext;
 
 public class HadoopShims {
     static public JobContext cloneJobContext(JobContext original) throws IOException, InterruptedException {
@@ -105,4 +105,8 @@ public class HadoopShims {
     public static long getDefaultBlockSize(FileSystem fs, Path path) {
         return fs.getDefaultBlockSize(path);
     }
+
+    public static Counters getCounters(Job job) throws IOException, InterruptedException {
+        return new Counters(job.getJob().getCounters());
+    }
 }

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java?rev=1399775&r1=1399774&r2=1399775&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java Thu Oct 18 18:44:51 2012
@@ -346,7 +346,6 @@ public class JobControlCompiler{
      * operation identifier
      */
     private void saveCounters(Job job, String operationID) {
-        JobClient jobClient;
         Counters counters;
         Group groupCounters;
 
@@ -355,8 +354,7 @@ public class JobControlCompiler{
         ArrayList<Pair<String,Long>> counterPairs;
 
         try {
-            jobClient = job.getJobClient();
-            counters = jobClient.getJob(job.getAssignedJobID()).getCounters();
+            counters = HadoopShims.getCounters(job);
             groupCounters = counters.getGroup(getGroupName(counters.getGroupNames()));
 
             Iterator<Counter> it = groupCounters.iterator();