Browse Source

Reverting MAPREDUCE-4346 r1353757

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1354656 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 13 years ago
parent
commit
1b21c9836e

+ 0 - 2
hadoop-mapreduce-project/CHANGES.txt

@@ -129,8 +129,6 @@ Branch-2 ( Unreleased changes )
 
 
   NEW FEATURES
   NEW FEATURES
 
 
-    MAPREDUCE-4355. Add JobStatus getJobStatus(JobID) to JobClient. (kkambatl via tucu)
-
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     MAPREDUCE-4146. Support limits on task status string length and number of
     MAPREDUCE-4146. Support limits on task status string length and number of

+ 2 - 20
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestJobClient.java → hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestJobClientGetJob.java

@@ -19,7 +19,6 @@
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
 import static junit.framework.Assert.assertNotNull;
 import static junit.framework.Assert.assertNotNull;
-import static org.junit.Assert.assertEquals;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
@@ -29,7 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
 import org.junit.Test;
 
 
-public class TestJobClient {
+public class TestJobClientGetJob {
   
   
   private static Path TEST_ROOT_DIR =
   private static Path TEST_ROOT_DIR =
     new Path(System.getProperty("test.build.data","/tmp"));
     new Path(System.getProperty("test.build.data","/tmp"));
@@ -46,7 +45,7 @@ public class TestJobClient {
   
   
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
   @Test
   @Test
-  public void testGetRunningJob() throws Exception {
+  public void testGetRunningJobFromJobClient() throws Exception {
     JobConf conf = new JobConf();
     JobConf conf = new JobConf();
     conf.set("mapreduce.framework.name", "local");
     conf.set("mapreduce.framework.name", "local");
     FileInputFormat.addInputPath(conf, createTempFile("in", "hello"));
     FileInputFormat.addInputPath(conf, createTempFile("in", "hello"));
@@ -61,21 +60,4 @@ public class TestJobClient {
     assertNotNull("New running job", newRunningJob);
     assertNotNull("New running job", newRunningJob);
   }
   }
 
 
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testGetJobStatus() throws Exception {
-    JobConf conf = new JobConf();
-    conf.set("mapreduce.framework.name", "local");
-    FileInputFormat.addInputPath(conf, createTempFile("in", "hello"));
-    Path outputDir = new Path(TEST_ROOT_DIR, getClass().getSimpleName());
-    outputDir.getFileSystem(conf).delete(outputDir, true);
-    FileOutputFormat.setOutputPath(conf, outputDir);
-    JobClient jc = new JobClient(conf);
-    RunningJob runningJob = jc.submitJob(conf);
-    assertNotNull("Running job", runningJob);
-    JobID jobid = runningJob.getID();
-    JobStatus jobStatus = jc.getJobStatus(jobid);
-    assertNotNull("New running job", jobStatus);
-    assertEquals("Equal JobIDs", jobid, jobStatus.getJobID());
-  }
 }
 }

+ 13 - 34
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java

@@ -620,15 +620,6 @@ public class JobClient extends CLI {
     }
     }
   }
   }
 
 
-  private JobStatus getJobStatusUsingCluster(final JobID jobId)
-      throws IOException, InterruptedException {
-    return clientUgi.doAs(new PrivilegedExceptionAction<JobStatus>() {
-      public JobStatus run() throws IOException, InterruptedException {
-        return JobStatus.downgrade(cluster.getJobStatus(jobId));
-      }
-    });
-  }
-
   private Job getJobUsingCluster(final JobID jobid) throws IOException,
   private Job getJobUsingCluster(final JobID jobid) throws IOException,
   InterruptedException {
   InterruptedException {
     return clientUgi.doAs(new PrivilegedExceptionAction<Job>() {
     return clientUgi.doAs(new PrivilegedExceptionAction<Job>() {
@@ -637,40 +628,28 @@ public class JobClient extends CLI {
       }
       }
     });
     });
   }
   }
-
   /**
   /**
-   * Get {@link JobStatus} of a job. Returns null if the id does not correspond
-   * to any known job.
+   * Get an {@link RunningJob} object to track an ongoing job.  Returns
+   * null if the id does not correspond to any known job.
    * 
    * 
-   * @param jobid
-   *          the jobid of the job.
-   * @return the {@link JobStatus} object to retrieve the job stats, null if the
+   * @param jobid the jobid of the job.
+   * @return the {@link RunningJob} handle to track the job, null if the 
    *         <code>jobid</code> doesn't correspond to any known job.
    *         <code>jobid</code> doesn't correspond to any known job.
    * @throws IOException
    * @throws IOException
    */
    */
-  public JobStatus getJobStatus(JobID jobId) throws IOException {
+  public RunningJob getJob(final JobID jobid) throws IOException {
     try {
     try {
-      return getJobStatusUsingCluster(jobId);
+      
+      Job job = getJobUsingCluster(jobid);
+      if (job != null) {
+        JobStatus status = JobStatus.downgrade(job.getStatus());
+        if (status != null) {
+          return new NetworkedJob(status, cluster);
+        } 
+      }
     } catch (InterruptedException ie) {
     } catch (InterruptedException ie) {
       throw new IOException(ie);
       throw new IOException(ie);
     }
     }
-  }
-
-  /**
-   * Get an {@link RunningJob} object to track an ongoing job. Returns null if
-   * the id does not correspond to any known job.
-   * 
-   * @param jobid
-   *          the jobid of the job.
-   * @return the {@link RunningJob} handle to track the job, null if the
-   *         <code>jobid</code> doesn't correspond to any known job.
-   * @throws IOException
-   */
-  public RunningJob getJob(JobID jobId) throws IOException {
-    JobStatus status = getJobStatus(jobId);
-    if (status != null) {
-      return new NetworkedJob(status, cluster);
-    }
     return null;
     return null;
   }
   }
 
 

+ 1 - 14
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java

@@ -172,19 +172,6 @@ public class Cluster {
     return fs;
     return fs;
   }
   }
 
 
-  /**
-   * Get JobStatus corresponding to jobId.
-   * 
-   * @param jobId
-   * @return object of {@link JobStatus}
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public JobStatus getJobStatus(JobID jobId) throws IOException,
-      InterruptedException {
-    return client.getJobStatus(jobId);
-  }
-
   /**
   /**
    * Get job corresponding to jobid.
    * Get job corresponding to jobid.
    * 
    * 
@@ -194,7 +181,7 @@ public class Cluster {
    * @throws InterruptedException
    * @throws InterruptedException
    */
    */
   public Job getJob(JobID jobId) throws IOException, InterruptedException {
   public Job getJob(JobID jobId) throws IOException, InterruptedException {
-    JobStatus status = getJobStatus(jobId);
+    JobStatus status = client.getJobStatus(jobId);
     if (status != null) {
     if (status != null) {
       return Job.getInstance(this, status, new JobConf(status.getJobFile()));
       return Job.getInstance(this, status, new JobConf(status.getJobFile()));
     }
     }