瀏覽代碼

YARN-2250. FairScheduler.findLowestCommonAncestorQueue returns null when queues not identical (Krisztian Horvath via Sandy Ryza)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1607872 13f79535-47bb-0310-9956-ffa450edef68
Sanford Ryza 11 年之前
父節點
當前提交
5644f529f3

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

@@ -349,6 +349,9 @@ Release 2.5.0 - UNRELEASED
     YARN-2232. Fixed ResourceManager to allow DelegationToken owners to be able
     to cancel their own tokens in secure mode. (Varun Vasudev via vinodkv)
 
+    YARN-2250. FairScheduler.findLowestCommonAncestorQueue returns null when
+    queues not identical (Krisztian Horvath via Sandy Ryza)
+
 Release 2.4.1 - 2014-06-23 
 
   INCOMPATIBLE CHANGES

+ 5 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
 
-import com.google.common.base.Preconditions;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -93,6 +92,7 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
 /**
  * A scheduler that schedules resources between a set of queues. The scheduler
@@ -1468,8 +1468,9 @@ public class FairScheduler extends
       maxRunningEnforcer.updateRunnabilityOnAppRemoval(attempt, oldQueue);
     }
   }
-  
-  private FSQueue findLowestCommonAncestorQueue(FSQueue queue1, FSQueue queue2) {
+
+  @VisibleForTesting
+  FSQueue findLowestCommonAncestorQueue(FSQueue queue1, FSQueue queue2) {
     // Because queue names include ancestors, separated by periods, we can find
     // the lowest common ancestors by going from the start of the names until
     // there's a character that doesn't match.
@@ -1481,7 +1482,7 @@ public class FairScheduler extends
     for (int i = 0; i < Math.max(name1.length(), name2.length()); i++) {
       if (name1.length() <= i || name2.length() <= i ||
           name1.charAt(i) != name2.charAt(i)) {
-        return queueMgr.getQueue(name1.substring(lastPeriodIndex));
+        return queueMgr.getQueue(name1.substring(0, lastPeriodIndex));
       } else if (name1.charAt(i) == '.') {
         lastPeriodIndex = i;
       }

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

@@ -25,6 +25,8 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.io.File;
 import java.io.FileWriter;
@@ -3077,4 +3079,71 @@ public class TestFairScheduler extends FairSchedulerTestBase {
         createSchedulingRequest(1024, 1, "queue1", "user1", 3);
     scheduler.moveApplication(appAttId.getApplicationId(), "queue2");
   }
+
+  @Test
+  public void testLowestCommonAncestorForNonRootParent() throws Exception {
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    FSLeafQueue aQueue = mock(FSLeafQueue.class);
+    FSLeafQueue bQueue = mock(FSLeafQueue.class);
+    when(aQueue.getName()).thenReturn("root.queue1.a");
+    when(bQueue.getName()).thenReturn("root.queue1.b");
+
+    QueueManager queueManager = scheduler.getQueueManager();
+    FSParentQueue queue1 = queueManager.getParentQueue("queue1", true);
+    queue1.addChildQueue(aQueue);
+    queue1.addChildQueue(bQueue);
+
+    FSQueue ancestorQueue =
+        scheduler.findLowestCommonAncestorQueue(aQueue, bQueue);
+    assertEquals(ancestorQueue, queue1);
+  }
+
+  @Test
+  public void testLowestCommonAncestorRootParent() throws Exception {
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    FSLeafQueue aQueue = mock(FSLeafQueue.class);
+    FSLeafQueue bQueue = mock(FSLeafQueue.class);
+    when(aQueue.getName()).thenReturn("root.a");
+    when(bQueue.getName()).thenReturn("root.b");
+
+    QueueManager queueManager = scheduler.getQueueManager();
+    FSParentQueue queue1 = queueManager.getParentQueue("root", false);
+    queue1.addChildQueue(aQueue);
+    queue1.addChildQueue(bQueue);
+
+    FSQueue ancestorQueue =
+        scheduler.findLowestCommonAncestorQueue(aQueue, bQueue);
+    assertEquals(ancestorQueue, queue1);
+  }
+
+  @Test
+  public void testLowestCommonAncestorDeeperHierarchy() throws Exception {
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    FSQueue aQueue = mock(FSLeafQueue.class);
+    FSQueue bQueue = mock(FSLeafQueue.class);
+    FSQueue a1Queue = mock(FSLeafQueue.class);
+    FSQueue b1Queue = mock(FSLeafQueue.class);
+    when(a1Queue.getName()).thenReturn("root.queue1.a.a1");
+    when(b1Queue.getName()).thenReturn("root.queue1.b.b1");
+    when(aQueue.getChildQueues()).thenReturn(Arrays.asList(a1Queue));
+    when(bQueue.getChildQueues()).thenReturn(Arrays.asList(b1Queue));
+
+    QueueManager queueManager = scheduler.getQueueManager();
+    FSParentQueue queue1 = queueManager.getParentQueue("queue1", true);
+    queue1.addChildQueue(aQueue);
+    queue1.addChildQueue(bQueue);
+
+    FSQueue ancestorQueue =
+        scheduler.findLowestCommonAncestorQueue(a1Queue, b1Queue);
+    assertEquals(ancestorQueue, queue1);
+  }
 }