[ https://issues.apache.org/jira/browse/YARN-11191?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17605924#comment-17605924 ]
ASF GitHub Bot commented on YARN-11191: --------------------------------------- goiri commented on code in PR #4726: URL: https://github.com/apache/hadoop/pull/4726#discussion_r973210207 ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java: ########## @@ -1384,7 +1385,19 @@ public List<CSQueue> getChildQueues() { } } - + + @Override + public List<CSQueue> getChildQueuesByTryLock() { + try { + while (!readLock.tryLock()){ Review Comment: Why not just a regular lock()? ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java: ########## @@ -3026,4 +3030,69 @@ public void testReservedContainerLeakWhenMoveApplication() throws Exception { Assert.assertEquals(0, desQueue.getUsedResources().getMemorySize()); rm1.close(); } + @Test + public void testRefreshQueueWithOpenPreemption() throws Exception { + CapacitySchedulerConfiguration csConf Review Comment: The line limit is 100 chars so this should fit. ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptionManager.java: ########## @@ -25,10 +25,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue; import org.apache.hadoop.yarn.util.resource.Resources; -import java.util.Collections; Review Comment: Avoid. ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java: ########## @@ -3026,4 +3030,69 @@ public void testReservedContainerLeakWhenMoveApplication() throws Exception { Assert.assertEquals(0, desQueue.getUsedResources().getMemorySize()); rm1.close(); } + @Test + public void testRefreshQueueWithOpenPreemption() throws Exception { + CapacitySchedulerConfiguration csConf + = new CapacitySchedulerConfiguration(); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {"a"}); + csConf.setCapacity("root.a", 100); + csConf.setMaximumCapacity("root.a", 100); + csConf.setUserLimitFactor("root.a", 100); + + YarnConfiguration conf=new YarnConfiguration(csConf); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + RMNodeLabelsManager mgr=new NullRMNodeLabelsManager(); + mgr.init(conf); + MockRM rm1 = new MockRM(csConf); + CapacityScheduler scheduler=(CapacityScheduler) rm1.getResourceScheduler(); + PreemptionManager preemptionManager = scheduler.getPreemptionManager();; Review Comment: ;; ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java: ########## @@ -3026,4 +3030,69 @@ public void testReservedContainerLeakWhenMoveApplication() throws Exception { Assert.assertEquals(0, desQueue.getUsedResources().getMemorySize()); rm1.close(); } + @Test + public void testRefreshQueueWithOpenPreemption() throws Exception { + CapacitySchedulerConfiguration csConf + = new CapacitySchedulerConfiguration(); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {"a"}); + csConf.setCapacity("root.a", 100); + csConf.setMaximumCapacity("root.a", 100); + csConf.setUserLimitFactor("root.a", 100); + + YarnConfiguration conf=new YarnConfiguration(csConf); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + RMNodeLabelsManager mgr=new NullRMNodeLabelsManager(); Review Comment: Spaces ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java: ########## @@ -3026,4 +3030,69 @@ public void testReservedContainerLeakWhenMoveApplication() throws Exception { Assert.assertEquals(0, desQueue.getUsedResources().getMemorySize()); rm1.close(); } + @Test + public void testRefreshQueueWithOpenPreemption() throws Exception { + CapacitySchedulerConfiguration csConf + = new CapacitySchedulerConfiguration(); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {"a"}); + csConf.setCapacity("root.a", 100); + csConf.setMaximumCapacity("root.a", 100); + csConf.setUserLimitFactor("root.a", 100); + + YarnConfiguration conf=new YarnConfiguration(csConf); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, Review Comment: 1 line ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java: ########## @@ -3026,4 +3030,69 @@ public void testReservedContainerLeakWhenMoveApplication() throws Exception { Assert.assertEquals(0, desQueue.getUsedResources().getMemorySize()); rm1.close(); } + @Test + public void testRefreshQueueWithOpenPreemption() throws Exception { + CapacitySchedulerConfiguration csConf + = new CapacitySchedulerConfiguration(); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {"a"}); + csConf.setCapacity("root.a", 100); + csConf.setMaximumCapacity("root.a", 100); + csConf.setUserLimitFactor("root.a", 100); + + YarnConfiguration conf=new YarnConfiguration(csConf); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + RMNodeLabelsManager mgr=new NullRMNodeLabelsManager(); + mgr.init(conf); + MockRM rm1 = new MockRM(csConf); + CapacityScheduler scheduler=(CapacityScheduler) rm1.getResourceScheduler(); + PreemptionManager preemptionManager = scheduler.getPreemptionManager();; + rm1.getRMContext().setNodeLabelManager(mgr); + rm1.start(); + + LeafQueue srcQueue = (LeafQueue) scheduler.getQueue("a"); + + Thread schedulerThread = new Thread(()-> { + srcQueue.readLock.lock(); + try { + Thread.sleep(1000 * 15); + } catch (InterruptedException e) { + e.printStackTrace(); + } + preemptionManager.getKillableContainers("a", + srcQueue.getDefaultNodeLabelExpression()); + srcQueue.readLock.unlock(); + }); + + Thread completeThread = new Thread(() ->{ + try { + Thread.sleep(1000 * 5); + } catch (InterruptedException e) { + e.printStackTrace(); + } + srcQueue.writeLock.lock(); + srcQueue.writeLock.unlock(); + }); + + Thread refreshQueueThread = new Thread(()->{ + preemptionManager.getWriteLock().lock(); + try { + Thread.sleep(1000 * 10); Review Comment: Spaces ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java: ########## @@ -3026,4 +3030,69 @@ public void testReservedContainerLeakWhenMoveApplication() throws Exception { Assert.assertEquals(0, desQueue.getUsedResources().getMemorySize()); rm1.close(); } + @Test + public void testRefreshQueueWithOpenPreemption() throws Exception { + CapacitySchedulerConfiguration csConf + = new CapacitySchedulerConfiguration(); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[] {"a"}); + csConf.setCapacity("root.a", 100); + csConf.setMaximumCapacity("root.a", 100); + csConf.setUserLimitFactor("root.a", 100); + + YarnConfiguration conf=new YarnConfiguration(csConf); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + RMNodeLabelsManager mgr=new NullRMNodeLabelsManager(); + mgr.init(conf); + MockRM rm1 = new MockRM(csConf); + CapacityScheduler scheduler=(CapacityScheduler) rm1.getResourceScheduler(); + PreemptionManager preemptionManager = scheduler.getPreemptionManager();; + rm1.getRMContext().setNodeLabelManager(mgr); + rm1.start(); + + LeafQueue srcQueue = (LeafQueue) scheduler.getQueue("a"); + + Thread schedulerThread = new Thread(()-> { + srcQueue.readLock.lock(); + try { + Thread.sleep(1000 * 15); + } catch (InterruptedException e) { + e.printStackTrace(); + } + preemptionManager.getKillableContainers("a", Review Comment: 1 line ########## hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java: ########## @@ -3026,4 +3030,69 @@ public void testReservedContainerLeakWhenMoveApplication() throws Exception { Assert.assertEquals(0, desQueue.getUsedResources().getMemorySize()); rm1.close(); } + @Test + public void testRefreshQueueWithOpenPreemption() throws Exception { Review Comment: Add a description explaining the locking part. > Global Scheduler refreshQueue cause deadLock > --------------------------------------------- > > Key: YARN-11191 > URL: https://issues.apache.org/jira/browse/YARN-11191 > Project: Hadoop YARN > Issue Type: Bug > Components: capacity scheduler > Affects Versions: 2.9.0, 3.0.0, 3.1.0, 2.10.0, 3.2.0, 3.3.0 > Reporter: ben yang > Priority: Major > Labels: pull-request-available > Attachments: 1.jstack, Lock holding status.png, YARN-11191.001.patch > > > This is a potential bug may impact all open premmption cluster.In our > current version with preemption enabled, the capacityScheduler will call the > refreshQueue method of the PreemptionManager when it refreshQueue. This > process hold the preemptionManager write lock and require csqueue read > lock.Meanwhile,ParentQueue.canAssignToThisQueue will hold csqueue readLock > and require PreemptionManager ReadLock. > There is a possibility of deadlock at this time.Because readlock has one rule > on unfair policy, when a lock is already occupied by a read lock and the > first request in the lock competition queue is a write lock request,other > read lock requests cann‘t acquire the lock. > So the potential deadlock is: > {code:java} > CapacityScheduler.refreshQueue: hold: PremmptionManager.writeLock > require: csqueue.readLock > CapacityScheduler.schedule: hold: csqueue.readLock > require: PremmptionManager.readLock > other thread(completeContainer,release Resource,etc.): require: > csqueue.writeLock > {code} > The jstack logs at the time were as follows -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: yarn-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: yarn-issues-h...@hadoop.apache.org