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

MAPREDUCE-2558. Add queue-level metrics 0.20-security branch - test fix
(jeffrey nasbit via mahadev)


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-204@1136701 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley пре 14 година
родитељ
комит
ae3ebf14f0

+ 3 - 0
CHANGES.txt

@@ -11,6 +11,9 @@ Release 0.20.204.0 - unreleased
 
   BUG FIXES
 
+    MAPREDUCE-2558. Add queue-level metrics 0.20-security branch - test fix
+    (jeffrey nasbit via mahadev)
+    
     HADOOP-7364. TestMiniMRDFSCaching fails if test.build.dir is set to 
     something other than build/test. (Thomas Graves via mahadev)
 

+ 13 - 0
src/mapred/org/apache/hadoop/mapred/JobInProgress.java

@@ -347,6 +347,12 @@ public class JobInProgress {
     this.mapFailuresPercent = conf.getMaxMapTaskFailuresPercent();
     this.reduceFailuresPercent = conf.getMaxReduceTaskFailuresPercent();
 
+    Queue queue = this.jobtracker.getQueueManager().getQueue(queueName);
+    if (queue == null) {
+      throw new IOException("Queue \"" + queueName + "\" does not exist");
+    }
+    this.queueMetrics = queue.getMetrics();
+
     // Check task limits
     checkTaskLimits();
 
@@ -422,6 +428,13 @@ public class JobInProgress {
           jobFile, url, conf.getJobName(),
           conf.getQueueName());
 
+      Queue queue = this.jobtracker.getQueueManager().getQueue(queueName);
+      if (queue == null) {
+        throw new IOException("Queue \"" + queueName + "\" does not exist");
+      }
+      this.queueMetrics = queue.getMetrics();
+      this.queueMetrics.addPrepJob(conf, jobId);
+
       this.submitHostName = conf.getJobSubmitHostName();
       this.submitHostAddress = conf.getJobSubmitHostAddress();
       this.numMapTasks = conf.getNumMapTasks();

+ 1 - 6
src/mapred/org/apache/hadoop/mapred/JobTracker.java

@@ -3926,13 +3926,8 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     }
     
     synchronized (this) {
-      String queue = job.getProfile().getQueueName();
-      if (!(queueManager.getQueues().contains(queue))) {
-        job.fail();
-        throw new IOException("Queue \"" + queue + "\" does not exist");
-      }
-
       // check if queue is RUNNING
+      String queue = job.getProfile().getQueueName();
       if (!queueManager.isRunning(queue)) {
         throw new IOException("Queue \"" + queue + "\" is not running");
       }

+ 1 - 1
src/test/org/apache/hadoop/mapred/TestJobHistoryConfig.java

@@ -110,7 +110,7 @@ public class TestJobHistoryConfig extends TestCase {
     conf.setSpeculativeExecution(false);
     conf.setJobName("test");
     conf.setUser("testuser");
-    conf.setQueueName("testQueue");
+    conf.setQueueName("default");
     String TEST_ROOT_DIR = new Path(System.getProperty("test.build.data",
         "/tmp")).toString().replace(' ', '+');
     JobTracker jt = JobTracker.startTracker(conf);