YARN-10850. TimelineService v2 lists containers for all attempts when filtering for one. Contributed by Benjamin Teke

This commit is contained in:
Szilard Nemeth 2022-05-18 14:08:41 +02:00
parent b0012ee578
commit b4550b3356
3 changed files with 35 additions and 6 deletions

View File

@ -137,9 +137,8 @@ public List<ContainerReport> getContainers(ApplicationAttemptId
ApplicationId appId = applicationAttemptId.getApplicationId(); ApplicationId appId = applicationAttemptId.getApplicationId();
ApplicationReport appReport = getApplicationReport(appId); ApplicationReport appReport = getApplicationReport(appId);
Map<String, String> filters = new HashMap<>(); Map<String, String> filters = new HashMap<>();
filters.put("infofilters", "SYSTEM_INFO_PARENT_ENTITY eq {\"id\":\"" + filters.put("infofilters", "SYSTEM_INFO_PARENT_ENTITY eq "
applicationAttemptId.toString() + + "{\"type\":\"YARN_APPLICATION_ATTEMPT\",\"id\":\"" + applicationAttemptId + "\"}");
"\",\"type\":\"YARN_APPLICATION_ATTEMPT\"}");
List<TimelineEntity> entities = readerClient.getContainerEntities( List<TimelineEntity> entities = readerClient.getContainerEntities(
appId, "ALL", filters, 0, null); appId, "ALL", filters, 0, null);
List<ContainerReport> containers = List<ContainerReport> containers =

View File

@ -38,7 +38,10 @@
import javax.ws.rs.core.MediaType; import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.MultivaluedMap; import javax.ws.rs.core.MultivaluedMap;
import java.io.IOException; import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.net.URI; import java.net.URI;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -208,12 +211,21 @@ public List<TimelineEntity> getContainerEntities(
return Arrays.asList(entity); return Arrays.asList(entity);
} }
@VisibleForTesting
protected String encodeValue(String value) throws UnsupportedEncodingException {
// Since URLEncoder doesn't use and doesn't have an option for percent-encoding
// (as specified in RFC 3986) the spaces are encoded to + signs, which need to be replaced
// manually
return URLEncoder.encode(value, StandardCharsets.UTF_8.toString())
.replaceAll("\\+", "%20");
}
private void mergeFilters(MultivaluedMap<String, String> defaults, private void mergeFilters(MultivaluedMap<String, String> defaults,
Map<String, String> filters) { Map<String, String> filters) throws UnsupportedEncodingException {
if (filters != null && !filters.isEmpty()) { if (filters != null && !filters.isEmpty()) {
for (Map.Entry<String, String> entry : filters.entrySet()) { for (Map.Entry<String, String> entry : filters.entrySet()) {
if (!defaults.containsKey(entry.getKey())) { if (!defaults.containsKey(entry.getKey())) {
defaults.add(entry.getKey(), filters.get(entry.getValue())); defaults.add(entry.getKey(), encodeValue(entry.getValue()));
} }
} }
} }

View File

@ -25,6 +25,7 @@
import com.sun.jersey.api.client.ClientResponse; import com.sun.jersey.api.client.ClientResponse;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
@ -47,6 +48,8 @@
*/ */
public class TestTimelineReaderClientImpl { public class TestTimelineReaderClientImpl {
private final String appAttemptInfoFilter = "{\"type\":\"YARN_APPLICATION_ATTEMPT\"," +
"\"id\":\"appattempt_1234_0001_000001\"}";
private TimelineReaderClient client; private TimelineReaderClient client;
@Before @Before
@ -107,6 +110,16 @@ public void testGetContainers() throws Exception {
Assert.assertEquals("mockContainer2", entities.get(1).getId()); Assert.assertEquals("mockContainer2", entities.get(1).getId());
} }
@Test
public void testGetContainersForAppAttempt() throws Exception {
ApplicationId appId =
ApplicationId.fromString("application_1234_0001");
List<TimelineEntity> entities = client.getContainerEntities(appId,
null, ImmutableMap.of("infofilters", appAttemptInfoFilter), 0, null);
Assert.assertEquals(2, entities.size());
Assert.assertEquals("mockContainer4", entities.get(1).getId());
}
@After @After
public void tearDown() { public void tearDown() {
if (client != null) { if (client != null) {
@ -135,11 +148,15 @@ private class MockTimelineReaderClient extends TimelineReaderClientImpl {
protected ClientResponse doGetUri(URI base, String path, protected ClientResponse doGetUri(URI base, String path,
MultivaluedMap<String, String> params) throws IOException { MultivaluedMap<String, String> params) throws IOException {
ClientResponse mockClientResponse = mock(ClientResponse.class); ClientResponse mockClientResponse = mock(ClientResponse.class);
if (path.contains(YARN_CONTAINER.toString())) { if (path.contains(YARN_CONTAINER.toString()) && !params.containsKey("infofilters")) {
when(mockClientResponse.getEntity(TimelineEntity.class)).thenReturn( when(mockClientResponse.getEntity(TimelineEntity.class)).thenReturn(
createTimelineEntity("mockContainer1")); createTimelineEntity("mockContainer1"));
when(mockClientResponse.getEntity(TimelineEntity[].class)).thenReturn( when(mockClientResponse.getEntity(TimelineEntity[].class)).thenReturn(
createTimelineEntities("mockContainer1", "mockContainer2")); createTimelineEntities("mockContainer1", "mockContainer2"));
} else if (path.contains(YARN_CONTAINER.toString()) && params.containsKey("infofilters")) {
Assert.assertEquals(encodeValue(appAttemptInfoFilter), params.get("infofilters").get(0));
when(mockClientResponse.getEntity(TimelineEntity[].class)).thenReturn(
createTimelineEntities("mockContainer3", "mockContainer4"));
} else if (path.contains(YARN_APPLICATION_ATTEMPT.toString())) { } else if (path.contains(YARN_APPLICATION_ATTEMPT.toString())) {
when(mockClientResponse.getEntity(TimelineEntity.class)).thenReturn( when(mockClientResponse.getEntity(TimelineEntity.class)).thenReturn(
createTimelineEntity("mockAppAttempt1")); createTimelineEntity("mockAppAttempt1"));
@ -151,6 +168,7 @@ protected ClientResponse doGetUri(URI base, String path,
when(mockClientResponse.getEntity(TimelineEntity[].class)).thenReturn( when(mockClientResponse.getEntity(TimelineEntity[].class)).thenReturn(
createTimelineEntities("mockApp1", "mockApp2")); createTimelineEntities("mockApp1", "mockApp2"));
} }
return mockClientResponse; return mockClientResponse;
} }
} }