From 1ae834fe0b1efb28a4fec56e9532b88864392cbe Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Wed, 18 May 2022 14:04:51 +0200 Subject: [PATCH] YARN-10850. TimelineService v2 lists containers for all attempts when filtering for one. Contributed by Benjamin Teke --- .../yarn/client/api/impl/AHSv2ClientImpl.java | 5 ++--- .../api/impl/TimelineReaderClientImpl.java | 16 +++++++++++++-- .../impl/TestTimelineReaderClientImpl.java | 20 ++++++++++++++++++- 3 files changed, 35 insertions(+), 6 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSv2ClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSv2ClientImpl.java index e797c2816252c..3422493aa1a4c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSv2ClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSv2ClientImpl.java @@ -129,9 +129,8 @@ public List getContainers(ApplicationAttemptId applicationAttemptId) throws YarnException, IOException { ApplicationId appId = applicationAttemptId.getApplicationId(); Map filters = new HashMap<>(); - filters.put("infofilters", "SYSTEM_INFO_PARENT_ENTITY eq {\"id\":\"" + - applicationAttemptId.toString() + - "\",\"type\":\"YARN_APPLICATION_ATTEMPT\"}"); + filters.put("infofilters", "SYSTEM_INFO_PARENT_ENTITY eq " + + "{\"type\":\"YARN_APPLICATION_ATTEMPT\",\"id\":\"" + applicationAttemptId + "\"}"); List entities = readerClient.getContainerEntities( appId, "ALL", filters, 0, null); List containers = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineReaderClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineReaderClientImpl.java index db53f93136f73..aafe4c3db11d8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineReaderClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineReaderClientImpl.java @@ -38,7 +38,10 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.MultivaluedMap; import java.io.IOException; +import java.io.UnsupportedEncodingException; import java.net.URI; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -208,12 +211,21 @@ public List getContainerEntities( 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 defaults, - Map filters) { + Map filters) throws UnsupportedEncodingException { if (filters != null && !filters.isEmpty()) { for (Map.Entry entry : filters.entrySet()) { if (!defaults.containsKey(entry.getKey())) { - defaults.add(entry.getKey(), filters.get(entry.getValue())); + defaults.add(entry.getKey(), encodeValue(entry.getValue())); } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineReaderClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineReaderClientImpl.java index f668472256a7b..d6ca25f9501b4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineReaderClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineReaderClientImpl.java @@ -23,6 +23,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableMap; import com.sun.jersey.api.client.ClientResponse; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; @@ -47,6 +48,8 @@ */ public class TestTimelineReaderClientImpl { + private final String appAttemptInfoFilter = "{\"type\":\"YARN_APPLICATION_ATTEMPT\"," + + "\"id\":\"appattempt_1234_0001_000001\"}"; private TimelineReaderClient client; @Before @@ -107,6 +110,16 @@ public void testGetContainers() throws Exception { Assert.assertEquals("mockContainer2", entities.get(1).getId()); } + @Test + public void testGetContainersForAppAttempt() throws Exception { + ApplicationId appId = + ApplicationId.fromString("application_1234_0001"); + List entities = client.getContainerEntities(appId, + null, ImmutableMap.of("infofilters", appAttemptInfoFilter), 0, null); + Assert.assertEquals(2, entities.size()); + Assert.assertEquals("mockContainer4", entities.get(1).getId()); + } + @After public void tearDown() { if (client != null) { @@ -135,11 +148,15 @@ private class MockTimelineReaderClient extends TimelineReaderClientImpl { protected ClientResponse doGetUri(URI base, String path, MultivaluedMap params) throws IOException { 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( createTimelineEntity("mockContainer1")); when(mockClientResponse.getEntity(TimelineEntity[].class)).thenReturn( 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())) { when(mockClientResponse.getEntity(TimelineEntity.class)).thenReturn( createTimelineEntity("mockAppAttempt1")); @@ -151,6 +168,7 @@ protected ClientResponse doGetUri(URI base, String path, when(mockClientResponse.getEntity(TimelineEntity[].class)).thenReturn( createTimelineEntities("mockApp1", "mockApp2")); } + return mockClientResponse; } }