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 om...@apache.org on 2011/03/04 05:25:44 UTC

svn commit: r1077534 - in /hadoop/common/branches/branch-0.20-security-patches/src: contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/ mapred/ mapred/org/apache/hadoop/mapred/ mapred/org/apache/hadoop/mapred/tools/ test/org/apache/hadoop/map...

Author: omalley
Date: Fri Mar  4 04:25:43 2011
New Revision: 1077534

URL: http://svn.apache.org/viewvc?rev=1077534&view=rev
Log:
commit 1e8700bb5b428adde542fcc6ad8f2264158923bc
Author: Chris Douglas <cd...@apache.org>
Date:   Thu Jul 8 02:33:08 2010 -0700

    HADOOP:5913 from https://issues.apache.org/jira/secure/attachment/12448965/C5913-15y20s.patch

Added:
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/Queue.java
Modified:
    hadoop/common/branches/branch-0.20-security-patches/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/mapred-default.xml
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/AdminOperationsProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobQueueClient.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobQueueInfo.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/QueueManager.java
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/tools/MRAdmin.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestAdminOperationsProtocolWithServiceAuthorization.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueInformation.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManager.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManagerForJobKillAndJobPriority.java
    hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobqueue_details.jsp
    hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobtracker.jsp

Modified: hadoop/common/branches/branch-0.20-security-patches/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/contrib/capacity-scheduler/src/test/org/apache/hadoop/mapred/TestCapacityScheduler.java Fri Mar  4 04:25:43 2011
@@ -34,13 +34,13 @@ import junit.framework.TestCase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.mapred.JobStatusChangeEvent.EventType;
-import org.apache.hadoop.conf.Configuration;
-
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
 import org.apache.hadoop.mapreduce.split.JobSplit;
+import org.apache.hadoop.security.authorize.AccessControlList;
 
 public class TestCapacityScheduler extends TestCase {
 
@@ -402,15 +402,26 @@ public class TestCapacityScheduler exten
   }
   
   static class FakeQueueManager extends QueueManager {
-    private Set<String> queues = null;
+    private static final Map<String,AccessControlList> acls =
+      new HashMap<String,AccessControlList>() {
+        final AccessControlList allEnabledAcl = new AccessControlList("*");
+        @Override
+        public AccessControlList get(Object key) {
+          return allEnabledAcl;
+        }
+      };
     FakeQueueManager() {
       super(new Configuration());
     }
-    void setQueues(Set<String> queues) {
-      this.queues = queues;
-    }
-    public synchronized Set<String> getQueues() {
-      return queues;
+    void setQueues(Set<String> newQueues) {
+      queues.clear();
+      for (String qName : newQueues) {
+        try {
+          queues.put(qName, new Queue(qName, acls, Queue.QueueState.RUNNING));
+        } catch (Throwable t) {
+          throw new RuntimeException("Unable to initialize queue " + qName, t);
+        }
+      }
     }
   }
   

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/mapred-default.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/mapred-default.xml?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/mapred-default.xml (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/mapred-default.xml Fri Mar  4 04:25:43 2011
@@ -912,6 +912,16 @@
 </property>
 
 <property>
+  <name>mapred.queue.default.state</name>
+  <value>RUNNING</value>
+  <description>
+   This values defines the state , default queue is in.
+   the values can be either "STOPPED" or "RUNNING"
+   This value can be changed at runtime.
+  </description>
+</property>
+
+<property>
   <name>mapred.job.queue.name</name>
   <value>default</value>
   <description> Queue to which a job is submitted. This must match one of the

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/AdminOperationsProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/AdminOperationsProtocol.java?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/AdminOperationsProtocol.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/AdminOperationsProtocol.java Fri Mar  4 04:25:43 2011
@@ -34,13 +34,17 @@ public interface AdminOperationsProtocol
   /**
    * Version 1: Initial version. Added refreshQueueAcls.
    * Version 2: Added node refresh facility
+   * Version 3: Changed refreshQueueAcls to refreshQueues
    */
-  public static final long versionID = 2L;
+  public static final long versionID = 3L;
 
   /**
    * Refresh the queue acls in use currently.
+   * Refresh the queues used by the jobtracker and scheduler.
+   *
+   * Access control lists and queue states are refreshed.
    */
-  void refreshQueueAcls() throws IOException;
+  void refreshQueues() throws IOException;
   
   /**
    * Refresh the node list at the {@link JobTracker} 

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobQueueClient.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobQueueClient.java?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobQueueClient.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobQueueClient.java Fri Mar  4 04:25:43 2011
@@ -111,13 +111,14 @@ class JobQueueClient extends Configured 
    * @throws IOException
    */
 
-  private void displayQueueInfo(String queue, boolean showJobs) throws IOException {
-    JobQueueInfo schedInfo = jc.getQueueInfo(queue);
-    if (schedInfo == null) {
-      System.out.printf("Queue Name : %s has no scheduling information \n", queue);
+  private void displayQueueInfo(String queue, boolean showJobs)
+      throws IOException {
+    JobQueueInfo jobQueueInfo = jc.getQueueInfo(queue);
+    if (jobQueueInfo == null) {
+      System.out.println("Queue Name : " + queue +
+          " has no scheduling information");
     } else {
-      System.out.printf("Queue Name : %s \n", schedInfo.getQueueName());
-      System.out.printf("Scheduling Info : %s \n",schedInfo.getSchedulingInfo());
+      printJobQueueInfo(jobQueueInfo);
     }
     if (showJobs) {
       System.out.printf("Job List\n");
@@ -129,6 +130,19 @@ class JobQueueClient extends Configured 
   }
 
   /**
+   * format and print information about the passed in job queue.
+   */
+  private void printJobQueueInfo(JobQueueInfo jobQueueInfo) {
+    System.out.println("Queue Name : " + jobQueueInfo.getQueueName());
+    System.out.println("Queue State : " + jobQueueInfo.getQueueState());
+    String schedInfo = jobQueueInfo.getSchedulingInfo();
+    if (null == schedInfo || "".equals(schedInfo.trim())) {
+      schedInfo = JobQueueInfo.EMPTY_INFO;
+    }
+    System.out.println("Scheduling Info : " + schedInfo);
+  }
+
+  /**
    * Method used to display the list of the JobQueues registered
    * with the {@link QueueManager}
    * 
@@ -137,12 +151,7 @@ class JobQueueClient extends Configured 
   private void displayQueueList() throws IOException {
     JobQueueInfo[] queues = jc.getQueues();
     for (JobQueueInfo queue : queues) {
-      String schedInfo = queue.getSchedulingInfo();
-      if(schedInfo.trim().equals("")){
-        schedInfo = "N/A";
-      }
-      System.out.printf("Queue Name : %s \n", queue.getQueueName());
-      System.out.printf("Scheduling Info : %s \n",queue.getSchedulingInfo());
+      printJobQueueInfo(queue);
     }
   }
 

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobQueueInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobQueueInfo.java?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobQueueInfo.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobQueueInfo.java Fri Mar  4 04:25:43 2011
@@ -32,19 +32,19 @@ import org.apache.hadoop.io.Writable;
 
 public class JobQueueInfo implements Writable {
 
+  /**
+   * String used for empty (null) scheduling information.
+   */
+  static final String EMPTY_INFO = "N/A";
+
   private String queueName = "";
+  private String queueState = Queue.QueueState.RUNNING.getStateName();
   //The scheduling Information object is read back as String.
   //Once the scheduling information is set there is no way to recover it.
-  private String schedulingInfo; 
-  
-  
-  /**
-   * Default constructor for Job Queue Info.
-   * 
-   */
-  public JobQueueInfo() {
-    
-  }
+  private String schedulingInfo = EMPTY_INFO;
+
+  public JobQueueInfo() { }
+
   /**
    * Construct a new JobQueueInfo object using the queue name and the
    * scheduling information passed.
@@ -83,7 +83,9 @@ public class JobQueueInfo implements Wri
    * @param schedulingInfo
    */
   public void setSchedulingInfo(String schedulingInfo) {
-    this.schedulingInfo = schedulingInfo;
+    this.schedulingInfo = (schedulingInfo != null)
+      ? schedulingInfo
+      : EMPTY_INFO;
   }
 
   /**
@@ -93,26 +95,36 @@ public class JobQueueInfo implements Wri
    * @return Scheduling information associated to particular Job Queue
    */
   public String getSchedulingInfo() {
-    if(schedulingInfo != null) {
-      return schedulingInfo;
-    }else {
-      return "N/A";
-    }
+    return schedulingInfo;
   }
   
+  /**
+   * Set the state of the queue
+   * @param state state of the queue.
+   */
+  public void setQueueState(String state) {
+    queueState = state;
+  }
+
+  /**
+   * Return the queue state
+   * @return the queue state.
+   */
+  public String getQueueState() {
+    return queueState;
+  }
+
   @Override
   public void readFields(DataInput in) throws IOException {
     queueName = Text.readString(in);
+    queueState = Text.readString(in);
     schedulingInfo = Text.readString(in);
   }
 
   @Override
   public void write(DataOutput out) throws IOException {
     Text.writeString(out, queueName);
-    if(schedulingInfo!= null) {
-      Text.writeString(out, schedulingInfo);
-    }else {
-      Text.writeString(out, "N/A");
-    }
+    Text.writeString(out, queueState);
+    Text.writeString(out, schedulingInfo);
   }
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobSubmissionProtocol.java Fri Mar  4 04:25:43 2011
@@ -79,8 +79,9 @@ interface JobSubmissionProtocol extends 
    * Version 25: Added JobACLs to JobStatus as part of MAPREDUCE-1307
    * Version 26: Added the method getQueueAdmins(queueName) as part of
    *             MAPREDUCE-1664.
+   * Version 27: Added queue state to JobQueueInfo as part of HADOOP-5913.
    */
-  public static final long versionID = 26L;
+  public static final long versionID = 27L;
 
   /**
    * Allocate a name for the job.

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/JobTracker.java Fri Mar  4 04:25:43 2011
@@ -3677,7 +3677,10 @@ public class JobTracker implements MRCon
         throw new IOException("Queue \"" + queue + "\" does not exist");
       }
 
-      // check for access
+      // check if queue is RUNNING
+      if (!queueManager.isRunning(queue)) {
+        throw new IOException("Queue \"" + queue + "\" is not running");
+      }
       try {
         aclsManager.checkAccess(job, ugi, Operation.SUBMIT_JOB);
       } catch (IOException ioe) {
@@ -4874,10 +4877,10 @@ public class JobTracker implements MRCon
   }
 
   @Override
-  public void refreshQueueAcls() throws IOException{
-    LOG.info("Refreshing queue acls. requested by : " + 
+  public void refreshQueues() throws IOException {
+    LOG.info("Refreshing queue information. requested by : " +
         UserGroupInformation.getCurrentUser().getShortUserName());
-    this.queueManager.refreshAcls(new Configuration(this.conf));
+    this.queueManager.refreshQueues(new Configuration(this.conf));
   }
   
   synchronized String getReasonsForBlacklisting(String host) {

Added: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/Queue.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/Queue.java?rev=1077534&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/Queue.java (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/Queue.java Fri Mar  4 04:25:43 2011
@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * A class for storing the properties of a job queue.
+ */
+class Queue {
+
+  private static final Log LOG = LogFactory.getLog(Queue.class);
+
+  private String name;
+  private Map<String,AccessControlList> acls;
+  private QueueState state = QueueState.RUNNING;
+
+  /**
+   * An Object that can be used by schedulers to fill in
+   * arbitrary scheduling information. The toString method
+   * of these objects will be called by the framework to
+   * get a String that can be displayed on UI.
+   */
+  private Object schedulingInfo;
+
+  /**
+   * Enum representing queue state
+   */
+  static enum QueueState {
+    STOPPED("stopped"), RUNNING("running");
+
+    private final String stateName;
+
+    QueueState(String stateName) {
+      this.stateName = stateName;
+    }
+
+    public String getStateName() {
+      return stateName;
+    }
+  }
+
+  /**
+   * Create a job queue
+   * @param name name of the queue
+   * @param acls ACLs for the queue
+   * @param state state of the queue
+   */
+  Queue(String name, Map<String, AccessControlList> acls, QueueState state) {
+	  this.name = name;
+	  this.acls = acls;
+	  this.state = state;
+  }
+
+  /**
+   * Return the name of the queue
+   *
+   * @return name of the queue
+   */
+  String getName() {
+    return name;
+  }
+
+  /**
+   * Set the name of the queue
+   * @param name name of the queue
+   */
+  void setName(String name) {
+    assert name != null;
+    this.name = name;
+  }
+
+  /**
+   * Return the ACLs for the queue
+   *
+   * The keys in the map indicate the operations that can be performed,
+   * and the values indicate the list of users/groups who can perform
+   * the operation.
+   *
+   * @return Map containing the operations that can be performed and
+   *          who can perform the operations.
+   */
+  Map<String, AccessControlList> getAcls() {
+    return acls;
+  }
+
+  /**
+   * Set the ACLs for the queue
+   * @param acls Map containing the operations that can be performed and
+   *          who can perform the operations.
+   */
+  void setAcls(Map<String, AccessControlList> acls) {
+    assert acls != null;
+    this.acls = acls;
+  }
+
+  /**
+   * Return the state of the queue.
+   * @return state of the queue
+   */
+  QueueState getState() {
+    return state;
+  }
+
+  /**
+   * Set the state of the queue.
+   * @param state state of the queue.
+   */
+  void setState(QueueState state) {
+    assert state != null;
+    this.state = state;
+  }
+
+  /**
+   * Return the scheduling information for the queue
+   * @return scheduling information for the queue.
+   */
+  Object getSchedulingInfo() {
+    return schedulingInfo;
+  }
+
+  /**
+   * Set the scheduling information from the queue.
+   * @param schedulingInfo scheduling information for the queue.
+   */
+  void setSchedulingInfo(Object schedulingInfo) {
+    this.schedulingInfo = schedulingInfo;
+  }
+}

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/QueueManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/QueueManager.java?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/QueueManager.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/QueueManager.java Fri Mar  4 04:25:43 2011
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.Writer;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 import org.apache.commons.logging.Log;
@@ -50,22 +51,17 @@ class QueueManager {
   
   private static final Log LOG = LogFactory.getLog(QueueManager.class);
   
-  // Prefix in configuration for queue related keys
-  private static final String QUEUE_CONF_PROPERTY_NAME_PREFIX 
-                                                        = "mapred.queue.";
-  // Configured queues
-  private Set<String> queueNames;
-  // Map of a queue and ACL property name with an ACL
-  private HashMap<String, AccessControlList> aclsMap;
-  // Map of a queue name to any generic object that represents 
-  // scheduler information 
-  private HashMap<String, Object> schedulerInfoObjects;
-  // Whether ACLs are enabled in the system or not.
-  private boolean aclsEnabled;
-  
-  //Resource in which queue acls are configured.
+  static final String QUEUE_STATE_SUFFIX = "state";
+  /** Prefix in configuration for queue related keys */
+  static final String QUEUE_CONF_PROPERTY_NAME_PREFIX = "mapred.queue.";
+
+  // Continue to add this resource, to avoid incompatible change
   static final String QUEUE_ACLS_FILE_NAME = "mapred-queue-acls.xml";
-  
+
+  /** Whether ACLs are enabled in the system or not. */
+  private boolean aclsEnabled;
+  /** Map of a queue name and Queue object */
+  final HashMap<String,Queue> queues;
   /**
    * Enum representing an AccessControlList that drives set of operations that
    * can be performed on a queue.
@@ -100,10 +96,22 @@ class QueueManager {
    * @param conf Configuration object where queue configuration is specified.
    */
   public QueueManager(Configuration conf) {
-    queueNames = new TreeSet<String>();
-    aclsMap = new HashMap<String, AccessControlList>();
-    schedulerInfoObjects = new HashMap<String, Object>();
-    initialize(conf);
+    checkDeprecation(conf);
+    conf.addResource(QUEUE_ACLS_FILE_NAME);
+    queues = new HashMap<String,Queue>();
+    // First get the queue names
+    String[] queueNameValues = conf.getStrings("mapred.queue.names",
+        new String[]{JobConf.DEFAULT_QUEUE_NAME});
+    // Get configured ACLs and state for each queue
+    aclsEnabled = conf.getBoolean("mapred.acls.enabled", false);
+    for (String name : queueNameValues) {
+      try {
+        queues.put(name, new Queue(name, getQueueAcls(name, conf),
+              getQueueState(name, conf)));
+      } catch (Throwable t) {
+        LOG.warn("Not able to initialize queue " + name, t);
+      }
+    }
   }
   
   /**
@@ -116,7 +124,7 @@ class QueueManager {
    * @return Set of queue names.
    */
   public synchronized Set<String> getQueues() {
-    return queueNames;
+    return queues.keySet();
   }
   
   /**
@@ -133,27 +141,39 @@ class QueueManager {
    * 
    * @return true if the operation is allowed, false otherwise.
    */
-  public synchronized boolean hasAccess(String queueName,
-      QueueACL qACL, UserGroupInformation ugi) {
+  public synchronized boolean hasAccess(String queueName, QueueACL qACL,
+      UserGroupInformation ugi) {
     if (!aclsEnabled) {
       return true;
     }
-    
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("checking access for : " + toFullPropertyName(queueName, 
-                                            qACL.getAclName()));      
-    }
-    
-    AccessControlList acl = aclsMap.get(toFullPropertyName(
-        queueName, qACL.getAclName()));
-    if (acl == null) {
+    final Queue q = queues.get(queueName);
+    if (null == q) {
+      LOG.info("Queue " + queueName + " is not present");
       return false;
     }
     
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("checking access for : " +
+          toFullPropertyName(queueName, qACL.getAclName()));
+    }
+
+    AccessControlList acl =
+      q.getAcls().get(toFullPropertyName(queueName, qACL.getAclName()));
+
     // Check if user is part of the ACL
-    return acl.isUserAllowed(ugi);
+    return acl != null && acl.isUserAllowed(ugi);
   }
-  
+
+  /**
+   * Checks whether the given queue is running or not.
+   * @param queueName name of the queue
+   * @return true, if the queue is running.
+   */
+  synchronized boolean isRunning(String queueName) {
+    Queue q = queues.get(queueName);
+    return q != null && Queue.QueueState.RUNNING.equals(q.getState());
+  }
+
   /**
    * Set a generic Object that represents scheduling information relevant
    * to a queue.
@@ -167,7 +187,10 @@ class QueueManager {
    */
   public synchronized void setSchedulerInfo(String queueName, 
                                               Object queueInfo) {
-    schedulerInfoObjects.put(queueName, queueInfo);
+    Queue q = queues.get(queueName);
+    if (q != null) {
+      q.setSchedulingInfo(queueInfo);
+    }
   }
   
   /**
@@ -179,7 +202,10 @@ class QueueManager {
    * @see #setSchedulerInfo(String, Object)
    */
   public synchronized Object getSchedulerInfo(String queueName) {
-    return schedulerInfoObjects.get(queueName);
+    Queue q = queues.get(queueName);
+    return (q != null)
+      ? q.getSchedulingInfo()
+      : null;
   }
   
   /**
@@ -191,91 +217,98 @@ class QueueManager {
    * 
    * @throws IOException when queue ACL configuration file is invalid.
    */
-  synchronized void refreshAcls(Configuration conf) throws IOException {
+  synchronized void refreshQueues(Configuration conf) throws IOException {
+    // First check if things are configured in mapred-site.xml,
+    // so we can print out a deprecation warning.
+    // This check is needed only until we support the configuration
+    // in mapred-site.xml
+    checkDeprecation(conf);
+
+    // Add the queue configuration file. Values from mapred-site.xml
+    // will be overridden.
+    conf.addResource(QUEUE_ACLS_FILE_NAME);
+
+    // Now we refresh the properties of the queues. Note that we
+    // do *not* refresh the queue names or the acls flag. Instead
+    // we use the older values configured for them.
+    LOG.info("Refreshing acls and state for configured queues.");
     try {
-      HashMap<String, AccessControlList> newAclsMap = 
-        getQueueAcls(conf);
-      aclsMap = newAclsMap;
+      for (String qName : getQueues()) {
+        Queue q = queues.get(qName);
+        q.setAcls(getQueueAcls(qName, conf));
+        q.setState(getQueueState(qName, conf));
+      }
     } catch (Throwable t) {
-      String exceptionString = StringUtils.stringifyException(t);
-      LOG.warn("Queue ACLs could not be refreshed because there was an " +
-      		"exception in parsing the configuration: "+ exceptionString +
-      		". Existing ACLs are retained.");
-      throw new IOException(exceptionString);
+      LOG.warn("Invalid queue configuration", t);
+      throw new IOException("Invalid queue configuration", t);
     }
 
   }
-  
+
   private void checkDeprecation(Configuration conf) {
-    for(String queue: queueNames) {
-      for (QueueACL qACL : QueueACL.values()) {
-        String key = toFullPropertyName(queue, qACL.getAclName());
-        String aclString = conf.get(key);
-        if(aclString != null) {
-          LOG.warn("Configuring queue ACLs in mapred-site.xml or " +
-          		"hadoop-site.xml is deprecated. Configure queue ACLs in " + 
-          		QUEUE_ACLS_FILE_NAME);
-          return;
+    // check if queues are defined.
+    String[] queues = conf.getStrings("mapred.queue.names");
+    // check if acls are defined
+    if (queues != null) {
+      for (String queue : queues) {
+        for (QueueACL oper : QueueACL.values()) {
+          String aclString =
+            conf.get(toFullPropertyName(queue, oper.getAclName()));
+          if (aclString != null) {
+            LOG.warn("Configuring queue ACLs in mapred-site.xml or " +
+                "hadoop-site.xml is deprecated. Configure queue ACLs in " +
+                QUEUE_ACLS_FILE_NAME);
+            // even if one string is configured, it is enough for printing
+            // the warning. so we can return from here.
+            return;
+          }
         }
       }
     }
   }
-  
-  private HashMap<String, AccessControlList> getQueueAcls(Configuration conf)  {
-    checkDeprecation(conf);
-    conf.addResource(QUEUE_ACLS_FILE_NAME);
-    HashMap<String, AccessControlList> aclsMap = 
-      new HashMap<String, AccessControlList>();
-    for (String queue : queueNames) {
-      for (QueueACL qACL : QueueACL.values()) {
-        String key = toFullPropertyName(queue, qACL.getAclName());
-        String aclString = conf.get(key, " ");// default is empty list of users
-        aclsMap.put(key, new AccessControlList(aclString));
-      }
-    } 
-    return aclsMap;
+
+  /** Parse ACLs for the queue from the configuration. */
+  HashMap<String, AccessControlList> getQueueAcls(
+      String name, Configuration conf) {
+    HashMap<String,AccessControlList> map =
+      new HashMap<String,AccessControlList>();
+    for (QueueACL oper : QueueACL.values()) {
+      String aclKey = toFullPropertyName(name, oper.getAclName());
+      map.put(aclKey, new AccessControlList(conf.get(aclKey, "*")));
+    }
+    return map;
   }
-  
-  private void initialize(Configuration conf) {
-    aclsEnabled = conf.getBoolean(JobConf.MR_ACLS_ENABLED, false);
-    String[] queues = conf.getStrings("mapred.queue.names", 
-        new String[] {JobConf.DEFAULT_QUEUE_NAME});
-    addToSet(queueNames, queues);
-    aclsMap = getQueueAcls(conf);
+
+  /** Parse state of the queue from the configuration. */
+  Queue.QueueState getQueueState(String name, Configuration conf) {
+    return conf.getEnum(
+        toFullPropertyName(name, QueueManager.QUEUE_STATE_SUFFIX),
+        Queue.QueueState.RUNNING);
   }
-  
-  static final String toFullPropertyName(String queue, 
-      String property) {
+
+  static final String toFullPropertyName(String queue, String property) {
     return QUEUE_CONF_PROPERTY_NAME_PREFIX + queue + "." + property;
   }
   
-  private static final void addToSet(Set<String> set, String[] elems) {
-    for (String elem : elems) {
-      set.add(elem);
+  synchronized JobQueueInfo getJobQueueInfo(String queue) {
+    Queue q = queues.get(queue);
+    if (q != null) {
+      JobQueueInfo qInfo = new JobQueueInfo();
+      qInfo.setQueueName(q.getName());
+      qInfo.setQueueState(q.getState().getStateName());
+      Object schedInfo = q.getSchedulingInfo();
+      qInfo.setSchedulingInfo(schedInfo == null ? null : schedInfo.toString());
+      return qInfo;
     }
-  }
-  
-  synchronized JobQueueInfo[] getJobQueueInfos() {
-    ArrayList<JobQueueInfo> queueInfoList = new ArrayList<JobQueueInfo>();
-    for(String queue : queueNames) {
-      Object schedulerInfo = schedulerInfoObjects.get(queue);
-      if(schedulerInfo != null) {
-        queueInfoList.add(new JobQueueInfo(queue,schedulerInfo.toString()));
-      }else {
-        queueInfoList.add(new JobQueueInfo(queue,null));
-      }
-    }
-    return (JobQueueInfo[]) queueInfoList.toArray(new JobQueueInfo[queueInfoList
-        .size()]);
+    return null;
   }
 
-  JobQueueInfo getJobQueueInfo(String queue) {
-    Object schedulingInfo = schedulerInfoObjects.get(queue);
-    if(schedulingInfo!=null){
-      return new JobQueueInfo(queue,schedulingInfo.toString());
-    }else {
-      return new JobQueueInfo(queue,null);
+  synchronized JobQueueInfo[] getJobQueueInfos() {
+    ArrayList<JobQueueInfo> ret = new ArrayList<JobQueueInfo>();
+    for (String qName : getQueues()) {
+      ret.add(getJobQueueInfo(qName));
     }
+    return (JobQueueInfo[]) ret.toArray(new JobQueueInfo[ret.size()]);
   }
 
   /**
@@ -285,13 +318,12 @@ class QueueManager {
    * @return QueueAclsInfo[]
    * @throws java.io.IOException
    */
-  synchronized QueueAclsInfo[] getQueueAcls(UserGroupInformation
-          ugi) throws IOException {
+  synchronized QueueAclsInfo[] getQueueAcls(UserGroupInformation ugi)
+      throws IOException {
     //List of all QueueAclsInfo objects , this list is returned
-    ArrayList<QueueAclsInfo> queueAclsInfolist =
-            new ArrayList<QueueAclsInfo>();
+    ArrayList<QueueAclsInfo> queueAclsInfolist = new ArrayList<QueueAclsInfo>();
     QueueACL[] acls = QueueACL.values();
-    for (String queueName : queueNames) {
+    for (String queueName : getQueues()) {
       QueueAclsInfo queueAclsInfo = null;
       ArrayList<String> operationsAllowed = null;
       for (QueueACL qACL : acls) {
@@ -305,13 +337,13 @@ class QueueManager {
       if (operationsAllowed != null) {
         //There is atleast 1 operation supported for queue <queueName>
         //, hence initialize queueAclsInfo
-        queueAclsInfo = new QueueAclsInfo(queueName, operationsAllowed.toArray
-                (new String[operationsAllowed.size()]));
+        queueAclsInfo = new QueueAclsInfo(queueName, operationsAllowed.toArray(
+              new String[operationsAllowed.size()]));
         queueAclsInfolist.add(queueAclsInfo);
       }
     }
-    return queueAclsInfolist.toArray(new QueueAclsInfo[
-            queueAclsInfolist.size()]);
+    return
+      queueAclsInfolist.toArray(new QueueAclsInfo[queueAclsInfolist.size()]);
   }
 
   /**
@@ -321,11 +353,11 @@ class QueueManager {
    * If acls are disabled(mapred.acls.enabled set to false), returns ACL with
    * all users.
    */
-  synchronized AccessControlList getQueueACL(String queueName,
-      QueueACL qACL) {
+  synchronized AccessControlList getQueueACL(String queueName, QueueACL qACL) {
     if (aclsEnabled) {
-      return aclsMap.get(toFullPropertyName(
-          queueName, qACL.getAclName()));
+      Queue q = queues.get(queueName);
+      assert q != null;
+      return q.getAcls().get(toFullPropertyName(queueName, qACL.getAclName()));
     }
     return new AccessControlList("*");
   }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/tools/MRAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/tools/MRAdmin.java?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/tools/MRAdmin.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/tools/MRAdmin.java Fri Mar  4 04:25:43 2011
@@ -55,16 +55,15 @@ public class MRAdmin extends Configured 
   private static void printHelp(String cmd) {
     String summary = "hadoop mradmin is the command to execute Map-Reduce administrative commands.\n" +
     "The full syntax is: \n\n" +
-    "hadoop mradmin [-refreshServiceAcl] [-refreshQueueAcls] " +
+    "hadoop mradmin [-refreshServiceAcl] [-refreshQueues] " +
     "[-refreshNodes] [-refreshUserToGroupsMappings] " +
     "[-refreshSuperUserGroupsConfiguration] [-help [cmd]]\n";
 
   String refreshServiceAcl = "-refreshServiceAcl: Reload the service-level authorization policy file\n" +
     "\t\tJobtracker will reload the authorization policy file.\n";
 
-  String refreshQueueAcls =
-        "-refreshQueueAcls: Reload the queue acls\n"
-            + "\t\tJobTracker will reload the mapred-queue-acls.xml file.\n";
+  String refreshQueues = "-refreshQueues: Reload the queue acls and state\n" +
+    "\t\tJobTracker will reload the mapred-queues.xml file.\n";
 
   String refreshUserToGroupsMappings = 
     "-refreshUserToGroupsMappings: Refresh user-to-groups mappings\n";
@@ -80,8 +79,8 @@ public class MRAdmin extends Configured 
 
   if ("refreshServiceAcl".equals(cmd)) {
     System.out.println(refreshServiceAcl);
-  } else if ("refreshQueueAcls".equals(cmd)) {
-    System.out.println(refreshQueueAcls);
+  } else if ("refreshQueues".equals(cmd)) {
+    System.out.println(refreshQueues);
   } else if ("refreshUserToGroupsMappings".equals(cmd)) {
     System.out.println(refreshUserToGroupsMappings);
   } else if ("refreshSuperUserGroupsConfiguration".equals(cmd)) {
@@ -93,7 +92,7 @@ public class MRAdmin extends Configured 
   } else {
     System.out.println(summary);
     System.out.println(refreshServiceAcl);
-    System.out.println(refreshQueueAcls);
+    System.out.println(refreshQueues);
     System.out.println(refreshUserToGroupsMappings);
     System.out.println(refreshSuperUserGroupsConfiguration);
     System.out.println(refreshNodes);
@@ -111,8 +110,8 @@ public class MRAdmin extends Configured 
   private static void printUsage(String cmd) {
     if ("-refreshServiceAcl".equals(cmd)) {
       System.err.println("Usage: java MRAdmin" + " [-refreshServiceAcl]");
-    } else if ("-refreshQueueAcls".equals(cmd)) {
-      System.err.println("Usage: java MRAdmin" + " [-refreshQueueAcls]");
+    } else if ("-refreshQueues".equals(cmd)) {
+      System.err.println("Usage: java MRAdmin" + " [-refreshQueues]");
     } else if ("-refreshUserToGroupsMappings".equals(cmd)) {
       System.err.println("Usage: java MRAdmin" + " [-refreshUserToGroupsMappings]");
     } else if ("-refreshSuperUserGroupsConfiguration".equals(cmd)) {
@@ -123,7 +122,7 @@ public class MRAdmin extends Configured 
     } else {
       System.err.println("Usage: java MRAdmin");
       System.err.println("           [-refreshServiceAcl]");
-      System.err.println("           [-refreshQueueAcls]");
+      System.err.println("           [-refreshQueues]");
       System.err.println("           [-refreshUserToGroupsMappings]");
       System.err.println("           [-refreshSuperUserGroupsConfiguration]");
       System.err.println("           [-refreshNodes]");
@@ -165,7 +164,7 @@ public class MRAdmin extends Configured 
     return 0;
   }
 
-  private int refreshQueueAcls() throws IOException {
+  private int refreshQueues() throws IOException {
     // Get the current configuration
     Configuration conf = getConf();
     
@@ -179,7 +178,7 @@ public class MRAdmin extends Configured 
                                              AdminOperationsProtocol.class));
     
     // Refresh the queue properties
-    adminOperationsProtocol.refreshQueueAcls();
+    adminOperationsProtocol.refreshQueues();
     
     return 0;
   }
@@ -288,7 +287,7 @@ public class MRAdmin extends Configured 
     //
     // verify that we have enough command line parameters
     //
-    if ("-refreshServiceAcl".equals(cmd) || "-refreshQueueAcls".equals(cmd)
+    if ("-refreshServiceAcl".equals(cmd) || "-refreshQueues".equals(cmd)
         || "-refreshNodes".equals(cmd) ||
         "-refreshUserToGroupsMappings".equals(cmd) ||
         "-refreshSuperUserGroupsConfiguration".equals(cmd)
@@ -303,8 +302,8 @@ public class MRAdmin extends Configured 
     try {
       if ("-refreshServiceAcl".equals(cmd)) {
         exitCode = refreshAuthorizationPolicy();
-      } else if ("-refreshQueueAcls".equals(cmd)) {
-        exitCode = refreshQueueAcls();
+      } else if ("-refreshQueues".equals(cmd)) {
+        exitCode = refreshQueues();
       } else if ("-refreshUserToGroupsMappings".equals(cmd)) {
         exitCode = refreshUserToGroupsMappings();
       } else if ("-refreshSuperUserGroupsConfiguration".equals(cmd)) {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestAdminOperationsProtocolWithServiceAuthorization.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestAdminOperationsProtocolWithServiceAuthorization.java?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestAdminOperationsProtocolWithServiceAuthorization.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestAdminOperationsProtocolWithServiceAuthorization.java Fri Mar  4 04:25:43 2011
@@ -26,7 +26,7 @@ import junit.framework.TestCase;
 
 /**
  * Test case to check if {@link AdminOperationsProtocol#refreshNodes()} and 
- * {@link AdminOperationsProtocol#refreshQueueAcls()} works with service-level
+ * {@link AdminOperationsProtocol#refreshQueues()} works with service-level
  * authorization enabled i.e 'hadoop.security.authorization' set to true.
  */
 public class TestAdminOperationsProtocolWithServiceAuthorization 
@@ -46,7 +46,7 @@ extends TestCase {
 
       // Invoke MRAdmin commands
       MRAdmin mrAdmin = new MRAdmin(mr.createJobConf());
-      assertEquals(0, mrAdmin.run(new String[] { "-refreshQueueAcls" }));
+      assertEquals(0, mrAdmin.run(new String[] { "-refreshQueues" }));
       assertEquals(0, mrAdmin.run(new String[] { "-refreshNodes" }));
     } finally {
       if (mr != null) { 

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueInformation.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueInformation.java?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueInformation.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestJobQueueInformation.java Fri Mar  4 04:25:43 2011
@@ -109,6 +109,8 @@ public class TestJobQueueInformation ext
     assertNotNull(queueInfos);
     assertEquals(1, queueInfos.length);
     assertEquals("default", queueInfos[0].getQueueName());
+    assertEquals(Queue.QueueState.RUNNING.getStateName(),
+                  queueInfos[0].getQueueState());
     JobConf conf = mrCluster.createJobConf();
     FileSystem fileSys = dfsCluster.getFileSystem();
     

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManager.java?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManager.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManager.java Fri Mar  4 04:25:43 2011
@@ -23,6 +23,9 @@ import java.security.PrivilegedException
 import java.util.Set;
 import java.util.TreeSet;
 
+import java.io.File;
+import java.util.Properties;
+
 import javax.security.auth.login.LoginException;
 
 import junit.framework.TestCase;
@@ -183,8 +186,90 @@ public class TestQueueManager extends Te
       tearDownCluster();
     }
   }
-  
-  
+
+  /**
+   * Test to verify refreshing of queue properties by using MRAdmin tool.
+   *
+   * @throws Exception
+   */
+  public void testStateRefresh() throws Exception {
+    String queueConfigPath =
+        System.getProperty("test.build.extraconf", "build/test/extraconf");
+    File queueConfigFile =
+        new File(queueConfigPath, QueueManager.QUEUE_ACLS_FILE_NAME);
+    try {
+      //Setting up default mapred-site.xml
+      Properties queueConfProps = new Properties();
+      //these properties should be retained.
+      queueConfProps.put("mapred.queue.names", "default,qu1");
+      queueConfProps.put("mapred.acls.enabled", "true");
+      //These property should always be overridden
+      queueConfProps.put("mapred.queue.default.state", "RUNNING");
+      queueConfProps.put("mapred.queue.qu1.state", "STOPPED");
+      UtilsForTests.setUpConfigFile(queueConfProps, queueConfigFile);
+
+      //Create a new configuration to be used with QueueManager
+      JobConf conf = new JobConf();
+      setUpCluster(conf);
+      QueueManager queueManager =
+        miniMRCluster.getJobTrackerRunner().getJobTracker().getQueueManager();
+
+      RunningJob job = submitSleepJob(1, 1, 100, 100, true,null, "default" );
+      assertTrue(job.isSuccessful());
+
+      try {
+        submitSleepJob(1, 1, 100, 100, true,null, "qu1" );
+        fail("submit job in default queue should be failed ");
+      } catch (Exception e) {
+        assertTrue(e.getMessage().contains(
+              "Queue \"" + "qu1" + "\" is not running"));
+      }
+
+      // verify state of queues before refresh
+      JobQueueInfo queueInfo = queueManager.getJobQueueInfo("default");
+      assertEquals(Queue.QueueState.RUNNING.getStateName(),
+                    queueInfo.getQueueState());
+      queueInfo = queueManager.getJobQueueInfo("qu1");
+      assertEquals(Queue.QueueState.STOPPED.getStateName(),
+                    queueInfo.getQueueState());
+
+      queueConfProps.put("mapred.queue.default.state", "STOPPED");
+      queueConfProps.put("mapred.queue.qu1.state", "RUNNING");
+      UtilsForTests.setUpConfigFile(queueConfProps, queueConfigFile);
+
+      //refresh configuration
+      queueManager.refreshQueues(conf);
+
+      //Job Submission should pass now because ugi to be used is set to blank.
+      try {
+        submitSleepJob(1, 1, 100, 100, true,null,"qu1");
+      } catch (Exception e) {
+        fail("submit job in qu1 queue should be sucessful ");
+      }
+
+      try {
+        submitSleepJob(1, 1, 100, 100, true,null, "default" );
+        fail("submit job in default queue should be failed ");
+      } catch (Exception e){
+        assertTrue(e.getMessage().contains(
+              "Queue \"" + "default" + "\" is not running"));
+      }
+
+      // verify state of queues after refresh
+      queueInfo = queueManager.getJobQueueInfo("default");
+      assertEquals(Queue.QueueState.STOPPED.getStateName(),
+                    queueInfo.getQueueState());
+      queueInfo = queueManager.getJobQueueInfo("qu1");
+      assertEquals(Queue.QueueState.RUNNING.getStateName(),
+                    queueInfo.getQueueState());
+    } finally{
+      if(queueConfigFile.exists()) {
+        queueConfigFile.delete();
+      }
+      this.tearDownCluster();
+    }
+  }
+
   JobConf setupConf(String aclName, String aclValue) {
     JobConf conf = new JobConf();
     conf.setBoolean(JobConf.MR_ACLS_ENABLED, true);

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManagerForJobKillAndJobPriority.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManagerForJobKillAndJobPriority.java?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManagerForJobKillAndJobPriority.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/TestQueueManagerForJobKillAndJobPriority.java Fri Mar  4 04:25:43 2011
@@ -196,7 +196,7 @@ public class TestQueueManagerForJobKillA
         //write out queue-acls.xml.
         UtilsForTests.setUpConfigFile(queueConfProps, queueConfigFile);
         //refresh configuration
-        queueManager.refreshAcls(conf);
+        queueManager.refreshQueues(conf);
         //Submission should succeed
         assertTrue("User Job Submission failed after refresh.",
                    queueManager.hasAccess("default", QueueACL.SUBMIT_JOB, ugi));
@@ -214,7 +214,7 @@ public class TestQueueManagerForJobKillA
         hadoopConfProps.put(QueueManager.toFullPropertyName
                             ("q1", submitAcl), ugi.getShortUserName());
         UtilsForTests.setUpConfigFile(hadoopConfProps, hadoopConfigFile);
-        queueManager.refreshAcls(conf);
+        queueManager.refreshQueues(conf);
         assertTrue("User Job Submission allowed after refresh and no queue acls file.",
                    queueManager.hasAccess("q1", QueueACL.SUBMIT_JOB, ugi));
       } finally{
@@ -235,7 +235,7 @@ public class TestQueueManagerForJobKillA
       String queueConfigPath =
         System.getProperty("test.build.extraconf", "build/test/extraconf");
       File queueConfigFile =
-        new File(queueConfigPath, QueueManager.QUEUE_ACLS_FILE_NAME);
+        new File(queueConfigPath, QueueManager.QUEUE_ACLS_FILE_NAME );
       File hadoopConfigFile = new File(queueConfigPath, "hadoop-site.xml");
       try {
         // queue properties with which the cluster is started.
@@ -274,7 +274,7 @@ public class TestQueueManagerForJobKillA
         try {
           //Exception to be thrown by queue manager because configuration passed
           //is invalid.
-          queueManager.refreshAcls(conf);
+          queueManager.refreshQueues(conf);
           fail("Refresh of ACLs should have failed with invalid conf file.");
         } catch (Exception e) {
         }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobqueue_details.jsp
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobqueue_details.jsp?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobqueue_details.jsp (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobqueue_details.jsp Fri Mar  4 04:25:43 2011
@@ -38,6 +38,7 @@ private static final long serialVersionU
   <a href="jobtracker.jsp"><%=trackerName%></a>
 </h1>
 <div>
+State : <%= schedInfo.getQueueState() %> <br/>
 Scheduling Information :
 <%= HtmlQuoting.quoteHtmlChars(schedulingInfoString).replaceAll("\n","<br/>") %>
 </div>

Modified: hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobtracker.jsp
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobtracker.jsp?rev=1077534&r1=1077533&r2=1077534&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobtracker.jsp (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/webapps/job/jobtracker.jsp Fri Mar  4 04:25:43 2011
@@ -115,6 +115,7 @@
 <thead style="font-weight: bold">
 <tr>
 <td> Queue Name </td>
+<td> State </td>
 <td> Scheduling Information</td>
 </tr>
 </thead>
@@ -122,6 +123,7 @@
 <%
 for(JobQueueInfo queue: queues) {
   String queueName = queue.getQueueName();
+  String state = queue.getQueueState();
   String schedulingInformation = queue.getSchedulingInfo();
   if(schedulingInformation == null || schedulingInformation.trim().equals("")) {
     schedulingInformation = "NA";
@@ -129,6 +131,7 @@ for(JobQueueInfo queue: queues) {
 %>
 <tr>
 <td><a href="jobqueue_details.jsp?queueName=<%=queueName%>"><%=queueName%></a></td>
+<td><%=state%></td>
 <td><%=HtmlQuoting.quoteHtmlChars(schedulingInformation).replaceAll("\n","<br/>") %>
 </td>
 </tr>