You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by va...@apache.org on 2017/08/20 14:29:10 UTC

hadoop git commit: YARN-6861. Reader API for sub application entities (Rohith Sharma K S via Varun Saxena)

Repository: hadoop
Updated Branches:
  refs/heads/YARN-5355 619649607 -> 73ee0d4fc


YARN-6861. Reader API for sub application entities (Rohith Sharma K S via Varun Saxena)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/73ee0d4f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/73ee0d4f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/73ee0d4f

Branch: refs/heads/YARN-5355
Commit: 73ee0d4fc05354527a4be6db9e5b83d06fc94047
Parents: 6196496
Author: Varun Saxena <va...@apache.org>
Authored: Sun Aug 20 00:35:14 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Sun Aug 20 00:35:14 2017 +0530

----------------------------------------------------------------------
 ...stTimelineReaderWebServicesHBaseStorage.java | 104 ++--
 .../storage/reader/GenericEntityReader.java     |   4 +-
 .../reader/SubApplicationEntityReader.java      | 488 +++++++++++++++++++
 .../reader/TimelineEntityReaderFactory.java     |   3 +
 .../reader/TimelineReaderContext.java           |  19 +-
 .../reader/TimelineReaderManager.java           |   9 +-
 .../reader/TimelineReaderWebServices.java       | 158 ++++++
 .../reader/TimelineReaderWebServicesUtils.java  |  10 +
 .../reader/TimelineUIDConverter.java            |  35 ++
 .../reader/TestTimelineUIDConverter.java        |   9 +
 10 files changed, 787 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/73ee0d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
index d3f6a44..fd14131 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -71,6 +71,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
   private static long ts = System.currentTimeMillis();
   private static long dayTs =
       HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts);
+  private static String doAsUser = "remoteuser";
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
@@ -337,7 +338,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
     HBaseTimelineWriterImpl hbi = null;
     Configuration c1 = getHBaseTestingUtility().getConfiguration();
     UserGroupInformation remoteUser =
-        UserGroupInformation.createRemoteUser(user);
+        UserGroupInformation.createRemoteUser(doAsUser);
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
@@ -2263,60 +2264,69 @@ public class TestTimelineReaderWebServicesHBaseStorage
   public void testGenericEntitiesForPagination() throws Exception {
     Client client = createClient();
     try {
-      int limit = 10;
-      String queryParam = "?limit=" + limit;
       String resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/"
           + "timeline/clusters/cluster1/apps/application_1111111111_1111/"
           + "entities/entitytype";
-      URI uri = URI.create(resourceUri + queryParam);
-
-      ClientResponse resp = getResponse(client, uri);
-      List<TimelineEntity> entities =
-          resp.getEntity(new GenericType<List<TimelineEntity>>() {
-          });
-      // verify for entity-10 to entity-1 in descending order.
-      verifyPaginatedEntites(entities, limit, limit);
-
-      limit = 4;
-      queryParam = "?limit=" + limit;
-      uri = URI.create(resourceUri + queryParam);
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
-      });
-      // verify for entity-10 to entity-7 in descending order.
-      TimelineEntity entity = verifyPaginatedEntites(entities, limit, 10);
-
-      queryParam = "?limit=" + limit + "&fromid="
-          + entity.getInfo().get(TimelineReaderUtils.FROMID_KEY);
-      uri = URI.create(resourceUri + queryParam);
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
-      });
-      // verify for entity-7 to entity-4 in descending order.
-      entity = verifyPaginatedEntites(entities, limit, 7);
-
-      queryParam = "?limit=" + limit + "&fromid="
-          + entity.getInfo().get(TimelineReaderUtils.FROMID_KEY);
-      uri = URI.create(resourceUri + queryParam);
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
-      });
-      // verify for entity-4 to entity-1 in descending order.
-      entity = verifyPaginatedEntites(entities, limit, 4);
-
-      queryParam = "?limit=" + limit + "&fromid="
-          + entity.getInfo().get(TimelineReaderUtils.FROMID_KEY);
-      uri = URI.create(resourceUri + queryParam);
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
-      });
-      // always entity-1 will be retrieved
-      entity = verifyPaginatedEntites(entities, 1, 1);
+      verifyEntitiesForPagination(client, resourceUri);
+      resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/"
+          + "timeline/clusters/cluster1/users/" + doAsUser
+          + "/entities/entitytype";
+      verifyEntitiesForPagination(client, resourceUri);
     } finally {
       client.destroy();
     }
   }
 
+  private void verifyEntitiesForPagination(Client client, String resourceUri)
+      throws Exception {
+    int limit = 10;
+    String queryParam = "?limit=" + limit;
+    URI uri = URI.create(resourceUri + queryParam);
+
+    ClientResponse resp = getResponse(client, uri);
+    List<TimelineEntity> entities =
+        resp.getEntity(new GenericType<List<TimelineEntity>>() {
+        });
+    // verify for entity-10 to entity-1 in descending order.
+    verifyPaginatedEntites(entities, limit, limit);
+
+    limit = 4;
+    queryParam = "?limit=" + limit;
+    uri = URI.create(resourceUri + queryParam);
+    resp = getResponse(client, uri);
+    entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
+    });
+    // verify for entity-10 to entity-7 in descending order.
+    TimelineEntity entity = verifyPaginatedEntites(entities, limit, 10);
+
+    queryParam = "?limit=" + limit + "&fromid="
+        + entity.getInfo().get(TimelineReaderUtils.FROMID_KEY);
+    uri = URI.create(resourceUri + queryParam);
+    resp = getResponse(client, uri);
+    entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
+    });
+    // verify for entity-7 to entity-4 in descending order.
+    entity = verifyPaginatedEntites(entities, limit, 7);
+
+    queryParam = "?limit=" + limit + "&fromid="
+        + entity.getInfo().get(TimelineReaderUtils.FROMID_KEY);
+    uri = URI.create(resourceUri + queryParam);
+    resp = getResponse(client, uri);
+    entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
+    });
+    // verify for entity-4 to entity-1 in descending order.
+    entity = verifyPaginatedEntites(entities, limit, 4);
+
+    queryParam = "?limit=" + limit + "&fromid="
+        + entity.getInfo().get(TimelineReaderUtils.FROMID_KEY);
+    uri = URI.create(resourceUri + queryParam);
+    resp = getResponse(client, uri);
+    entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
+    });
+    // always entity-1 will be retrieved
+    entity = verifyPaginatedEntites(entities, 1, 1);
+  }
+
   private TimelineEntity verifyPaginatedEntites(List<TimelineEntity> entities,
       int limit, int startFrom) {
     assertNotNull(entities);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73ee0d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java
index d7aca74..3a44445 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java
@@ -135,7 +135,7 @@ class GenericEntityReader extends TimelineEntityReader {
    *
    * @return true if we need to fetch some of the columns, false otherwise.
    */
-  private boolean fetchPartialEventCols(TimelineFilterList eventFilters,
+  protected boolean fetchPartialEventCols(TimelineFilterList eventFilters,
       EnumSet<Field> fieldsToRetrieve) {
     return (eventFilters != null && !eventFilters.getFilterList().isEmpty() &&
         !hasField(fieldsToRetrieve, Field.EVENTS));
@@ -146,7 +146,7 @@ class GenericEntityReader extends TimelineEntityReader {
    *
    * @return true if we need to fetch some of the columns, false otherwise.
    */
-  private boolean fetchPartialRelatesToCols(TimelineFilterList relatesTo,
+  protected boolean fetchPartialRelatesToCols(TimelineFilterList relatesTo,
       EnumSet<Field> fieldsToRetrieve) {
     return (relatesTo != null && !relatesTo.getFilterList().isEmpty() &&
         !hasField(fieldsToRetrieve, Field.RELATES_TO));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73ee0d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/SubApplicationEntityReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/SubApplicationEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/SubApplicationEntityReader.java
new file mode 100644
index 0000000..e780dcc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/SubApplicationEntityReader.java
@@ -0,0 +1,488 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Query;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.FamilyFilter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.FilterList.Operator;
+import org.apache.hadoop.hbase.filter.PageFilter;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKeyPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTable;
+import org.apache.hadoop.yarn.webapp.BadRequestException;
+
+import com.google.common.base.Preconditions;
+
+class SubApplicationEntityReader extends GenericEntityReader {
+  private static final SubApplicationTable SUB_APPLICATION_TABLE =
+      new SubApplicationTable();
+
+  SubApplicationEntityReader(TimelineReaderContext ctxt,
+      TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) {
+    super(ctxt, entityFilters, toRetrieve);
+  }
+
+  SubApplicationEntityReader(TimelineReaderContext ctxt,
+      TimelineDataToRetrieve toRetrieve) {
+    super(ctxt, toRetrieve);
+  }
+
+  /**
+   * Uses the {@link SubApplicationTable}.
+   */
+  protected BaseTable<?> getTable() {
+    return SUB_APPLICATION_TABLE;
+  }
+
+  @Override
+  protected FilterList constructFilterListBasedOnFilters() throws IOException {
+    // Filters here cannot be null for multiple entity reads as they are set in
+    // augmentParams if null.
+    FilterList listBasedOnFilters = new FilterList();
+    TimelineEntityFilters filters = getFilters();
+    // Create filter list based on created time range and add it to
+    // listBasedOnFilters.
+    long createdTimeBegin = filters.getCreatedTimeBegin();
+    long createdTimeEnd = filters.getCreatedTimeEnd();
+    if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) {
+      listBasedOnFilters.addFilter(TimelineFilterUtils
+          .createSingleColValueFiltersByRange(SubApplicationColumn.CREATED_TIME,
+              createdTimeBegin, createdTimeEnd));
+    }
+    // Create filter list based on metric filters and add it to
+    // listBasedOnFilters.
+    TimelineFilterList metricFilters = filters.getMetricFilters();
+    if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) {
+      listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList(
+          SubApplicationColumnPrefix.METRIC, metricFilters));
+    }
+    // Create filter list based on config filters and add it to
+    // listBasedOnFilters.
+    TimelineFilterList configFilters = filters.getConfigFilters();
+    if (configFilters != null && !configFilters.getFilterList().isEmpty()) {
+      listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList(
+          SubApplicationColumnPrefix.CONFIG, configFilters));
+    }
+    // Create filter list based on info filters and add it to listBasedOnFilters
+    TimelineFilterList infoFilters = filters.getInfoFilters();
+    if (infoFilters != null && !infoFilters.getFilterList().isEmpty()) {
+      listBasedOnFilters.addFilter(TimelineFilterUtils
+          .createHBaseFilterList(SubApplicationColumnPrefix.INFO, infoFilters));
+    }
+    return listBasedOnFilters;
+  }
+
+  /**
+   * Add {@link QualifierFilter} filters to filter list for each column of
+   * entity table.
+   *
+   * @param list filter list to which qualifier filters have to be added.
+   */
+  protected void updateFixedColumns(FilterList list) {
+    for (SubApplicationColumn column : SubApplicationColumn.values()) {
+      list.addFilter(new QualifierFilter(CompareOp.EQUAL,
+          new BinaryComparator(column.getColumnQualifierBytes())));
+    }
+  }
+
+  /**
+   * Creates a filter list which indicates that only some of the column
+   * qualifiers in the info column family will be returned in result.
+   *
+   * @param isApplication If true, it means operations are to be performed for
+   *          application table, otherwise for entity table.
+   * @return filter list.
+   * @throws IOException if any problem occurs while creating filter list.
+   */
+  private FilterList createFilterListForColsOfInfoFamily() throws IOException {
+    FilterList infoFamilyColsFilter = new FilterList(Operator.MUST_PASS_ONE);
+    // Add filters for each column in entity table.
+    updateFixedColumns(infoFamilyColsFilter);
+    EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
+    // If INFO field has to be retrieved, add a filter for fetching columns
+    // with INFO column prefix.
+    if (hasField(fieldsToRetrieve, Field.INFO)) {
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.EQUAL,
+              SubApplicationColumnPrefix.INFO));
+    }
+    TimelineFilterList relatesTo = getFilters().getRelatesTo();
+    if (hasField(fieldsToRetrieve, Field.RELATES_TO)) {
+      // If RELATES_TO field has to be retrieved, add a filter for fetching
+      // columns with RELATES_TO column prefix.
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.EQUAL,
+              SubApplicationColumnPrefix.RELATES_TO));
+    } else if (relatesTo != null && !relatesTo.getFilterList().isEmpty()) {
+      // Even if fields to retrieve does not contain RELATES_TO, we still
+      // need to have a filter to fetch some of the column qualifiers if
+      // relatesTo filters are specified. relatesTo filters will then be
+      // matched after fetching rows from HBase.
+      Set<String> relatesToCols =
+          TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo);
+      infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
+          SubApplicationColumnPrefix.RELATES_TO, relatesToCols));
+    }
+    TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo();
+    if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
+      // If IS_RELATED_TO field has to be retrieved, add a filter for fetching
+      // columns with IS_RELATED_TO column prefix.
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.EQUAL,
+              SubApplicationColumnPrefix.IS_RELATED_TO));
+    } else if (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty()) {
+      // Even if fields to retrieve does not contain IS_RELATED_TO, we still
+      // need to have a filter to fetch some of the column qualifiers if
+      // isRelatedTo filters are specified. isRelatedTo filters will then be
+      // matched after fetching rows from HBase.
+      Set<String> isRelatedToCols =
+          TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo);
+      infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
+          SubApplicationColumnPrefix.IS_RELATED_TO, isRelatedToCols));
+    }
+    TimelineFilterList eventFilters = getFilters().getEventFilters();
+    if (hasField(fieldsToRetrieve, Field.EVENTS)) {
+      // If EVENTS field has to be retrieved, add a filter for fetching columns
+      // with EVENT column prefix.
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.EQUAL,
+              SubApplicationColumnPrefix.EVENT));
+    } else if (eventFilters != null
+        && !eventFilters.getFilterList().isEmpty()) {
+      // Even if fields to retrieve does not contain EVENTS, we still need to
+      // have a filter to fetch some of the column qualifiers on the basis of
+      // event filters specified. Event filters will then be matched after
+      // fetching rows from HBase.
+      Set<String> eventCols =
+          TimelineFilterUtils.fetchColumnsFromFilterList(eventFilters);
+      infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
+          SubApplicationColumnPrefix.EVENT, eventCols));
+    }
+    return infoFamilyColsFilter;
+  }
+
+  /**
+   * Exclude column prefixes via filters which are not required(based on fields
+   * to retrieve) from info column family. These filters are added to filter
+   * list which contains a filter for getting info column family.
+   *
+   * @param infoColFamilyList filter list for info column family.
+   */
+  private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) {
+    EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
+    // Events not required.
+    if (!hasField(fieldsToRetrieve, Field.EVENTS)) {
+      infoColFamilyList.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
+              SubApplicationColumnPrefix.EVENT));
+    }
+    // info not required.
+    if (!hasField(fieldsToRetrieve, Field.INFO)) {
+      infoColFamilyList.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
+              SubApplicationColumnPrefix.INFO));
+    }
+    // is related to not required.
+    if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
+      infoColFamilyList.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
+              SubApplicationColumnPrefix.IS_RELATED_TO));
+    }
+    // relates to not required.
+    if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) {
+      infoColFamilyList.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
+              SubApplicationColumnPrefix.RELATES_TO));
+    }
+  }
+
+  /**
+   * Updates filter list based on fields for confs and metrics to retrieve.
+   *
+   * @param listBasedOnFields filter list based on fields.
+   * @throws IOException if any problem occurs while updating filter list.
+   */
+  private void updateFilterForConfsAndMetricsToRetrieve(
+      FilterList listBasedOnFields) throws IOException {
+    TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
+    // Please note that if confsToRetrieve is specified, we would have added
+    // CONFS to fields to retrieve in augmentParams() even if not specified.
+    if (dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS)) {
+      // Create a filter list for configs.
+      listBasedOnFields.addFilter(
+          TimelineFilterUtils.createFilterForConfsOrMetricsToRetrieve(
+              dataToRetrieve.getConfsToRetrieve(),
+              SubApplicationColumnFamily.CONFIGS,
+              SubApplicationColumnPrefix.CONFIG));
+    }
+
+    // Please note that if metricsToRetrieve is specified, we would have added
+    // METRICS to fields to retrieve in augmentParams() even if not specified.
+    if (dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS)) {
+      // Create a filter list for metrics.
+      listBasedOnFields.addFilter(
+          TimelineFilterUtils.createFilterForConfsOrMetricsToRetrieve(
+              dataToRetrieve.getMetricsToRetrieve(),
+              SubApplicationColumnFamily.METRICS,
+              SubApplicationColumnPrefix.METRIC));
+    }
+  }
+
+  @Override
+  protected FilterList constructFilterListBasedOnFields() throws IOException {
+    if (!needCreateFilterListBasedOnFields()) {
+      // Fetch all the columns. No need of a filter.
+      return null;
+    }
+    FilterList listBasedOnFields = new FilterList(Operator.MUST_PASS_ONE);
+    FilterList infoColFamilyList = new FilterList();
+    // By default fetch everything in INFO column family.
+    FamilyFilter infoColumnFamily = new FamilyFilter(CompareOp.EQUAL,
+        new BinaryComparator(SubApplicationColumnFamily.INFO.getBytes()));
+    infoColFamilyList.addFilter(infoColumnFamily);
+    if (fetchPartialColsFromInfoFamily()) {
+      // We can fetch only some of the columns from info family.
+      infoColFamilyList.addFilter(createFilterListForColsOfInfoFamily());
+    } else {
+      // Exclude column prefixes in info column family which are not required
+      // based on fields to retrieve.
+      excludeFieldsFromInfoColFamily(infoColFamilyList);
+    }
+    listBasedOnFields.addFilter(infoColFamilyList);
+    updateFilterForConfsAndMetricsToRetrieve(listBasedOnFields);
+    return listBasedOnFields;
+  }
+
+  @Override
+  protected void validateParams() {
+    Preconditions.checkNotNull(getContext(), "context shouldn't be null");
+    Preconditions.checkNotNull(getDataToRetrieve(),
+        "data to retrieve shouldn't be null");
+    Preconditions.checkNotNull(getContext().getClusterId(),
+        "clusterId shouldn't be null");
+    Preconditions.checkNotNull(getContext().getDoAsUser(),
+        "DoAsUser shouldn't be null");
+    Preconditions.checkNotNull(getContext().getEntityType(),
+        "entityType shouldn't be null");
+  }
+
+  @Override
+  protected void augmentParams(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve();
+    createFiltersIfNull();
+  }
+
+  private void setMetricsTimeRange(Query query) {
+    // Set time range for metric values.
+    HBaseTimelineStorageUtils.setMetricsTimeRange(query,
+        SubApplicationColumnFamily.METRICS.getBytes(),
+        getDataToRetrieve().getMetricsTimeBegin(),
+        getDataToRetrieve().getMetricsTimeEnd());
+  }
+
+  @Override
+  protected ResultScanner getResults(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException {
+
+    // Scan through part of the table to find the entities belong to one app
+    // and one type
+    Scan scan = new Scan();
+    TimelineReaderContext context = getContext();
+    if (context.getDoAsUser() == null) {
+      throw new BadRequestException("Invalid user!");
+    }
+
+    RowKeyPrefix<SubApplicationRowKey> subApplicationRowKeyPrefix = null;
+    // default mode, will always scans from beginning of entity type.
+    if (getFilters() == null || getFilters().getFromId() == null) {
+      subApplicationRowKeyPrefix = new SubApplicationRowKeyPrefix(
+          context.getDoAsUser(), context.getClusterId(),
+          context.getEntityType(), null, null, null);
+      scan.setRowPrefixFilter(subApplicationRowKeyPrefix.getRowKeyPrefix());
+    } else { // pagination mode, will scan from given entityIdPrefix!enitityId
+
+      SubApplicationRowKey entityRowKey = null;
+      try {
+        entityRowKey = SubApplicationRowKey
+            .parseRowKeyFromString(getFilters().getFromId());
+      } catch (IllegalArgumentException e) {
+        throw new BadRequestException("Invalid filter fromid is provided.");
+      }
+      if (!context.getClusterId().equals(entityRowKey.getClusterId())) {
+        throw new BadRequestException(
+            "fromid doesn't belong to clusterId=" + context.getClusterId());
+      }
+
+      // set start row
+      scan.setStartRow(entityRowKey.getRowKey());
+
+      // get the bytes for stop row
+      subApplicationRowKeyPrefix = new SubApplicationRowKeyPrefix(
+          context.getDoAsUser(), context.getClusterId(),
+          context.getEntityType(), null, null, null);
+
+      // set stop row
+      scan.setStopRow(
+          HBaseTimelineStorageUtils.calculateTheClosestNextRowKeyForPrefix(
+              subApplicationRowKeyPrefix.getRowKeyPrefix()));
+
+      // set page filter to limit. This filter has to set only in pagination
+      // mode.
+      filterList.addFilter(new PageFilter(getFilters().getLimit()));
+    }
+    setMetricsTimeRange(scan);
+    scan.setMaxVersions(getDataToRetrieve().getMetricsLimit());
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      scan.setFilter(filterList);
+    }
+    return getTable().getResultScanner(hbaseConf, conn, scan);
+  }
+
+  @Override
+  protected Result getResult(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException {
+    throw new UnsupportedOperationException(
+        "we don't support a single entity query");
+  }
+
+  @Override
+  protected TimelineEntity parseEntity(Result result) throws IOException {
+    if (result == null || result.isEmpty()) {
+      return null;
+    }
+    TimelineEntity entity = new TimelineEntity();
+    SubApplicationRowKey parseRowKey =
+        SubApplicationRowKey.parseRowKey(result.getRow());
+    entity.setType(parseRowKey.getEntityType());
+    entity.setId(parseRowKey.getEntityId());
+    entity.setIdPrefix(parseRowKey.getEntityIdPrefix().longValue());
+
+    TimelineEntityFilters filters = getFilters();
+    // fetch created time
+    Long createdTime =
+        (Long) SubApplicationColumn.CREATED_TIME.readResult(result);
+    entity.setCreatedTime(createdTime);
+
+    EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
+    // fetch is related to entities and match isRelatedTo filter. If isRelatedTo
+    // filters do not match, entity would be dropped. We have to match filters
+    // locally as relevant HBase filters to filter out rows on the basis of
+    // isRelatedTo are not set in HBase scan.
+    boolean checkIsRelatedTo =
+        filters.getIsRelatedTo() != null
+            && filters.getIsRelatedTo().getFilterList().size() > 0;
+    if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) {
+      readRelationship(entity, result, SubApplicationColumnPrefix.IS_RELATED_TO,
+          true);
+      if (checkIsRelatedTo && !TimelineStorageUtils.matchIsRelatedTo(entity,
+          filters.getIsRelatedTo())) {
+        return null;
+      }
+      if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
+        entity.getIsRelatedToEntities().clear();
+      }
+    }
+
+    // fetch relates to entities and match relatesTo filter. If relatesTo
+    // filters do not match, entity would be dropped. We have to match filters
+    // locally as relevant HBase filters to filter out rows on the basis of
+    // relatesTo are not set in HBase scan.
+    boolean checkRelatesTo =
+        !isSingleEntityRead() && filters.getRelatesTo() != null
+            && filters.getRelatesTo().getFilterList().size() > 0;
+    if (hasField(fieldsToRetrieve, Field.RELATES_TO) || checkRelatesTo) {
+      readRelationship(entity, result, SubApplicationColumnPrefix.RELATES_TO,
+          false);
+      if (checkRelatesTo && !TimelineStorageUtils.matchRelatesTo(entity,
+          filters.getRelatesTo())) {
+        return null;
+      }
+      if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) {
+        entity.getRelatesToEntities().clear();
+      }
+    }
+
+    // fetch info if fieldsToRetrieve contains INFO or ALL.
+    if (hasField(fieldsToRetrieve, Field.INFO)) {
+      readKeyValuePairs(entity, result, SubApplicationColumnPrefix.INFO, false);
+    }
+
+    // fetch configs if fieldsToRetrieve contains CONFIGS or ALL.
+    if (hasField(fieldsToRetrieve, Field.CONFIGS)) {
+      readKeyValuePairs(entity, result, SubApplicationColumnPrefix.CONFIG,
+          true);
+    }
+
+    // fetch events and match event filters if they exist. If event filters do
+    // not match, entity would be dropped. We have to match filters locally
+    // as relevant HBase filters to filter out rows on the basis of events
+    // are not set in HBase scan.
+    boolean checkEvents =
+        !isSingleEntityRead() && filters.getEventFilters() != null
+            && filters.getEventFilters().getFilterList().size() > 0;
+    if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) {
+      readEvents(entity, result, SubApplicationColumnPrefix.EVENT);
+      if (checkEvents && !TimelineStorageUtils.matchEventFilters(entity,
+          filters.getEventFilters())) {
+        return null;
+      }
+      if (!hasField(fieldsToRetrieve, Field.EVENTS)) {
+        entity.getEvents().clear();
+      }
+    }
+
+    // fetch metrics if fieldsToRetrieve contains METRICS or ALL.
+    if (hasField(fieldsToRetrieve, Field.METRICS)) {
+      readMetrics(entity, result, SubApplicationColumnPrefix.METRIC);
+    }
+
+    entity.getInfo().put(TimelineReaderUtils.FROMID_KEY,
+        parseRowKey.getRowKeyAsString());
+    return entity;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73ee0d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java
index 16fffa4..fa16077 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java
@@ -82,6 +82,9 @@ public final class TimelineEntityReaderFactory {
         YARN_FLOW_RUN.matches(context.getEntityType())) {
       return new FlowRunEntityReader(context, filters, dataToRetrieve);
     } else {
+      if (context.getDoAsUser() != null) {
+        return new SubApplicationEntityReader(context, filters, dataToRetrieve);
+      }
       // assume we're dealing with a generic entity read
       return new GenericEntityReader(context, filters, dataToRetrieve);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73ee0d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java
index 5f308cb..67c3d29 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderContext.java
@@ -32,6 +32,7 @@ public class TimelineReaderContext extends TimelineContext {
   private String entityType;
   private String entityId;
   private Long entityIdPrefix;
+  private String doAsUser;
   public TimelineReaderContext(String clusterId, String userId, String flowName,
       Long flowRunId, String appId, String entityType, String entityId) {
     super(clusterId, userId, flowName, flowRunId, appId);
@@ -46,10 +47,18 @@ public class TimelineReaderContext extends TimelineContext {
     this.entityIdPrefix = entityIdPrefix;
   }
 
+  public TimelineReaderContext(String clusterId, String userId, String flowName,
+      Long flowRunId, String appId, String entityType, Long entityIdPrefix,
+      String entityId, String doasUser) {
+    this(clusterId, userId, flowName, flowRunId, appId, entityType, entityId);
+    this.entityIdPrefix = entityIdPrefix;
+    this.doAsUser = doasUser;
+  }
+
   public TimelineReaderContext(TimelineReaderContext other) {
     this(other.getClusterId(), other.getUserId(), other.getFlowName(),
         other.getFlowRunId(), other.getAppId(), other.getEntityType(),
-        other.getEntityIdPrefix(), other.getEntityId());
+        other.getEntityIdPrefix(), other.getEntityId(), other.getDoAsUser());
   }
 
   @Override
@@ -113,4 +122,12 @@ public class TimelineReaderContext extends TimelineContext {
   public void setEntityIdPrefix(Long entityIdPrefix) {
     this.entityIdPrefix = entityIdPrefix;
   }
+
+  public String getDoAsUser() {
+    return doAsUser;
+  }
+
+  public void setDoAsUser(String doAsUser) {
+    this.doAsUser = doAsUser;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73ee0d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java
index ee827da..67e5849 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderManager.java
@@ -111,8 +111,13 @@ public class TimelineReaderManager extends AbstractService {
     context.setEntityType(entity.getType());
     context.setEntityIdPrefix(entity.getIdPrefix());
     context.setEntityId(entity.getId());
-    entity.setUID(TimelineReaderUtils.UID_KEY,
-        TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(context));
+    if (context.getDoAsUser() != null) {
+      entity.setUID(TimelineReaderUtils.UID_KEY,
+          TimelineUIDConverter.SUB_APPLICATION_ENTITY_UID.encodeUID(context));
+    } else {
+      entity.setUID(TimelineReaderUtils.UID_KEY,
+          TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(context));
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73ee0d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
index eba8f56..6798930 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServices.java
@@ -3245,4 +3245,162 @@ public class TimelineReaderWebServices {
         " (Took " + (endTime - startTime) + " ms.)");
     return results;
   }
+
+  @GET
+  @Path("/users/{userid}/entities/{entitytype}")
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
+  public Set<TimelineEntity> getSubAppEntities(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("userid") String userId,
+      @PathParam("entitytype") String entityType,
+      @QueryParam("limit") String limit,
+      @QueryParam("createdtimestart") String createdTimeStart,
+      @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("relatesto") String relatesTo,
+      @QueryParam("isrelatedto") String isRelatedTo,
+      @QueryParam("infofilters") String infofilters,
+      @QueryParam("conffilters") String conffilters,
+      @QueryParam("metricfilters") String metricfilters,
+      @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("metricstimestart") String metricsTimeStart,
+      @QueryParam("metricstimeend") String metricsTimeEnd,
+      @QueryParam("fromid") String fromId) {
+    return getSubAppEntities(req, res, null, userId, entityType, limit,
+        createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo, infofilters,
+        conffilters, metricfilters, eventfilters, confsToRetrieve,
+        metricsToRetrieve, fields, metricsLimit, metricsTimeStart,
+        metricsTimeEnd, fromId);
+  }
+
+  @GET
+  @Path("/clusters/{clusterid}/users/{userid}/entities/{entitytype}")
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
+  public Set<TimelineEntity> getSubAppEntities(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("clusterid") String clusterId,
+      @PathParam("userid") String userId,
+      @PathParam("entitytype") String entityType,
+      @QueryParam("limit") String limit,
+      @QueryParam("createdtimestart") String createdTimeStart,
+      @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("relatesto") String relatesTo,
+      @QueryParam("isrelatedto") String isRelatedTo,
+      @QueryParam("infofilters") String infofilters,
+      @QueryParam("conffilters") String conffilters,
+      @QueryParam("metricfilters") String metricfilters,
+      @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("metricstimestart") String metricsTimeStart,
+      @QueryParam("metricstimeend") String metricsTimeEnd,
+      @QueryParam("fromid") String fromId) {
+    String url = req.getRequestURI() +
+        (req.getQueryString() == null ? "" :
+            QUERY_STRING_SEP + req.getQueryString());
+    UserGroupInformation callerUGI =
+        TimelineReaderWebServicesUtils.getUser(req);
+    LOG.info("Received URL " + url + " from user " +
+        TimelineReaderWebServicesUtils.getUserName(callerUGI));
+    long startTime = Time.monotonicNow();
+    init(res);
+    TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
+    Set<TimelineEntity> entities = null;
+    try {
+      TimelineReaderContext context =
+          TimelineReaderWebServicesUtils.createTimelineReaderContext(clusterId,
+              null, null, null, null, entityType, null, null, userId);
+      entities = timelineReaderManager.getEntities(context,
+          TimelineReaderWebServicesUtils.createTimelineEntityFilters(
+          limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
+              infofilters, conffilters, metricfilters, eventfilters,
+              fromId),
+          TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+          metricsTimeStart, metricsTimeEnd));
+    } catch (Exception e) {
+      handleException(e, url, startTime,
+          "createdTime start/end or limit");
+    }
+    long endTime = Time.monotonicNow();
+    if (entities == null) {
+      entities = Collections.emptySet();
+    }
+    LOG.info("Processed URL " + url +
+        " (Took " + (endTime - startTime) + " ms.)");
+    return entities;
+  }
+
+  @GET
+  @Path("/users/{userid}/entities/{entitytype}/{entityid}")
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
+  public Set<TimelineEntity> getSubAppEntities(@Context HttpServletRequest req,
+      @Context HttpServletResponse res, @PathParam("userid") String userId,
+      @PathParam("entitytype") String entityType,
+      @PathParam("entityid") String entityId,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("metricstimestart") String metricsTimeStart,
+      @QueryParam("metricstimeend") String metricsTimeEnd,
+      @QueryParam("entityidprefix") String entityIdPrefix) {
+    return getSubAppEntities(req, res, null, userId, entityType, entityId,
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+        metricsTimeStart, metricsTimeEnd, entityIdPrefix);
+  }
+
+  @GET
+  @Path("/clusters/{clusterid}/users/{userid}/entities/{entitytype}/{entityid}")
+  @Produces(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8)
+  public Set<TimelineEntity> getSubAppEntities(@Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("clusterid") String clusterId,
+      @PathParam("userid") String userId,
+      @PathParam("entitytype") String entityType,
+      @PathParam("entityid") String entityId,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("metricstimestart") String metricsTimeStart,
+      @QueryParam("metricstimeend") String metricsTimeEnd,
+      @QueryParam("entityidprefix") String entityIdPrefix) {
+    String url = req.getRequestURI() + (req.getQueryString() == null ? ""
+        : QUERY_STRING_SEP + req.getQueryString());
+    UserGroupInformation callerUGI =
+        TimelineReaderWebServicesUtils.getUser(req);
+    LOG.info("Received URL " + url + " from user "
+        + TimelineReaderWebServicesUtils.getUserName(callerUGI));
+    long startTime = Time.monotonicNow();
+    init(res);
+    TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
+    Set<TimelineEntity> entities = null;
+    try {
+      TimelineReaderContext context = TimelineReaderWebServicesUtils
+          .createTimelineReaderContext(clusterId, null, null, null, null,
+              entityType, entityIdPrefix, entityId, userId);
+      entities = timelineReaderManager.getEntities(context,
+          new TimelineEntityFilters.Builder().build(),
+          TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
+              confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+              metricsTimeStart, metricsTimeEnd));
+    } catch (Exception e) {
+      handleException(e, url, startTime, "");
+    }
+    long endTime = Time.monotonicNow();
+    if (entities == null) {
+      entities = Collections.emptySet();
+    }
+    LOG.info(
+        "Processed URL " + url + " (Took " + (endTime - startTime) + " ms.)");
+    return entities;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73ee0d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
index d613eab..f83c1ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
@@ -57,6 +57,16 @@ public final class TimelineReaderWebServicesUtils {
         parseStr(entityType), parseLongStr(entityIdPrefix), parseStr(entityId));
   }
 
+  static TimelineReaderContext createTimelineReaderContext(String clusterId,
+      String userId, String flowName, String flowRunId, String appId,
+      String entityType, String entityIdPrefix, String entityId,
+      String doAsUser) {
+    return new TimelineReaderContext(parseStr(clusterId), parseStr(userId),
+        parseStr(flowName), parseLongStr(flowRunId), parseStr(appId),
+        parseStr(entityType), parseLongStr(entityIdPrefix), parseStr(entityId),
+        parseStr(doAsUser));
+  }
+
   /**
    * Parse the passed filters represented as strings and convert them into a
    * {@link TimelineEntityFilters} object.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73ee0d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineUIDConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineUIDConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineUIDConverter.java
index 52e24e1..b875828 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineUIDConverter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineUIDConverter.java
@@ -137,6 +137,41 @@ enum TimelineUIDConverter {
     }
   },
 
+  // Sub Application Entity UID should contain cluster, user, entity type and
+  // entity id
+  SUB_APPLICATION_ENTITY_UID {
+    @Override
+    String encodeUID(TimelineReaderContext context) {
+      if (context == null) {
+        return null;
+      }
+      if (context.getClusterId() == null || context.getDoAsUser() == null
+          || context.getEntityType() == null || context.getEntityId() == null) {
+        return null;
+      }
+      String[] entityTupleArr = {context.getClusterId(), context.getDoAsUser(),
+          context.getEntityType(), context.getEntityIdPrefix().toString(),
+          context.getEntityId()};
+      return joinAndEscapeUIDParts(entityTupleArr);
+    }
+
+    @Override
+    TimelineReaderContext decodeUID(String uId) throws Exception {
+      if (uId == null) {
+        return null;
+      }
+      List<String> entityTupleList = splitUID(uId);
+      if (entityTupleList.size() == 5) {
+        // Flow information exists.
+        return new TimelineReaderContext(entityTupleList.get(0), null, null,
+            null, null, entityTupleList.get(2),
+            Long.parseLong(entityTupleList.get(3)), entityTupleList.get(4),
+            entityTupleList.get(1));
+      }
+      return null;
+    }
+  },
+
   // Generic Entity UID should contain cluster, user, flow name, flowrun id,
   // app id, entity type and entity id OR should contain cluster, appid, entity
   // type and entity id(i.e.without flow context info).

http://git-wip-us.apache.org/repos/asf/hadoop/blob/73ee0d4f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java
index 11dc913..12b3fc0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineUIDConverter.java
@@ -68,6 +68,15 @@ public class TestTimelineUIDConverter {
         "54321!container_1111111111_1111_01_000001", uid);
     assertEquals(
         context, TimelineUIDConverter.GENERIC_ENTITY_UID.decodeUID(uid));
+
+    context = new TimelineReaderContext("yarn_cluster", null, null, null, null,
+        "YARN_CONTAINER", 54321L, "container_1111111111_1111_01_000001",
+        "user1");
+    uid = TimelineUIDConverter.SUB_APPLICATION_ENTITY_UID.encodeUID(context);
+    assertEquals("yarn_cluster!user1!YARN_CONTAINER!"
+        + "54321!container_1111111111_1111_01_000001", uid);
+    assertEquals(context,
+        TimelineUIDConverter.SUB_APPLICATION_ENTITY_UID.decodeUID(uid));
   }
 
   @Test


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org