Skip to content

Commit 304a47e

Browse files
committed
YARN-9608. DecommissioningNodesWatcher should get lists of running applications on node from RMNode. Contributed by Abhishek Modi.
1 parent ba681bb commit 304a47e

File tree

2 files changed

+92
-56
lines changed

2 files changed

+92
-56
lines changed

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DecommissioningNodesWatcher.java

Lines changed: 7 additions & 40 deletions
Original file line numberDiff line numberDiff line change
@@ -17,9 +17,11 @@
1717
*/
1818
package org.apache.hadoop.yarn.server.resourcemanager;
1919

20+
import java.util.ArrayList;
2021
import java.util.HashMap;
2122
import java.util.HashSet;
2223
import java.util.Iterator;
24+
import java.util.List;
2325
import java.util.Map;
2426
import java.util.Set;
2527
import java.util.Timer;
@@ -36,7 +38,6 @@
3638
import org.apache.hadoop.yarn.conf.YarnConfiguration;
3739
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
3840
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
39-
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
4041
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
4142
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
4243
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
@@ -58,13 +59,8 @@
5859
* a DECOMMISSIONING node will be DECOMMISSIONED no later than
5960
* DECOMMISSIONING_TIMEOUT regardless of running containers or applications.
6061
*
61-
* To be efficient, DecommissioningNodesWatcher skip tracking application
62-
* containers on a particular node before the node is in DECOMMISSIONING state.
63-
* It only tracks containers once the node is in DECOMMISSIONING state.
6462
* DecommissioningNodesWatcher basically is no cost when no node is
65-
* DECOMMISSIONING. This sacrifices the possibility that the node once
66-
* host containers of an application that is still running
67-
* (the affected map tasks will be rescheduled).
63+
* DECOMMISSIONING.
6864
*/
6965
public class DecommissioningNodesWatcher {
7066
private static final Logger LOG =
@@ -88,8 +84,8 @@ class DecommissioningNodeContext {
8884
// number of running containers at the moment.
8985
private int numActiveContainers;
9086

91-
// All applications run on the node at or after decommissioningStartTime.
92-
private Set<ApplicationId> appIds;
87+
// All applications run on the node.
88+
private List<ApplicationId> appIds;
9389

9490
// First moment the node is observed in DECOMMISSIONED state.
9591
private long decommissionedTime;
@@ -102,7 +98,7 @@ class DecommissioningNodeContext {
10298

10399
public DecommissioningNodeContext(NodeId nodeId, int timeoutSec) {
104100
this.nodeId = nodeId;
105-
this.appIds = new HashSet<ApplicationId>();
101+
this.appIds = new ArrayList<>();
106102
this.decommissioningStartTime = mclock.getTime();
107103
this.timeoutMs = 1000L * timeoutSec;
108104
}
@@ -164,9 +160,7 @@ public synchronized void update(RMNode rmNode, NodeStatus remoteNodeStatus) {
164160
context.updateTimeout(rmNode.getDecommissioningTimeout());
165161
context.lastUpdateTime = now;
166162

167-
if (remoteNodeStatus.getKeepAliveApplications() != null) {
168-
context.appIds.addAll(remoteNodeStatus.getKeepAliveApplications());
169-
}
163+
context.appIds = rmNode.getRunningApps();
170164

171165
// Count number of active containers.
172166
int numActiveContainers = 0;
@@ -176,14 +170,7 @@ public synchronized void update(RMNode rmNode, NodeStatus remoteNodeStatus) {
176170
newState == ContainerState.NEW) {
177171
numActiveContainers++;
178172
}
179-
context.numActiveContainers = numActiveContainers;
180-
ApplicationId aid = cs.getContainerId()
181-
.getApplicationAttemptId().getApplicationId();
182-
if (!context.appIds.contains(aid)) {
183-
context.appIds.add(aid);
184-
}
185173
}
186-
187174
context.numActiveContainers = numActiveContainers;
188175

189176
// maintain lastContainerFinishTime.
@@ -254,7 +241,6 @@ public DecommissioningNodeStatus checkDecommissioningStatus(NodeId nodeId) {
254241
DecommissioningNodeStatus.TIMEOUT;
255242
}
256243

257-
removeCompletedApps(context);
258244
if (context.appIds.size() == 0) {
259245
return DecommissioningNodeStatus.READY;
260246
} else {
@@ -336,25 +322,6 @@ private RMNode getRmNode(NodeId nodeId) {
336322
return rmNode;
337323
}
338324

339-
private void removeCompletedApps(DecommissioningNodeContext context) {
340-
Iterator<ApplicationId> it = context.appIds.iterator();
341-
while (it.hasNext()) {
342-
ApplicationId appId = it.next();
343-
RMApp rmApp = rmContext.getRMApps().get(appId);
344-
if (rmApp == null) {
345-
LOG.debug("Consider non-existing app {} as completed", appId);
346-
it.remove();
347-
continue;
348-
}
349-
if (rmApp.getState() == RMAppState.FINISHED ||
350-
rmApp.getState() == RMAppState.FAILED ||
351-
rmApp.getState() == RMAppState.KILLED) {
352-
LOG.debug("Remove {} app {}", rmApp.getState(), appId);
353-
it.remove();
354-
}
355-
}
356-
}
357-
358325
// Time in second to be decommissioned.
359326
private int getTimeoutInSec(DecommissioningNodeContext context) {
360327
if (context.nodeState == NodeState.DECOMMISSIONED) {

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestDecommissioningNodesWatcher.java

Lines changed: 85 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -19,11 +19,11 @@
1919
package org.apache.hadoop.yarn.server.resourcemanager;
2020

2121
import java.util.ArrayList;
22+
import java.util.Collections;
2223
import java.util.List;
2324

2425
import org.apache.hadoop.conf.Configuration;
2526
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
26-
import org.apache.hadoop.yarn.api.records.ApplicationId;
2727
import org.apache.hadoop.yarn.api.records.ContainerId;
2828
import org.apache.hadoop.yarn.api.records.ContainerState;
2929
import org.apache.hadoop.yarn.api.records.ContainerStatus;
@@ -35,7 +35,8 @@
3535
import org.apache.hadoop.yarn.server.resourcemanager.DecommissioningNodesWatcher.DecommissioningNodeStatus;
3636
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
3737
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
38-
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
38+
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
39+
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
3940
import org.junit.After;
4041
import org.junit.Assert;
4142
import org.junit.Test;
@@ -58,38 +59,106 @@ public void testDecommissioningNodesWatcher() throws Exception {
5859
new DecommissioningNodesWatcher(rm.getRMContext());
5960

6061
MockNM nm1 = rm.registerNode("host1:1234", 10240);
61-
RMNode node1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId());
62+
RMNodeImpl node1 =
63+
(RMNodeImpl) rm.getRMContext().getRMNodes().get(nm1.getNodeId());
6264
NodeId id1 = nm1.getNodeId();
6365

6466
rm.waitForState(id1, NodeState.RUNNING);
65-
Assert.assertFalse(watcher.checkReadyToBeDecommissioned(id1));
6667

6768
RMApp app = rm.submitApp(2000);
6869
MockAM am = MockRM.launchAndRegisterAM(app, rm, nm1);
6970

71+
NodeStatus nodeStatus = createNodeStatus(id1, app, 3);
72+
node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
73+
7074
// Setup nm1 as DECOMMISSIONING for DecommissioningNodesWatcher.
7175
rm.sendNodeGracefulDecommission(nm1,
7276
YarnConfiguration.DEFAULT_RM_NODE_GRACEFUL_DECOMMISSION_TIMEOUT);
7377
rm.waitForState(id1, NodeState.DECOMMISSIONING);
7478

7579
// Update status with decreasing number of running containers until 0.
76-
watcher.update(node1, createNodeStatus(id1, app, 12));
77-
watcher.update(node1, createNodeStatus(id1, app, 11));
80+
nodeStatus = createNodeStatus(id1, app, 3);
81+
node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
82+
watcher.update(node1, nodeStatus);
83+
84+
nodeStatus = createNodeStatus(id1, app, 2);
85+
node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
86+
watcher.update(node1, nodeStatus);
7887
Assert.assertFalse(watcher.checkReadyToBeDecommissioned(id1));
7988

80-
watcher.update(node1, createNodeStatus(id1, app, 1));
89+
nodeStatus = createNodeStatus(id1, app, 1);
90+
node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
91+
watcher.update(node1, nodeStatus);
8192
Assert.assertEquals(DecommissioningNodeStatus.WAIT_CONTAINER,
82-
watcher.checkDecommissioningStatus(id1));
93+
watcher.checkDecommissioningStatus(id1));
94+
95+
nodeStatus = createNodeStatus(id1, app, 0);
96+
watcher.update(node1, nodeStatus);
97+
node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
98+
Assert.assertEquals(DecommissioningNodeStatus.WAIT_APP,
99+
watcher.checkDecommissioningStatus(id1));
100+
101+
// Set app to be FINISHED and verified DecommissioningNodeStatus is READY.
102+
MockRM.finishAMAndVerifyAppState(app, rm, nm1, am);
103+
rm.waitForState(app.getApplicationId(), RMAppState.FINISHED);
104+
watcher.update(node1, nodeStatus);
105+
Assert.assertEquals(DecommissioningNodeStatus.READY,
106+
watcher.checkDecommissioningStatus(id1));
107+
}
108+
109+
@Test
110+
public void testDecommissioningNodesWatcherWithPreviousRunningApps()
111+
throws Exception {
112+
Configuration conf = new Configuration();
113+
conf.set(YarnConfiguration.RM_NODE_GRACEFUL_DECOMMISSION_TIMEOUT, "40");
114+
115+
rm = new MockRM(conf);
116+
rm.start();
117+
118+
DecommissioningNodesWatcher watcher =
119+
new DecommissioningNodesWatcher(rm.getRMContext());
120+
121+
MockNM nm1 = rm.registerNode("host1:1234", 10240);
122+
RMNodeImpl node1 =
123+
(RMNodeImpl) rm.getRMContext().getRMNodes().get(nm1.getNodeId());
124+
NodeId id1 = nm1.getNodeId();
125+
126+
rm.waitForState(id1, NodeState.RUNNING);
127+
128+
RMApp app = rm.submitApp(2000);
129+
MockAM am = MockRM.launchAndRegisterAM(app, rm, nm1);
83130

84-
watcher.update(node1, createNodeStatus(id1, app, 0));
131+
NodeStatus nodeStatus = createNodeStatus(id1, app, 3);
132+
node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
133+
134+
Assert.assertEquals(1, node1.getRunningApps().size());
135+
136+
// update node with 0 running containers
137+
nodeStatus = createNodeStatus(id1, app, 0);
138+
node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus));
139+
140+
Assert.assertEquals(1, node1.getRunningApps().size());
141+
142+
// Setup nm1 as DECOMMISSIONING for DecommissioningNodesWatcher. Right now
143+
// there is no container running on the node.
144+
rm.sendNodeGracefulDecommission(nm1,
145+
YarnConfiguration.DEFAULT_RM_NODE_GRACEFUL_DECOMMISSION_TIMEOUT);
146+
rm.waitForState(id1, NodeState.DECOMMISSIONING);
147+
148+
// we should still get WAIT_APP as container for a running app previously
149+
// ran on this node.
150+
watcher.update(node1, nodeStatus);
151+
Assert.assertFalse(watcher.checkReadyToBeDecommissioned(id1));
85152
Assert.assertEquals(DecommissioningNodeStatus.WAIT_APP,
86-
watcher.checkDecommissioningStatus(id1));
153+
watcher.checkDecommissioningStatus(id1));
87154

88155
// Set app to be FINISHED and verified DecommissioningNodeStatus is READY.
89156
MockRM.finishAMAndVerifyAppState(app, rm, nm1, am);
90157
rm.waitForState(app.getApplicationId(), RMAppState.FINISHED);
158+
Assert.assertEquals(0, node1.getRunningApps().size());
159+
watcher.update(node1, nodeStatus);
91160
Assert.assertEquals(DecommissioningNodeStatus.READY,
92-
watcher.checkDecommissioningStatus(id1));
161+
watcher.checkDecommissioningStatus(id1));
93162
}
94163

95164
@After
@@ -103,7 +172,7 @@ private NodeStatus createNodeStatus(
103172
NodeId nodeId, RMApp app, int numRunningContainers) {
104173
return NodeStatus.newInstance(
105174
nodeId, 0, getContainerStatuses(app, numRunningContainers),
106-
new ArrayList<ApplicationId>(),
175+
Collections.emptyList(),
107176
NodeHealthStatus.newInstance(
108177
true, "", System.currentTimeMillis() - 1000),
109178
null, null, null);
@@ -113,17 +182,17 @@ private NodeStatus createNodeStatus(
113182
// where numRunningContainers are RUNNING.
114183
private List<ContainerStatus> getContainerStatuses(
115184
RMApp app, int numRunningContainers) {
116-
// Total 12 containers
117-
final int total = 12;
185+
// Total 3 containers
186+
final int total = 3;
118187
numRunningContainers = Math.min(total, numRunningContainers);
119188
List<ContainerStatus> output = new ArrayList<ContainerStatus>();
120189
for (int i = 0; i < total; i++) {
121190
ContainerState cstate = (i >= numRunningContainers)?
122191
ContainerState.COMPLETE : ContainerState.RUNNING;
123192
output.add(ContainerStatus.newInstance(
124193
ContainerId.newContainerId(
125-
ApplicationAttemptId.newInstance(app.getApplicationId(), i), 1),
126-
cstate, "Dummy", 0));
194+
ApplicationAttemptId.newInstance(app.getApplicationId(), 0), i),
195+
cstate, "", 0));
127196
}
128197
return output;
129198
}

0 commit comments

Comments
 (0)