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/10/02 00:08:35 UTC
svn commit: r1628852 - in /pig/trunk: CHANGES.txt
shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java
Author: daijy
Date: Wed Oct 1 22:08:35 2014
New Revision: 1628852
URL: http://svn.apache.org/r1628852
Log:
PIG-4164: After Pig job finish, Pig client spend too much time retry to connect to AM
Modified:
pig/trunk/CHANGES.txt
pig/trunk/shims/src/hadoop23/org/apache/pig/backend/hadoop/executionengine/shims/HadoopShims.java
Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1628852&r1=1628851&r2=1628852&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Wed Oct 1 22:08:35 2014
@@ -92,6 +92,8 @@ OPTIMIZATIONS
BUG FIXES
+PIG-4164: After Pig job finish, Pig client spend too much time retry to connect to AM (daijy)
+
PIG-4212: Allow LIMIT of 0 for variableLimit (constant 0 is already allowed) (knoguchi)
PIG-4196: Auto ship udf jar is broken (daijy)
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=1628852&r1=1628851&r2=1628852&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 Wed Oct 1 22:08:35 2014
@@ -33,6 +33,7 @@ import org.apache.hadoop.mapred.TIPStatu
import org.apache.hadoop.mapred.TaskReport;
import org.apache.hadoop.mapred.jobcontrol.Job;
import org.apache.hadoop.mapred.jobcontrol.JobControl;
+import org.apache.hadoop.mapreduce.Cluster;
import org.apache.hadoop.mapreduce.ContextFactory;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.JobID;
@@ -121,7 +122,12 @@ public class HadoopShims {
public static Counters getCounters(Job job) throws IOException {
try {
- return new Counters(job.getJob().getCounters());
+ Cluster cluster = new Cluster(job.getJobConf());
+ org.apache.hadoop.mapreduce.Job mrJob = cluster.getJob(job.getAssignedJobID());
+ if (mrJob == null) { // In local mode, mrJob will be null
+ mrJob = job.getJob();
+ }
+ return new Counters(mrJob.getCounters());
} catch (Exception ir) {
throw new IOException(ir);
}
@@ -220,8 +226,12 @@ public class HadoopShims {
LOG.info("TaskReports are disabled for job: " + job.getAssignedJobID());
return null;
}
- org.apache.hadoop.mapreduce.Job mrJob = job.getJob();
+ Cluster cluster = new Cluster(job.getJobConf());
try {
+ org.apache.hadoop.mapreduce.Job mrJob = cluster.getJob(job.getAssignedJobID());
+ if (mrJob == null) { // In local mode, mrJob will be null
+ mrJob = job.getJob();
+ }
org.apache.hadoop.mapreduce.TaskReport[] reports = mrJob.getTaskReports(type);
return DowngradeHelper.downgradeTaskReports(reports);
} catch (InterruptedException ir) {