ソースを参照

HADOOP-3338. Fix Eclipse plugin to compile after HADOOP-544 was committed. Updated all references to use the new JobID representation. Contributed by Christophe Taton.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@653105 13f79535-47bb-0310-9956-ffa450edef68
Nigel Daley 17 年 前
コミット
42f4407fa4

+ 4 - 0
CHANGES.txt

@@ -131,6 +131,10 @@ Trunk (unreleased changes)
     HADOOP-3301. Fix misleading error message when S3 URI hostname
     contains an underscore. (tomwhite via omalley)
 
+    HADOOP-3338. Fix Eclipse plugin to compile after HADOOP-544 was
+    committed. Updated all references to use the new JobID representation.
+    (taton via nigel)
+
 Release 0.17.0 - Unreleased
 
   INCOMPATIBLE CHANGES

+ 1 - 1
src/contrib/eclipse-plugin/META-INF/MANIFEST.MF

@@ -2,7 +2,7 @@ Manifest-Version: 1.0
 Bundle-ManifestVersion: 2
 Bundle-Name: MapReduce Tools for Eclipse
 Bundle-SymbolicName: org.apache.hadoop.eclipse;singleton:=true
-Bundle-Version: 0.16.2
+Bundle-Version: 0.18
 Bundle-Activator: org.apache.hadoop.eclipse.Activator
 Bundle-Localization: plugin
 Require-Bundle: org.eclipse.ui,

+ 7 - 6
src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/server/HadoopJob.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.Counters;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobID;
 import org.apache.hadoop.mapred.JobStatus;
 import org.apache.hadoop.mapred.RunningJob;
 
@@ -72,7 +73,7 @@ public class HadoopJob {
   /**
    * Unique identifier of this Job
    */
-  final String jobId;
+  final JobID jobId;
 
   /**
    * Status representation of a running job. This actually contains a
@@ -123,7 +124,7 @@ public class HadoopJob {
    * @param running
    * @param status
    */
-  public HadoopJob(HadoopServer location, String id, RunningJob running,
+  public HadoopJob(HadoopServer location, JobID id, RunningJob running,
       JobStatus status) {
 
     this.location = location;
@@ -143,7 +144,7 @@ public class HadoopJob {
     try {
       String jobFile = getJobFile();
       FileSystem fs = location.getDFS();
-      File tmp = File.createTempFile(getJobId(), ".xml");
+      File tmp = File.createTempFile(getJobID().toString(), ".xml");
       if (FileUtil.copy(fs, new Path(jobFile), tmp, false, location
           .getConfiguration())) {
         this.jobConf = new JobConf(tmp.toString());
@@ -211,7 +212,7 @@ public class HadoopJob {
   /**
    * @return
    */
-  public String getJobId() {
+  public JobID getJobID() {
     return this.jobId;
   }
 
@@ -326,8 +327,8 @@ public class HadoopJob {
    * Print this job status (for debugging purpose)
    */
   public void display() {
-    System.out.printf("Job id=%s, name=%s\n", getJobId(), getJobName());
-    System.out.printf("Configuration file: %s\n", getJobId());
+    System.out.printf("Job id=%s, name=%s\n", getJobID(), getJobName());
+    System.out.printf("Configuration file: %s\n", getJobID());
     System.out.printf("Tracking URL: %s\n", getTrackingURL());
 
     System.out.printf("Completion: map: %f reduce %f\n",

+ 9 - 8
src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/server/HadoopServer.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.eclipse.Activator;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobID;
 import org.apache.hadoop.mapred.JobStatus;
 import org.apache.hadoop.mapred.RunningJob;
 import org.eclipse.core.runtime.IProgressMonitor;
@@ -110,13 +111,13 @@ public class HadoopServer {
 
       try {
         // Set of all known existing Job IDs we want fresh info of
-        Set<String> missingJobIds =
-            new HashSet<String>(runningJobs.keySet());
+        Set<JobID> missingJobIds =
+            new HashSet<JobID>(runningJobs.keySet());
 
         JobStatus[] jstatus = client.jobsToComplete();
         for (JobStatus status : jstatus) {
 
-          String jobId = status.getJobId();
+          JobID jobId = status.getJobID();
           missingJobIds.remove(jobId);
 
           HadoopJob hJob;
@@ -136,7 +137,7 @@ public class HadoopServer {
         }
 
         // Ask explicitly for fresh info for these Job IDs
-        for (String jobId : missingJobIds) {
+        for (JobID jobId : missingJobIds) {
           HadoopJob hJob = runningJobs.get(jobId);
           if (!hJob.isCompleted())
             updateJob(hJob, null);
@@ -161,7 +162,7 @@ public class HadoopServer {
      * @param data
      */
     private void newJob(final HadoopJob data) {
-      runningJobs.put(data.getJobId(), data);
+      runningJobs.put(data.getJobID(), data);
 
       Display.getDefault().asyncExec(new Runnable() {
         public void run() {
@@ -203,8 +204,8 @@ public class HadoopServer {
   /**
    * Jobs running on this location. The keys of this map are the Job IDs.
    */
-  private transient Map<String, HadoopJob> runningJobs =
-      Collections.synchronizedMap(new TreeMap<String, HadoopJob>());
+  private transient Map<JobID, HadoopJob> runningJobs =
+      Collections.synchronizedMap(new TreeMap<JobID, HadoopJob>());
 
   /**
    * Status updater for this location
@@ -272,7 +273,7 @@ public class HadoopServer {
    * @param job the job to remove
    */
   public void purgeJob(final HadoopJob job) {
-    runningJobs.remove(job.getJobId());
+    runningJobs.remove(job.getJobID());
     Display.getDefault().asyncExec(new Runnable() {
       public void run() {
         fireJobRemoved(job);

+ 2 - 2
src/contrib/eclipse-plugin/src/java/org/apache/hadoop/eclipse/view/servers/ServerView.java

@@ -106,7 +106,7 @@ public class ServerView extends ViewPart implements ITreeContentProvider,
             // Job is running, kill the job?
             if (MessageDialog.openConfirm(Display.getDefault()
                 .getActiveShell(), "Confirm kill running Job",
-                "Do you really want to kill running Job: " + job.getJobId())) {
+                "Do you really want to kill running Job: " + job.getJobID())) {
               job.kill();
             }
           }
@@ -385,7 +385,7 @@ public class ServerView extends ViewPart implements ITreeContentProvider,
 
       switch (columnIndex) {
         case 0:
-          return job.getJobId();
+          return job.getJobID().toString();
         case 1:
           return "";
         case 2: