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 yh...@apache.org on 2009/05/05 12:49:54 UTC

svn commit: r771661 - in /hadoop/core/trunk: CHANGES.txt src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacityTaskScheduler.java src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java

Author: yhemanth
Date: Tue May  5 10:49:53 2009
New Revision: 771661

URL: http://svn.apache.org/viewvc?rev=771661&view=rev
Log:
HADOOP-5718. Remove the check for the default queue in capacity scheduler. Contributed by Sreekanth Ramakrishnan.

Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacityTaskScheduler.java
    hadoop/core/trunk/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=771661&r1=771660&r2=771661&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Tue May  5 10:49:53 2009
@@ -564,6 +564,9 @@
     HADOOP-4674. Fix fs help messages for -test, -text, -tail, -stat 
     and -touchz options.  (Ravi Phulari via szetszwo)
 
+    HADOOP-5718. Remove the check for the default queue in capacity scheduler.
+    (Sreekanth Ramakrishnan via yhemanth)
+
 Release 0.20.0 - Unreleased
 
   INCOMPATIBLE CHANGES

Modified: hadoop/core/trunk/src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacityTaskScheduler.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacityTaskScheduler.java?rev=771661&r1=771660&r2=771661&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacityTaskScheduler.java (original)
+++ hadoop/core/trunk/src/contrib/capacity-scheduler/src/java/org/apache/hadoop/mapred/CapacityTaskScheduler.java Tue May  5 10:49:53 2009
@@ -592,9 +592,7 @@
   private int prevMapClusterCapacity = 0;
   private int prevReduceClusterCapacity = 0;
   
-  /** name of the default queue. */ 
-  static final String DEFAULT_QUEUE_NAME = "default";
-  
+    
   static final Log LOG = LogFactory.getLog(CapacityTaskScheduler.class);
   protected JobQueuesManager jobQueuesManager;
   protected CapacitySchedulerConf schedConf;
@@ -717,10 +715,6 @@
       schedConf.setCapacity(queue, quantityToAllocate);
     }    
     
-    // check if there's a queue with the default name. If not, we quit.
-    if (!queueInfoMap.containsKey(DEFAULT_QUEUE_NAME)) {
-      throw new IllegalStateException("System has no default queue configured");
-    }
     if (totalCapacity > 100.0) {
       throw new IllegalArgumentException("Sum of queue capacities over 100% at "
                                          + totalCapacity);

Modified: hadoop/core/trunk/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java?rev=771661&r1=771660&r2=771661&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java (original)
+++ hadoop/core/trunk/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java Tue May  5 10:49:53 2009
@@ -1254,6 +1254,7 @@
 
     //Raise status change event so that jobs can move to running queue.
     raiseStatusChangeEvents(scheduler.jobQueuesManager);
+    raiseStatusChangeEvents(scheduler.jobQueuesManager, "q2");
     //assign one job
     Task t1 = checkAssignment("tt1", "attempt_test_0001_m_000001_0 on tt1");
     //Initalize extra job.
@@ -1338,6 +1339,7 @@
     //in running queue as we just failed the second job which was initialized
     //and completed the first one.
     raiseStatusChangeEvents(scheduler.jobQueuesManager);
+    raiseStatusChangeEvents(scheduler.jobQueuesManager, "q2");
 
     //Now schedule a map should be job3 of the user as job1 succeeded job2
     //failed and now job3 is running
@@ -1995,6 +1997,25 @@
     
   }
   
+  public void testStartWithoutDefaultQueueConfigured() throws Exception {
+    //configure a single queue which is not default queue
+    String[] qs = {"q1"};
+    taskTrackerManager.addQueues(qs);
+    ArrayList<FakeQueueInfo> queues = new ArrayList<FakeQueueInfo>();
+    queues.add(new FakeQueueInfo("q1", 100.0f, true, 100));
+    resConf.setFakeQueues(queues);
+    scheduler.setResourceManagerConf(resConf);
+    //Start the scheduler.
+    scheduler.start();
+    //Submit a job and wait till it completes
+    FakeJobInProgress job = 
+      submitJob(JobStatus.PREP, 1, 1, "q1", "u1");
+    controlledInitializationPoller.selectJobsToInitialize();
+    raiseStatusChangeEvents(scheduler.jobQueuesManager, "q1");
+    Task t = checkAssignment("tt1", "attempt_test_0001_m_000001_0 on tt1");
+    t = checkAssignment("tt1", "attempt_test_0001_r_000001_0 on tt1");
+  }
+  
   private void checkRunningJobMovementAndCompletion() throws IOException {
     
     JobQueuesManager mgr = scheduler.jobQueuesManager;
@@ -2110,7 +2131,11 @@
   }
   
   private void raiseStatusChangeEvents(JobQueuesManager mgr) {
-    Collection<JobInProgress> jips = mgr.getWaitingJobs("default");
+    raiseStatusChangeEvents(mgr, "default");
+  }
+  
+  private void raiseStatusChangeEvents(JobQueuesManager mgr, String queueName) {
+    Collection<JobInProgress> jips = mgr.getWaitingJobs(queueName);
     for(JobInProgress jip : jips) {
       if(jip.getStatus().getRunState() == JobStatus.RUNNING) {
         JobStatusChangeEvent evt = new JobStatusChangeEvent(jip,