Skip to content

Commit

Permalink
YARN-10850. TimelineService v2 lists containers for all attempts when…
Browse files Browse the repository at this point in the history
… filtering for one. Contributed by Benjamin Teke
  • Loading branch information
szilard-nemeth committed May 18, 2022
1 parent e885633 commit 1ae834f
Show file tree
Hide file tree
Showing 3 changed files with 35 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -129,9 +129,8 @@ public List<ContainerReport> getContainers(ApplicationAttemptId
applicationAttemptId) throws YarnException, IOException {
ApplicationId appId = applicationAttemptId.getApplicationId();
Map<String, String> 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<TimelineEntity> entities = readerClient.getContainerEntities(
appId, "ALL", filters, 0, null);
List<ContainerReport> containers =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -208,12 +211,21 @@ public List<TimelineEntity> 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<String, String> defaults,
Map<String, String> filters) {
Map<String, String> filters) throws UnsupportedEncodingException {
if (filters != null && !filters.isEmpty()) {
for (Map.Entry<String, String> entry : filters.entrySet()) {
if (!defaults.containsKey(entry.getKey())) {
defaults.add(entry.getKey(), filters.get(entry.getValue()));
defaults.add(entry.getKey(), encodeValue(entry.getValue()));
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -47,6 +48,8 @@
*/
public class TestTimelineReaderClientImpl {

private final String appAttemptInfoFilter = "{\"type\":\"YARN_APPLICATION_ATTEMPT\"," +
"\"id\":\"appattempt_1234_0001_000001\"}";
private TimelineReaderClient client;

@Before
Expand Down Expand Up @@ -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<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
public void tearDown() {
if (client != null) {
Expand Down Expand Up @@ -135,11 +148,15 @@ private class MockTimelineReaderClient extends TimelineReaderClientImpl {
protected ClientResponse doGetUri(URI base, String path,
MultivaluedMap<String, String> 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"));
Expand All @@ -151,6 +168,7 @@ protected ClientResponse doGetUri(URI base, String path,
when(mockClientResponse.getEntity(TimelineEntity[].class)).thenReturn(
createTimelineEntities("mockApp1", "mockApp2"));
}

return mockClientResponse;
}
}
Expand Down

0 comments on commit 1ae834f

Please # to comment.