Преглед изворни кода

MAPREDUCE-2679. Minor changes to sync trunk with MR-279 branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1145870 13f79535-47bb-0310-9956-ffa450edef68
Arun Murthy пре 14 година
родитељ
комит
f57eaf3989

+ 2 - 0
mapreduce/CHANGES.txt

@@ -37,6 +37,8 @@ Trunk (unreleased changes)
 
 
   IMPROVEMENTS
   IMPROVEMENTS
 
 
+    MAPREDUCE-2679. Minor changes to sync trunk with MR-279 branch. (acmurthy) 
+ 
     MAPREDUCE-2400. Remove Cluster's dependency on JobTracker via a 
     MAPREDUCE-2400. Remove Cluster's dependency on JobTracker via a 
     ServiceProvider for the actual implementation. (tomwhite via acmurthy) 
     ServiceProvider for the actual implementation. (tomwhite via acmurthy) 
  
  

+ 4 - 3
mapreduce/src/java/org/apache/hadoop/mapred/JobACLsManager.java

@@ -21,6 +21,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
@@ -30,9 +31,9 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 class JobACLsManager {
 class JobACLsManager {
 
 
-  JobConf conf;
+  Configuration conf;
 
 
-  public JobACLsManager(JobConf conf) {
+  public JobACLsManager(Configuration conf) {
     this.conf = conf;
     this.conf = conf;
   }
   }
 
 
@@ -47,7 +48,7 @@ class JobACLsManager {
    * 
    * 
    * @return JobACL to AccessControlList map.
    * @return JobACL to AccessControlList map.
    */
    */
-  Map<JobACL, AccessControlList> constructJobACLs(JobConf conf) {
+  Map<JobACL, AccessControlList> constructJobACLs(Configuration conf) {
 
 
     Map<JobACL, AccessControlList> acls =
     Map<JobACL, AccessControlList> acls =
         new HashMap<JobACL, AccessControlList>();
         new HashMap<JobACL, AccessControlList>();

+ 2 - 2
mapreduce/src/java/org/apache/hadoop/mapred/JobContextImpl.java

@@ -33,14 +33,14 @@ public class JobContextImpl
   private JobConf job;
   private JobConf job;
   private Progressable progress;
   private Progressable progress;
 
 
-  JobContextImpl(JobConf conf, org.apache.hadoop.mapreduce.JobID jobId, 
+  public JobContextImpl(JobConf conf, org.apache.hadoop.mapreduce.JobID jobId, 
                  Progressable progress) {
                  Progressable progress) {
     super(conf, jobId);
     super(conf, jobId);
     this.job = conf;
     this.job = conf;
     this.progress = progress;
     this.progress = progress;
   }
   }
 
 
-  JobContextImpl(JobConf conf, org.apache.hadoop.mapreduce.JobID jobId) {
+  public JobContextImpl(JobConf conf, org.apache.hadoop.mapreduce.JobID jobId) {
     this(conf, jobId, Reporter.NULL);
     this(conf, jobId, Reporter.NULL);
   }
   }
   
   

+ 1 - 1
mapreduce/src/java/org/apache/hadoop/mapred/TaskAttemptContextImpl.java

@@ -34,7 +34,7 @@ public class TaskAttemptContextImpl
        implements TaskAttemptContext {
        implements TaskAttemptContext {
   private Reporter reporter;
   private Reporter reporter;
 
 
-  TaskAttemptContextImpl(JobConf conf, TaskAttemptID taskid) {
+  public TaskAttemptContextImpl(JobConf conf, TaskAttemptID taskid) {
     this(conf, taskid, Reporter.NULL);
     this(conf, taskid, Reporter.NULL);
   }
   }
   
   

+ 7 - 3
mapreduce/src/java/org/apache/hadoop/mapreduce/Job.java

@@ -1113,13 +1113,17 @@ public class Job extends JobContextImpl implements JobContext {
       eventCounter += events.length;
       eventCounter += events.length;
       printTaskEvents(events, filter, profiling, mapRanges, reduceRanges);
       printTaskEvents(events, filter, profiling, mapRanges, reduceRanges);
     }
     }
+    boolean success = isSuccessful();
+    if (success) {
+      LOG.info("Job " + jobId + " completed successfully");
+    } else {
+      LOG.info("Job " + jobId + " failed with state " + status.getState());
+    }
     Counters counters = getCounters();
     Counters counters = getCounters();
     if (counters != null) {
     if (counters != null) {
       LOG.info(counters.toString());
       LOG.info(counters.toString());
     }
     }
-    LOG.info("Job " + jobId + " completed with status: "
-          + getStatus().getState());
-    return isSuccessful();
+    return success;
   }
   }
 
 
   /**
   /**

+ 2 - 1
mapreduce/src/java/org/apache/hadoop/mapreduce/jobhistory/EventWriter.java

@@ -66,8 +66,9 @@ class EventWriter {
     out.writeBytes("\n");
     out.writeBytes("\n");
   }
   }
   
   
-  void flush() throws IOException { 
+  void flush() throws IOException {
     encoder.flush();
     encoder.flush();
+    out.flush();
   }
   }
 
 
   void close() throws IOException {
   void close() throws IOException {