You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ma...@apache.org on 2011/06/09 09:31:07 UTC

svn commit: r1133690 - in /hadoop/common/branches/branch-0.20-security: CHANGES.txt src/mapred/org/apache/hadoop/mapred/JobInProgress.java src/mapred/org/apache/hadoop/mapred/JobTracker.java src/test/org/apache/hadoop/mapred/TestJobHistoryConfig.java

Author: mahadev
Date: Thu Jun  9 07:31:07 2011
New Revision: 1133690

URL: http://svn.apache.org/viewvc?rev=1133690&view=rev
Log:
MAPREDUCE-2558. Add queue-level metrics 0.20-security branch (test fixes)

Modified:
    hadoop/common/branches/branch-0.20-security/CHANGES.txt
    hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapred/JobInProgress.java
    hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapred/JobTracker.java
    hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/mapred/TestJobHistoryConfig.java

Modified: hadoop/common/branches/branch-0.20-security/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/CHANGES.txt?rev=1133690&r1=1133689&r2=1133690&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.20-security/CHANGES.txt Thu Jun  9 07:31:07 2011
@@ -71,6 +71,9 @@ Release 0.20.205.0 - unreleased
     HDFS-2044. TestQueueProcessingStatistics failing automatic test due to 
     timing issues. (mattf)
 
+    MAPREDUCE-2558. Add queue-level metrics 0.20-security branch (test fixes)
+    (Jeffrey Naisbitt via mahadev) 
+
 Release 0.20.204.0 - unreleased
 
   NEW FEATURES

Modified: hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapred/JobInProgress.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapred/JobInProgress.java?rev=1133690&r1=1133689&r2=1133690&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapred/JobInProgress.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapred/JobInProgress.java Thu Jun  9 07:31:07 2011
@@ -349,7 +349,11 @@ public class JobInProgress {
     this.mapFailuresPercent = conf.getMaxMapTaskFailuresPercent();
     this.reduceFailuresPercent = conf.getMaxReduceTaskFailuresPercent();
 
-    this.queueMetrics = this.jobtracker.getQueueManager().getQueue(queueName).getMetrics();
+    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();
@@ -427,7 +431,11 @@ public class JobInProgress {
       this.profile = new JobProfile(user, jobId, 
           jobFile, url, conf.getJobName(), queueName);
 
-      this.queueMetrics = this.jobtracker.getQueueManager().getQueue(queueName).getMetrics();
+      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();

Modified: hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapred/JobTracker.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapred/JobTracker.java?rev=1133690&r1=1133689&r2=1133690&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapred/JobTracker.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/mapred/org/apache/hadoop/mapred/JobTracker.java Thu Jun  9 07:31:07 2011
@@ -3932,13 +3932,8 @@ public class JobTracker implements MRCon
     }
     
     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");
       }

Modified: hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/mapred/TestJobHistoryConfig.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/mapred/TestJobHistoryConfig.java?rev=1133690&r1=1133689&r2=1133690&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/mapred/TestJobHistoryConfig.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/mapred/TestJobHistoryConfig.java Thu Jun  9 07:31:07 2011
@@ -110,7 +110,7 @@ public class TestJobHistoryConfig extend
     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);