浏览代码

YARN-321. Forwarding YARN-321 branch to latest trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/YARN-321@1558246 13f79535-47bb-0310-9956-ffa450edef68
Vinod Kumar Vavilapalli 11 年之前
父节点
当前提交
b2af4b232a
共有 17 个文件被更改,包括 353 次插入24 次删除
  1. 15 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
  3. 16 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  4. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  5. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
  6. 13 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  7. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  8. 50 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
  9. 10 0
      hadoop-mapreduce-project/CHANGES.txt
  10. 18 0
      hadoop-yarn-project/CHANGES.txt
  11. 4 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
  12. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
  13. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
  14. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java
  15. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
  16. 109 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
  17. 101 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.java

+ 15 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -430,6 +430,9 @@ Release 2.4.0 - UNRELEASED
     HADOOP-10173. Remove UGI from DIGEST-MD5 SASL server creation (daryn via
     kihwal)
 
+    HADOOP-10228. FsPermission#fromShort() should cache FsAction.values().
+    (Haohui Mai via cnauroth)
+
   BUG FIXES
 
     HADOOP-9964. Fix deadlocks in TestHttpServer by synchronize
@@ -1329,6 +1332,18 @@ Release 2.1.0-beta - 2013-08-22
 
     HADOOP-9701. mvn site ambiguous links in hadoop-common. (kkambatl via tucu)
 
+Release 2.0.6-alpha - 08/22/2013
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
 Release 2.0.5-alpha - 06/06/2013
 
   INCOMPATIBLE CHANGES

+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -120,8 +120,7 @@ public class FsPermission implements Writable {
   }
 
   public void fromShort(short n) {
-    FsAction[] v = FsAction.values();
-
+    FsAction[] v = FSACTION_VALUES;
     set(v[(n >>> 6) & 7], v[(n >>> 3) & 7], v[n & 7], (((n >>> 9) & 1) == 1) );
   }
 
@@ -210,6 +209,8 @@ public class FsPermission implements Writable {
   public static final int DEFAULT_UMASK = 
                   CommonConfigurationKeys.FS_PERMISSIONS_UMASK_DEFAULT;
 
+  private static final FsAction[] FSACTION_VALUES = FsAction.values();
+
   /** 
    * Get the user file creation mask (umask)
    * 

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -749,6 +749,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-5710. FSDirectory#getFullPathName should check inodes against null.
     (Uma Maheswara Rao G via jing9)
 
+    HDFS-5579. Under construction files make DataNode decommission take very long
+    hours. (zhaoyunjiong via jing9)
+
   BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
 
     HDFS-4985. Add storage type to the protocol and expose it in block report
@@ -921,6 +924,9 @@ Release 2.3.0 - UNRELEASED
     HDFS-5675. Add Mkdirs operation to NNThroughputBenchmark.
     (Plamen Jeliazkov via shv)
 
+    HDFS-5677. Need error checking for HA cluster configuration.
+    (Vincent Sheffer via cos)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -2301,6 +2307,16 @@ Release 2.1.0-beta - 2013-08-22
     HDFS-4982. JournalNode should relogin from keytab before fetching logs
     from other JNs (todd)
 
+Release 2.0.6-alpha - 08/22/2013
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
 Release 2.0.5-alpha - 06/06/2013
 
   INCOMPATIBLE CHANGES

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -585,6 +585,12 @@ public class DFSUtil {
       String address = getConfValue(defaultValue, suffix, conf, keys);
       if (address != null) {
         InetSocketAddress isa = NetUtils.createSocketAddr(address);
+        if (isa.isUnresolved()) {
+          LOG.warn("Namenode for " + nsId +
+                   " remains unresolved for ID " + nnId +
+                   ".  Check your hdfs-site.xml file to " +
+                   "ensure namenodes are configured properly.");
+        }
         ret.put(nnId, isa);
       }
     }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java

@@ -31,7 +31,7 @@ public interface BlockCollection {
   /**
    * Get the last block of the collection.
    */
-  public BlockInfo getLastBlock() throws IOException;
+  public BlockInfo getLastBlock();
 
   /** 
    * Get content summary.

+ 13 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -1214,8 +1214,10 @@ public class BlockManager {
             // block should belong to a file
             bc = blocksMap.getBlockCollection(block);
             // abandoned block or block reopened for append
-            if(bc == null || bc.isUnderConstruction()) {
-              neededReplications.remove(block, priority); // remove from neededReplications
+            if (bc == null
+                || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
+              // remove from neededReplications
+              neededReplications.remove(block, priority);
               continue;
             }
 
@@ -1295,7 +1297,7 @@ public class BlockManager {
           // block should belong to a file
           bc = blocksMap.getBlockCollection(block);
           // abandoned block or block reopened for append
-          if(bc == null || bc.isUnderConstruction()) {
+          if(bc == null || (bc.isUnderConstruction() && block.equals(bc.getLastBlock()))) {
             neededReplications.remove(block, priority); // remove from neededReplications
             rw.targets = null;
             continue;
@@ -2906,8 +2908,16 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
         NumberReplicas num = countNodes(block);
         int curReplicas = num.liveReplicas();
         int curExpectedReplicas = getReplication(block);
+                
         if (isNeededReplication(block, curExpectedReplicas, curReplicas)) {
           if (curExpectedReplicas > curReplicas) {
+            if (bc.isUnderConstruction()) {
+              if (block.equals(bc.getLastBlock()) && curReplicas > minReplication) {
+                continue;
+              }
+              underReplicatedInOpenFiles++;
+            }
+            
             // Log info about one block for this node which needs replication
             if (!status) {
               status = true;
@@ -2924,9 +2934,6 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
             if ((curReplicas == 0) && (num.decommissionedReplicas() > 0)) {
               decommissionOnlyReplicas++;
             }
-            if (bc.isUnderConstruction()) {
-              underReplicatedInOpenFiles++;
-            }
           }
           if (!neededReplications.contains(block) &&
             pendingReplications.getNumReplicas(block) == 0) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -640,7 +640,7 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override
-  public BlockInfo getLastBlock() throws IOException {
+  public BlockInfo getLastBlock() {
     return blocks == null || blocks.length == 0? null: blocks[blocks.length-1];
   }
 

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -779,4 +780,53 @@ public class TestDecommission {
       Thread.sleep(HEARTBEAT_INTERVAL * 1000);
     }
   }
+  
+  @Test(timeout=120000)
+  public void testDecommissionWithOpenfile() throws IOException, InterruptedException {
+    LOG.info("Starting test testDecommissionWithOpenfile");
+    
+    //At most 4 nodes will be decommissioned
+    startCluster(1, 7, conf);
+        
+    FileSystem fileSys = cluster.getFileSystem(0);
+    FSNamesystem ns = cluster.getNamesystem(0);
+    
+    String openFile = "/testDecommissionWithOpenfile.dat";
+           
+    writeFile(fileSys, new Path(openFile), (short)3);   
+    // make sure the file was open for write
+    FSDataOutputStream fdos =  fileSys.append(new Path(openFile)); 
+    
+    LocatedBlocks lbs = NameNodeAdapter.getBlockLocations(cluster.getNameNode(0), openFile, 0, fileSize);
+              
+    DatanodeInfo[] dnInfos4LastBlock = lbs.getLastLocatedBlock().getLocations();
+    DatanodeInfo[] dnInfos4FirstBlock = lbs.get(0).getLocations();
+    
+    ArrayList<String> nodes = new ArrayList<String>();
+    ArrayList<DatanodeInfo> dnInfos = new ArrayList<DatanodeInfo>();
+   
+    for (DatanodeInfo datanodeInfo : dnInfos4FirstBlock) {
+      DatanodeInfo found = datanodeInfo;
+      for (DatanodeInfo dif: dnInfos4LastBlock) {
+        if (datanodeInfo.equals(dif)) {
+         found = null;         
+        }
+      }
+      if (found != null) {
+        nodes.add(found.getXferAddr());
+        dnInfos.add(found);
+      }
+    }
+    //decommission one of the 3 nodes which have last block
+    nodes.add(dnInfos4LastBlock[0].getXferAddr());
+    dnInfos.add(dnInfos4LastBlock[0]);
+    
+    writeConfigFile(excludeFile, nodes);
+    refreshNodes(ns, conf);  
+    for (DatanodeInfo dn : dnInfos) {
+      waitNodeState(dn, AdminStates.DECOMMISSIONED);
+    }           
+
+    fdos.close();
+  }
 }

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

@@ -998,6 +998,16 @@ Release 2.1.0-beta - 2013-08-22
     MAPREDUCE-4374. Fix child task environment variable config and add support
     for Windows. (Chuan Liu via cnauroth)
 
+Release 2.0.6-alpha - 08/22/2013
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
 Release 2.0.5-alpha - 06/06/2013
 
   INCOMPATIBLE CHANGES

+ 18 - 0
hadoop-yarn-project/CHANGES.txt

@@ -307,6 +307,9 @@ Release 2.4.0 - UNRELEASED
     dependencies and thus compact the dependency list for leaf modules.
     (Alejandro Abdelnur via vinodkv)
 
+    YARN-1567. In Fair Scheduler, allow empty queues to change between leaf and
+    parent on allocation file reload (Sandy Ryza)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -1619,6 +1622,21 @@ Release 2.1.0-beta - 2013-08-22
     yarn.resourcemanager.connect.{max.wait.secs|retry_interval.secs}
     (Karthik Kambatla via acmurthy)
 
+Release 2.0.6-alpha - 08/22/2013
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    YARN-854. Fixing YARN bugs that are failing applications in secure
+    environment. (Omkar Vinit Joshi and shv)
+
 Release 2.0.5-alpha - 06/06/2013
 
   INCOMPATIBLE CHANGES

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java

@@ -509,4 +509,8 @@ public class QueueMetrics implements MetricsSource {
   public int getActiveApps() {
     return activeApplications.value();
   }
+  
+  public MetricsSystem getMetricsSystem() {
+    return metricsSystem;
+  }
 }

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java

@@ -76,7 +76,8 @@ public class AllocationConfiguration {
   @VisibleForTesting
   QueuePlacementPolicy placementPolicy;
   
-  private final Set<String> queueNames;
+  @VisibleForTesting
+  Set<String> queueNames;
   
   public AllocationConfiguration(Map<String, Resource> minQueueResources, 
       Map<String, Resource> maxQueueResources, 

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java

@@ -214,7 +214,7 @@ public class FSLeafQueue extends FSQueue {
   }
 
   @Override
-  public Collection<FSQueue> getChildQueues() {
+  public List<FSQueue> getChildQueues() {
     return new ArrayList<FSQueue>(1);
   }
   

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSParentQueue.java

@@ -157,7 +157,7 @@ public class FSParentQueue extends FSQueue {
   }
 
   @Override
-  public Collection<FSQueue> getChildQueues() {
+  public List<FSQueue> getChildQueues() {
     return childQueues;
   }
 

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -158,7 +159,7 @@ public abstract class FSQueue extends Schedulable implements Queue {
   /**
    * Gets the children of this queue, if any.
    */
-  public abstract Collection<FSQueue> getChildQueues();
+  public abstract List<FSQueue> getChildQueues();
   
   /**
    * Adds all applications in the queue and its subqueues to the given collection.

+ 109 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java

@@ -85,9 +85,7 @@ public class QueueManager {
    * could be referred to as just "parent1.queue2".
    */
   public FSLeafQueue getLeafQueue(String name, boolean create) {
-    if (!name.startsWith(ROOT_QUEUE + ".")) {
-      name = ROOT_QUEUE + "." + name;
-    }
+    name = ensureRootPrefix(name);
     synchronized (queues) {
       FSQueue queue = queues.get(name);
       if (queue == null && create) {
@@ -174,13 +172,107 @@ public class QueueManager {
     return leafQueue;
   }
 
+  /**
+   * Make way for the given leaf queue if possible, by removing incompatible
+   * queues with no apps in them. Incompatibility could be due to
+   * (1) leafToCreate being currently being a parent, or (2) an existing leaf queue in
+   * the ancestry of leafToCreate.
+   * 
+   * We will never remove the root queue or the default queue in this way.
+   *
+   * @return true if we can create leafToCreate or it already exists.
+   */
+  private boolean removeEmptyIncompatibleQueues(String leafToCreate) {
+    leafToCreate = ensureRootPrefix(leafToCreate);
+
+    // Ensure leafToCreate is not root and doesn't have the default queue in its
+    // ancestry.
+    if (leafToCreate.equals(ROOT_QUEUE) ||
+        leafToCreate.startsWith(
+            ROOT_QUEUE + "." + YarnConfiguration.DEFAULT_QUEUE_NAME + ".")) {
+      return false;
+    }
+
+    FSQueue queue = queues.get(leafToCreate);
+    // Queue exists already.
+    if (queue != null) {
+      if (queue instanceof FSLeafQueue) {
+        // If it's an already existing leaf, we're ok.
+        return true;
+      } else {
+        // If it's an existing parent queue, remove it if it's empty.
+        return removeQueueIfEmpty(queue);
+      }
+    }
+
+    // Queue doesn't exist already. Check if the new queue would be created
+    // under an existing leaf queue. If so, try removing that leaf queue.
+    int sepIndex = leafToCreate.length();
+    sepIndex = leafToCreate.lastIndexOf('.', sepIndex-1);
+    while (sepIndex != -1) {
+      String prefixString = leafToCreate.substring(0, sepIndex);
+      FSQueue prefixQueue = queues.get(prefixString);
+      if (prefixQueue != null && prefixQueue instanceof FSLeafQueue) {
+        return removeQueueIfEmpty(prefixQueue);
+      }
+      sepIndex = leafToCreate.lastIndexOf('.', sepIndex-1);
+    }
+    return true;
+  }
+
+  /**
+   * Remove the queue if it and its descendents are all empty.
+   * @param queue
+   * @return true if removed, false otherwise
+   */
+  private boolean removeQueueIfEmpty(FSQueue queue) {
+    if (isEmpty(queue)) {
+      removeQueue(queue);
+      return true;
+    }
+    return false;
+  }
+  
+  /**
+   * Remove a queue and all its descendents.
+   */
+  private void removeQueue(FSQueue queue) {
+    if (queue instanceof FSLeafQueue) {
+      leafQueues.remove(queue);
+    } else {
+      List<FSQueue> childQueues = queue.getChildQueues();
+      while (!childQueues.isEmpty()) {
+        removeQueue(childQueues.get(0));
+      }
+    }
+    queues.remove(queue.getName());
+    queue.getParent().getChildQueues().remove(queue);
+  }
+  
+  /**
+   * Returns true if there are no applications, running or not, in the given
+   * queue or any of its descendents.
+   */
+  protected boolean isEmpty(FSQueue queue) {
+    if (queue instanceof FSLeafQueue) {
+      FSLeafQueue leafQueue = (FSLeafQueue)queue;
+      return queue.getNumRunnableApps() == 0 &&
+          leafQueue.getNonRunnableAppSchedulables().isEmpty();
+    } else {
+      for (FSQueue child : queue.getChildQueues()) {
+        if (!isEmpty(child)) {
+          return false;
+        }
+      }
+      return true;
+    }
+  }
+
   /**
    * Gets a queue by name.
    */
   public FSQueue getQueue(String name) {
-    if (!name.startsWith(ROOT_QUEUE + ".") && !name.equals(ROOT_QUEUE)) {
-      name = ROOT_QUEUE + "." + name;
-    }
+    name = ensureRootPrefix(name);
     synchronized (queues) {
       return queues.get(name);
     }
@@ -190,9 +282,7 @@ public class QueueManager {
    * Return whether a queue exists already.
    */
   public boolean exists(String name) {
-    if (!name.startsWith(ROOT_QUEUE + ".") && !name.equals(ROOT_QUEUE)) {
-      name = ROOT_QUEUE + "." + name;
-    }
+    name = ensureRootPrefix(name);
     synchronized (queues) {
       return queues.containsKey(name);
     }
@@ -214,10 +304,19 @@ public class QueueManager {
     return queues.values();
   }
   
+  private String ensureRootPrefix(String name) {
+    if (!name.startsWith(ROOT_QUEUE + ".") && !name.equals(ROOT_QUEUE)) {
+      name = ROOT_QUEUE + "." + name;
+    }
+    return name;
+  }
+  
   public void updateAllocationConfiguration(AllocationConfiguration queueConf) {
     // Make sure all queues exist
     for (String name : queueConf.getQueueNames()) {
-      getLeafQueue(name, true);
+      if (removeEmptyIncompatibleQueues(name)) {
+        getLeafQueue(name, true);
+      }
     }
     
     for (FSQueue queue : queues.values()) {

+ 101 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.java

@@ -0,0 +1,101 @@
+/**
+* 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.yarn.server.resourcemanager.scheduler.fair;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Mockito.*;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Sets;
+
+public class TestQueueManager {
+  private FairSchedulerConfiguration conf;
+  private QueueManager queueManager;
+  private Set<FSQueue> notEmptyQueues;
+  
+  @Before
+  public void setUp() throws Exception {
+    conf = new FairSchedulerConfiguration();
+    FairScheduler scheduler = mock(FairScheduler.class);
+    AllocationConfiguration allocConf = new AllocationConfiguration(conf);
+    when(scheduler.getAllocationConfiguration()).thenReturn(allocConf);
+    when(scheduler.getConf()).thenReturn(conf);
+    SystemClock clock = new SystemClock();
+    when(scheduler.getClock()).thenReturn(clock);
+    notEmptyQueues = new HashSet<FSQueue>();
+    queueManager = new QueueManager(scheduler) {
+      @Override
+      public boolean isEmpty(FSQueue queue) {
+        return !notEmptyQueues.contains(queue);
+      }
+    };
+    FSQueueMetrics.forQueue("root", null, true, conf);
+    queueManager.initialize(conf);
+  }
+  
+  @Test
+  public void testReloadTurnsLeafQueueIntoParent() throws Exception {
+    updateConfiguredQueues(queueManager, "queue1");
+    
+    // When no apps are running in the leaf queue, should be fine turning it
+    // into a parent.
+    updateConfiguredQueues(queueManager, "queue1.queue2");
+    assertNull(queueManager.getLeafQueue("queue1", false));
+    assertNotNull(queueManager.getLeafQueue("queue1.queue2", false));
+    
+    // When leaf queues are empty, should be ok deleting them and
+    // turning parent into a leaf.
+    updateConfiguredQueues(queueManager, "queue1");
+    assertNull(queueManager.getLeafQueue("queue1.queue2", false));
+    assertNotNull(queueManager.getLeafQueue("queue1", false));
+    
+    // When apps exist in leaf queue, we shouldn't be able to create
+    // children under it, but things should work otherwise.
+    notEmptyQueues.add(queueManager.getLeafQueue("queue1", false));
+    updateConfiguredQueues(queueManager, "queue1.queue2");
+    assertNull(queueManager.getLeafQueue("queue1.queue2", false));
+    assertNotNull(queueManager.getLeafQueue("queue1", false));
+    
+    // When apps exist in leaf queues under a parent queue, shouldn't be
+    // able to turn it into a leaf queue, but things should work otherwise.
+    notEmptyQueues.clear();
+    updateConfiguredQueues(queueManager, "queue1.queue2");
+    notEmptyQueues.add(queueManager.getQueue("root.queue1"));
+    updateConfiguredQueues(queueManager, "queue1");
+    assertNotNull(queueManager.getLeafQueue("queue1.queue2", false));
+    assertNull(queueManager.getLeafQueue("queue1", false));
+    
+    // Should never to be able to create a queue under the default queue
+    updateConfiguredQueues(queueManager, "default.queue3");
+    assertNull(queueManager.getLeafQueue("default.queue3", false));
+    assertNotNull(queueManager.getLeafQueue("default", false));
+  }
+  
+  private void updateConfiguredQueues(QueueManager queueMgr, String... confQueues) {
+    AllocationConfiguration allocConf = new AllocationConfiguration(conf);
+    allocConf.queueNames = Sets.newHashSet(confQueues);
+    queueMgr.updateAllocationConfiguration(allocConf);
+  }
+}