Skip to content

Commit 90add05

Browse files
committed
YARN-9489. Support filtering by request-priorities and allocation-request-ids for query results of app activities. Contributed by Tao Yang.
1 parent 2595125 commit 90add05

File tree

13 files changed

+196
-13
lines changed

13 files changed

+196
-13
lines changed

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java

Lines changed: 14 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import com.google.common.annotations.VisibleForTesting;
2222
import org.apache.hadoop.yarn.api.records.Resource;
2323
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
24+
import org.apache.commons.collections.CollectionUtils;
2425
import org.slf4j.Logger;
2526
import org.slf4j.LoggerFactory;
2627
import org.apache.hadoop.service.AbstractService;
@@ -43,6 +44,7 @@
4344
import java.util.Set;
4445
import java.util.*;
4546
import java.util.ArrayList;
47+
import java.util.stream.Collectors;
4648

4749
/**
4850
* A class to store node or application allocations.
@@ -89,15 +91,25 @@ public ActivitiesManager(RMContext rmContext) {
8991
this.rmContext = rmContext;
9092
}
9193

92-
public AppActivitiesInfo getAppActivitiesInfo(ApplicationId applicationId) {
94+
public AppActivitiesInfo getAppActivitiesInfo(ApplicationId applicationId,
95+
Set<String> requestPriorities, Set<String> allocationRequestIds) {
9396
RMApp app = rmContext.getRMApps().get(applicationId);
9497
if (app != null && app.getFinalApplicationStatus()
9598
== FinalApplicationStatus.UNDEFINED) {
9699
Queue<AppAllocation> curAllocations =
97100
completedAppAllocations.get(applicationId);
98101
List<AppAllocation> allocations = null;
99102
if (curAllocations != null) {
100-
allocations = new ArrayList(curAllocations);
103+
if (CollectionUtils.isNotEmpty(requestPriorities) || CollectionUtils
104+
.isNotEmpty(allocationRequestIds)) {
105+
allocations = curAllocations.stream().map(e -> e
106+
.filterAllocationAttempts(requestPriorities,
107+
allocationRequestIds))
108+
.filter(e -> !e.getAllocationAttempts().isEmpty())
109+
.collect(Collectors.toList());
110+
} else {
111+
allocations = new ArrayList(curAllocations);
112+
}
101113
}
102114
return new AppActivitiesInfo(allocations, applicationId);
103115
} else {

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/AppAllocation.java

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,16 @@
1818

1919
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities;
2020

21+
import org.apache.commons.collections.CollectionUtils;
2122
import org.apache.hadoop.yarn.api.records.ContainerId;
2223
import org.apache.hadoop.yarn.api.records.NodeId;
2324
import org.apache.hadoop.yarn.api.records.Priority;
2425

2526
import java.util.ArrayList;
2627
import java.util.List;
28+
import java.util.Set;
29+
import java.util.function.Predicate;
30+
import java.util.stream.Collectors;
2731

2832
/*
2933
* It contains allocation information for one application within a period of
@@ -105,4 +109,23 @@ public long getTime() {
105109
public List<ActivityNode> getAllocationAttempts() {
106110
return allocationAttempts;
107111
}
112+
113+
public AppAllocation filterAllocationAttempts(Set<String> requestPriorities,
114+
Set<String> allocationRequestIds) {
115+
AppAllocation appAllocation =
116+
new AppAllocation(this.priority, this.nodeId, this.queueName);
117+
appAllocation.appState = this.appState;
118+
appAllocation.containerId = this.containerId;
119+
appAllocation.timestamp = this.timestamp;
120+
appAllocation.diagnostic = this.diagnostic;
121+
Predicate<ActivityNode> predicate = (e) ->
122+
(CollectionUtils.isEmpty(requestPriorities) || requestPriorities
123+
.contains(e.getRequestPriority())) && (
124+
CollectionUtils.isEmpty(allocationRequestIds)
125+
|| allocationRequestIds.contains(e.getAllocationRequestId()));
126+
appAllocation.allocationAttempts =
127+
this.allocationAttempts.stream().filter(predicate)
128+
.collect(Collectors.toList());
129+
return appAllocation;
130+
}
108131
}

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

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -226,6 +226,8 @@ public final class RMWSConsts {
226226
public static final String DESELECTS = "deSelects";
227227
public static final String CONTAINERS = "containers";
228228
public static final String QUEUE_ACL_TYPE = "queue-acl-type";
229+
public static final String REQUEST_PRIORITIES = "requestPriorities";
230+
public static final String ALLOCATION_REQUEST_IDS = "allocationRequestIds";
229231

230232
private RMWSConsts() {
231233
// not called

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

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -218,10 +218,15 @@ AppsInfo getApps(HttpServletRequest hsr, String stateQuery,
218218
* QueryParam.
219219
* @param time for how long we want to retrieve the activities. It is a
220220
* QueryParam.
221+
* @param requestPriorities the request priorities we want to retrieve the
222+
* activities. It is a QueryParam.
223+
* @param allocationRequestIds the allocation request ids we want to retrieve
224+
* the activities. It is a QueryParam.
221225
* @return all the activities about a specific app for a specific time
222226
*/
223227
AppActivitiesInfo getAppActivities(HttpServletRequest hsr, String appId,
224-
String time);
228+
String time, Set<String> requestPriorities,
229+
Set<String> allocationRequestIds);
225230

226231
/**
227232
* This method retrieves all the statistics for a specific app, and it is

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

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -706,7 +706,10 @@ public ActivitiesInfo getActivities(@Context HttpServletRequest hsr,
706706
@Override
707707
public AppActivitiesInfo getAppActivities(@Context HttpServletRequest hsr,
708708
@QueryParam(RMWSConsts.APP_ID) String appId,
709-
@QueryParam(RMWSConsts.MAX_TIME) String time) {
709+
@QueryParam(RMWSConsts.MAX_TIME) String time,
710+
@QueryParam(RMWSConsts.REQUEST_PRIORITIES) Set<String> requestPriorities,
711+
@QueryParam(RMWSConsts.ALLOCATION_REQUEST_IDS)
712+
Set<String> allocationRequestIds) {
710713
initForReadableEndpoints();
711714

712715
YarnScheduler scheduler = rm.getRMContext().getScheduler();
@@ -741,7 +744,8 @@ public AppActivitiesInfo getAppActivities(@Context HttpServletRequest hsr,
741744
applicationId = ApplicationId.fromString(appId);
742745
activitiesManager.turnOnAppActivitiesRecording(applicationId, maxTime);
743746
AppActivitiesInfo appActivitiesInfo =
744-
activitiesManager.getAppActivitiesInfo(applicationId);
747+
activitiesManager.getAppActivitiesInfo(applicationId,
748+
requestPriorities, allocationRequestIds);
745749

746750
return appActivitiesInfo;
747751
} catch (Exception e) {

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

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,9 @@
1818

1919
package org.apache.hadoop.yarn.server.resourcemanager.webapp;
2020

21+
import com.sun.jersey.api.client.ClientResponse;
22+
import com.sun.jersey.api.client.WebResource;
23+
import org.apache.hadoop.http.JettyUtils;
2124
import org.apache.hadoop.yarn.api.records.ExecutionType;
2225
import org.apache.hadoop.yarn.api.records.ExecutionTypeRequest;
2326
import org.apache.hadoop.yarn.api.records.Priority;
@@ -31,6 +34,8 @@
3134
import org.codehaus.jettison.json.JSONException;
3235
import org.codehaus.jettison.json.JSONObject;
3336

37+
import javax.ws.rs.core.MediaType;
38+
import javax.ws.rs.core.MultivaluedMap;
3439
import java.util.ArrayList;
3540
import java.util.Arrays;
3641
import java.util.HashSet;
@@ -192,4 +197,16 @@ public static void verifyNumberOfAllocationAttempts(JSONObject allocation,
192197
}
193198
}
194199
}
200+
201+
public static JSONObject requestWebResource(WebResource webResource,
202+
MultivaluedMap<String, String> params) {
203+
if (params != null) {
204+
webResource = webResource.queryParams(params);
205+
}
206+
ClientResponse response = webResource.accept(MediaType.APPLICATION_JSON)
207+
.get(ClientResponse.class);
208+
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
209+
response.getType().toString());
210+
return response.getEntity(JSONObject.class);
211+
}
195212
}

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

Lines changed: 111 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -953,4 +953,115 @@ public void testAppPlacementConstraintDiagnostic()
953953
rm.stop();
954954
}
955955
}
956+
957+
@Test (timeout=30000)
958+
public void testAppFilterByRequestPrioritiesAndAllocationRequestIds()
959+
throws Exception {
960+
rm.start();
961+
CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
962+
963+
MockNM nm1 = rm.registerNode("127.0.0.1:1234", 8 * 1024);
964+
965+
try {
966+
RMApp app1 = rm.submitApp(512, "app1", "user1", null, "b1");
967+
MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1);
968+
969+
WebResource r = resource().path(RMWSConsts.RM_WEB_SERVICE_PATH)
970+
.path(RMWSConsts.SCHEDULER_APP_ACTIVITIES);
971+
MultivaluedMapImpl params = new MultivaluedMapImpl();
972+
params.add("appId", app1.getApplicationId().toString());
973+
JSONObject json = ActivitiesTestUtils.requestWebResource(r, params);
974+
assertEquals("waiting for display",
975+
json.getString("diagnostic"));
976+
977+
// am1 asks for 1 * 1GB container with requestPriority=-1
978+
// and allocationRequestId=1
979+
am1.allocate(Arrays.asList(
980+
ResourceRequest.newBuilder().priority(Priority.UNDEFINED)
981+
.allocationRequestId(1).resourceName("*")
982+
.capability(Resources.createResource(1 * 1024)).numContainers(1)
983+
.build()), null);
984+
// trigger scheduling
985+
cs.handle(new NodeUpdateSchedulerEvent(
986+
rm.getRMContext().getRMNodes().get(nm1.getNodeId())));
987+
988+
// am1 asks for 1 * 1GB container with requestPriority=-1
989+
// and allocationRequestId=2
990+
am1.allocate(Arrays.asList(
991+
ResourceRequest.newBuilder().priority(Priority.UNDEFINED)
992+
.allocationRequestId(2).resourceName("*")
993+
.capability(Resources.createResource(1 * 1024)).numContainers(1)
994+
.build()), null);
995+
// trigger scheduling
996+
cs.handle(new NodeUpdateSchedulerEvent(
997+
rm.getRMContext().getRMNodes().get(nm1.getNodeId())));
998+
999+
// am1 asks for 1 * 1GB container with requestPriority=0
1000+
// and allocationRequestId=1
1001+
am1.allocate(Arrays.asList(
1002+
ResourceRequest.newBuilder().priority(Priority.newInstance(0))
1003+
.allocationRequestId(1).resourceName("*")
1004+
.capability(Resources.createResource(1 * 1024)).numContainers(1)
1005+
.build()), null);
1006+
// trigger scheduling
1007+
cs.handle(new NodeUpdateSchedulerEvent(
1008+
rm.getRMContext().getRMNodes().get(nm1.getNodeId())));
1009+
1010+
// am1 asks for 1 * 1GB container with requestPriority=0
1011+
// and allocationRequestId=3
1012+
am1.allocate(Arrays.asList(
1013+
ResourceRequest.newBuilder().priority(Priority.newInstance(0))
1014+
.allocationRequestId(3).resourceName("*")
1015+
.capability(Resources.createResource(1 * 1024)).numContainers(1)
1016+
.build()), null);
1017+
// trigger scheduling
1018+
cs.handle(new NodeUpdateSchedulerEvent(
1019+
rm.getRMContext().getRMNodes().get(nm1.getNodeId())));
1020+
1021+
// query app activities with requestPriorities={0,1}
1022+
MultivaluedMapImpl filterParams1 = new MultivaluedMapImpl(params);
1023+
filterParams1.add(RMWSConsts.REQUEST_PRIORITIES, "-1");
1024+
filterParams1.add(RMWSConsts.REQUEST_PRIORITIES, "0");
1025+
json = ActivitiesTestUtils.requestWebResource(r, filterParams1);
1026+
verifyNumberOfAllocations(json, 4);
1027+
1028+
// query app activities with requestPriorities=0
1029+
MultivaluedMapImpl filterParams2 = new MultivaluedMapImpl(params);
1030+
filterParams2.add(RMWSConsts.REQUEST_PRIORITIES, "-1");
1031+
json = ActivitiesTestUtils.requestWebResource(r, filterParams2);
1032+
verifyNumberOfAllocations(json, 2);
1033+
JSONArray allocations = json.getJSONArray("allocations");
1034+
for (int i=0; i<allocations.length(); i++) {
1035+
assertEquals("-1",
1036+
allocations.getJSONObject(i).getJSONObject("requestAllocation")
1037+
.optString("requestPriority"));
1038+
}
1039+
1040+
// query app activities with allocationRequestId=1
1041+
MultivaluedMapImpl filterParams3 = new MultivaluedMapImpl(params);
1042+
filterParams3.add(RMWSConsts.ALLOCATION_REQUEST_IDS, "1");
1043+
json = ActivitiesTestUtils.requestWebResource(r, filterParams3);
1044+
verifyNumberOfAllocations(json, 2);
1045+
allocations = json.getJSONArray("allocations");
1046+
for (int i = 0; i < allocations.length(); i++) {
1047+
assertEquals("1",
1048+
allocations.getJSONObject(i).getJSONObject("requestAllocation")
1049+
.optString("allocationRequestId"));
1050+
}
1051+
1052+
// query app activities with requestPriorities=0 and allocationRequestId=1
1053+
MultivaluedMapImpl filterParams4 = new MultivaluedMapImpl(params);
1054+
filterParams4.add(RMWSConsts.REQUEST_PRIORITIES, "0");
1055+
filterParams4.add(RMWSConsts.ALLOCATION_REQUEST_IDS, "1");
1056+
json = ActivitiesTestUtils.requestWebResource(r, filterParams4);
1057+
verifyNumberOfAllocations(json, 1);
1058+
JSONObject allocation = json.getJSONObject("allocations");
1059+
assertEquals("0", allocation.getJSONObject("requestAllocation")
1060+
.optString("requestPriority"));
1061+
assertEquals("1", allocation.getJSONObject("requestAllocation")
1062+
.optString("allocationRequestId"));
1063+
} finally {
1064+
rm.stop();
1065+
}
1066+
}
9561067
}

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/DefaultRequestInterceptorREST.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -190,7 +190,8 @@ public ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId) {
190190

191191
@Override
192192
public AppActivitiesInfo getAppActivities(HttpServletRequest hsr,
193-
String appId, String time) {
193+
String appId, String time, Set<String> requestPriorities,
194+
Set<String> allocationRequestIds) {
194195
// time and appId are specified inside hsr
195196
return RouterWebServiceUtil.genericForward(webAppAddress, hsr,
196197
AppActivitiesInfo.class, HTTPMethods.GET,

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1144,7 +1144,8 @@ public ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId) {
11441144

11451145
@Override
11461146
public AppActivitiesInfo getAppActivities(HttpServletRequest hsr,
1147-
String appId, String time) {
1147+
String appId, String time, Set<String> requestPriorities,
1148+
Set<String> allocationRequestIds) {
11481149
throw new NotImplementedException("Code is not implemented");
11491150
}
11501151

hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/RouterWebServices.java

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -459,10 +459,14 @@ public ActivitiesInfo getActivities(@Context HttpServletRequest hsr,
459459
@Override
460460
public AppActivitiesInfo getAppActivities(@Context HttpServletRequest hsr,
461461
@QueryParam(RMWSConsts.APP_ID) String appId,
462-
@QueryParam(RMWSConsts.MAX_TIME) String time) {
462+
@QueryParam(RMWSConsts.MAX_TIME) String time,
463+
@QueryParam(RMWSConsts.REQUEST_PRIORITIES) Set<String> requestPriorities,
464+
@QueryParam(RMWSConsts.ALLOCATION_REQUEST_IDS)
465+
Set<String> allocationRequestIds) {
463466
init();
464467
RequestInterceptorChainWrapper pipeline = getInterceptorChain(hsr);
465-
return pipeline.getRootInterceptor().getAppActivities(hsr, appId, time);
468+
return pipeline.getRootInterceptor().getAppActivities(hsr, appId, time,
469+
requestPriorities, allocationRequestIds);
466470
}
467471

468472
@GET

0 commit comments

Comments
 (0)