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/03 06:57:38 UTC

[01/50] [abbrv] hadoop git commit: YARN-6064. Support fromId for flowRuns and flow/flowRun apps REST API's (Rohith Sharma K S via Varun Saxena)

Repository: hadoop
Updated Branches:
  refs/heads/YARN-5355-branch-2 [deleted] 6d72ddab0
  refs/heads/YARN-5355_branch2 [created] 6d72ddab0


YARN-6064. Support fromId for flowRuns and flow/flowRun apps REST API's (Rohith Sharma K S via Varun Saxena)

(cherry picked from commit 4b1ba4ea314147f8a06cc4f446c1d9336de89fc1)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 5d27b1ca24d32724dd75c19a1654fbe9c7d45ccc
Parents: 44216a4
Author: Varun Saxena <va...@apache.org>
Authored: Wed Jan 18 10:30:15 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Tue Apr 25 23:14:28 2017 +0530

----------------------------------------------------------------------
 ...stTimelineReaderWebServicesHBaseStorage.java | 256 +++++++++++++++----
 .../reader/TimelineReaderWebServices.java       |  72 ++++--
 .../storage/reader/ApplicationEntityReader.java |  43 +++-
 .../storage/reader/FlowRunEntityReader.java     |  29 ++-
 4 files changed, 327 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d27b1ca/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 fa35fc5..485c191 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
@@ -40,6 +40,8 @@ import javax.ws.rs.core.MediaType;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
@@ -53,6 +55,7 @@ import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
 import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -352,6 +355,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           flowVersion2, runid2, entity3.getId(), te3);
       hbi.write(cluster, user, flow, flowVersion, runid,
           "application_1111111111_1111", userEntities);
+      writeApplicationEntities(hbi);
       hbi.flush();
     } finally {
       if (hbi != null) {
@@ -360,6 +364,35 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     }
   }
 
+  static void writeApplicationEntities(HBaseTimelineWriterImpl hbi)
+      throws IOException {
+    long currentTimeMillis = System.currentTimeMillis();
+    int count = 1;
+    for (long i = 1; i <= 3; i++) {
+      for (int j = 1; j <= 5; j++) {
+        TimelineEntities te = new TimelineEntities();
+        ApplicationId appId =
+            BuilderUtils.newApplicationId(currentTimeMillis, count++);
+        ApplicationEntity appEntity = new ApplicationEntity();
+        appEntity.setId(appId.toString());
+        appEntity.setCreatedTime(currentTimeMillis);
+
+        TimelineEvent created = new TimelineEvent();
+        created.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+        created.setTimestamp(currentTimeMillis);
+        appEntity.addEvent(created);
+        TimelineEvent finished = new TimelineEvent();
+        finished.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+        finished.setTimestamp(currentTimeMillis + i * j);
+
+        appEntity.addEvent(finished);
+        te.addEntity(appEntity);
+        hbi.write("cluster1", "user1", "flow1", "CF7022C10F1354", i,
+            appEntity.getId(), te);
+      }
+    }
+  }
+
   @AfterClass
   public static void tearDown() throws Exception {
     util.shutdownMiniCluster();
@@ -697,7 +730,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       Set<FlowActivityEntity> flowEntities =
           resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
       assertNotNull(flowEntities);
-      assertEquals(2, flowEntities.size());
+      assertEquals(3, flowEntities.size());
       List<String> listFlowUIDs = new ArrayList<String>();
       for (FlowActivityEntity entity : flowEntities) {
         String flowUID =
@@ -709,7 +742,9 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         assertTrue((entity.getId().endsWith("@flow_name") &&
             entity.getFlowRuns().size() == 2) ||
             (entity.getId().endsWith("@flow_name2") &&
-            entity.getFlowRuns().size() == 1));
+                entity.getFlowRuns().size() == 1)
+            || (entity.getId().endsWith("@flow1")
+                && entity.getFlowRuns().size() == 3));
       }
 
       // Query flowruns based on UID returned in query above.
@@ -731,7 +766,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
               flowRunUID);
         }
       }
-      assertEquals(3, listFlowRunUIDs.size());
+      assertEquals(6, listFlowRunUIDs.size());
 
       // Query single flowrun based on UIDs' returned in query to get flowruns.
       for (String flowRunUID : listFlowRunUIDs) {
@@ -763,7 +798,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
               context.getFlowRunId(), entity.getId(), null, null)), appUID);
         }
       }
-      assertEquals(4, listAppUIDs.size());
+      assertEquals(19, listAppUIDs.size());
 
       // Query single app based on UIDs' returned in query to get apps.
       for (String appUID : listAppUIDs) {
@@ -944,32 +979,20 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/clusters/cluster1/flows");
-      ClientResponse resp = getResponse(client, uri);
-      Set<FlowActivityEntity> entities =
-          resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-      for (FlowActivityEntity entity : entities) {
-        assertTrue((entity.getId().endsWith("@flow_name") &&
-            entity.getFlowRuns().size() == 2) ||
-            (entity.getId().endsWith("@flow_name2") &&
-            entity.getFlowRuns().size() == 1));
-      }
+
+      verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1},
+          new String[] {"flow1", "flow_name", "flow_name2"});
 
       // Query without specifying cluster ID.
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/flows/");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
+      verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1},
+          new String[] {"flow1", "flow_name", "flow_name2"});
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
               "timeline/clusters/cluster1/flows?limit=1");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
+      verifyFlowEntites(client, uri, 1, new int[] {3},
+          new String[] {"flow1"});
 
       long firstFlowActivity =
           HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(1425016501000L);
@@ -979,40 +1002,25 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           "timeline/clusters/cluster1/flows?daterange="
           + fmt.format(firstFlowActivity) + "-"
           + fmt.format(dayTs));
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-      for (FlowActivityEntity entity : entities) {
-        assertTrue((entity.getId().endsWith("@flow_name") &&
-            entity.getFlowRuns().size() == 2) ||
-            (entity.getId().endsWith("@flow_name2") &&
-            entity.getFlowRuns().size() == 1));
-      }
+      verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1},
+          new String[] {"flow1", "flow_name", "flow_name2"});
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/clusters/cluster1/flows?daterange=" +
           fmt.format(dayTs + (4*86400000L)));
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(0, entities.size());
+      verifyFlowEntites(client, uri, 0, new int[] {}, new String[] {});
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/clusters/cluster1/flows?daterange=-" +
           fmt.format(dayTs));
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
+      verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1},
+          new String[] {"flow1", "flow_name", "flow_name2"});
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/clusters/cluster1/flows?daterange=" +
            fmt.format(firstFlowActivity) + "-");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
+      verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1},
+          new String[] {"flow1", "flow_name", "flow_name2"});
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/clusters/cluster1/flows?daterange=20150711:20150714");
@@ -2242,4 +2250,162 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     }
     return entity;
   }
+
+  private void verifyFlowEntites(Client client, URI uri, int noOfEntities,
+      int[] a, String[] flowsInSequence) throws Exception {
+    ClientResponse resp = getResponse(client, uri);
+    List<FlowActivityEntity> entities =
+        resp.getEntity(new GenericType<List<FlowActivityEntity>>() {
+        });
+    assertNotNull(entities);
+    assertEquals(noOfEntities, entities.size());
+    assertEquals(noOfEntities, flowsInSequence.length);
+    assertEquals(noOfEntities, a.length);
+    int count = 0;
+    for (FlowActivityEntity timelineEntity : entities) {
+      assertEquals(flowsInSequence[count],
+          timelineEntity.getInfo().get("SYSTEM_INFO_FLOW_NAME"));
+      assertEquals(a[count++], timelineEntity.getFlowRuns().size());
+    }
+  }
+
+  @Test
+  public void testForFlowAppsPagination() throws Exception {
+    Client client = createClient();
+    try {
+      // app entities stored is 15 during initialization.
+      int totalAppEntities = 15;
+      String resourceUri = "http://localhost:" + serverPort + "/ws/v2/"
+          + "timeline/clusters/cluster1/users/user1/flows/flow1/apps";
+      URI uri = URI.create(resourceUri);
+      ClientResponse resp = getResponse(client, uri);
+      List<TimelineEntity> entities =
+          resp.getEntity(new GenericType<List<TimelineEntity>>() {
+          });
+      assertNotNull(entities);
+      assertEquals(totalAppEntities, entities.size());
+      TimelineEntity entity1 = entities.get(0);
+      TimelineEntity entity15 = entities.get(totalAppEntities - 1);
+
+      int limit = 10;
+      String queryParam = "?limit=" + limit;
+      uri = URI.create(resourceUri + queryParam);
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
+      });
+      assertNotNull(entities);
+      assertEquals(limit, entities.size());
+      assertEquals(entity1, entities.get(0));
+      TimelineEntity entity10 = entities.get(limit - 1);
+
+      uri =
+          URI.create(resourceUri + queryParam + "&fromid=" + entity10.getId());
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
+      });
+      assertNotNull(entities);
+      assertEquals(6, entities.size());
+      assertEquals(entity10, entities.get(0));
+      assertEquals(entity15, entities.get(5));
+
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testForFlowRunAppsPagination() throws Exception {
+    Client client = createClient();
+    try {
+      // app entities stored is 15 during initialization.
+      int totalAppEntities = 5;
+      String resourceUri = "http://localhost:" + serverPort + "/ws/v2/"
+          + "timeline/clusters/cluster1/users/user1/flows/flow1/runs/1/apps";
+      URI uri = URI.create(resourceUri);
+      ClientResponse resp = getResponse(client, uri);
+      List<TimelineEntity> entities =
+          resp.getEntity(new GenericType<List<TimelineEntity>>() {
+          });
+      assertNotNull(entities);
+      assertEquals(totalAppEntities, entities.size());
+      TimelineEntity entity1 = entities.get(0);
+      TimelineEntity entity5 = entities.get(totalAppEntities - 1);
+
+      int limit = 3;
+      String queryParam = "?limit=" + limit;
+      uri = URI.create(resourceUri + queryParam);
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
+      });
+      assertNotNull(entities);
+      assertEquals(limit, entities.size());
+      assertEquals(entity1, entities.get(0));
+      TimelineEntity entity3 = entities.get(limit - 1);
+
+      uri =
+          URI.create(resourceUri + queryParam + "&fromid=" + entity3.getId());
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
+      });
+      assertNotNull(entities);
+      assertEquals(3, entities.size());
+      assertEquals(entity3, entities.get(0));
+      assertEquals(entity5, entities.get(2));
+
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testForFlowRunsPagination() throws Exception {
+    Client client = createClient();
+    try {
+      // app entities stored is 15 during initialization.
+      int totalRuns = 3;
+      String resourceUri = "http://localhost:" + serverPort + "/ws/v2/"
+          + "timeline/clusters/cluster1/users/user1/flows/flow1/runs";
+      URI uri = URI.create(resourceUri);
+      ClientResponse resp = getResponse(client, uri);
+      List<TimelineEntity> entities =
+          resp.getEntity(new GenericType<List<TimelineEntity>>() {
+          });
+      assertNotNull(entities);
+      assertEquals(totalRuns, entities.size());
+      TimelineEntity entity1 = entities.get(0);
+      TimelineEntity entity3 = entities.get(totalRuns - 1);
+
+      int limit = 2;
+      String queryParam = "?limit=" + limit;
+      uri = URI.create(resourceUri + queryParam);
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
+      });
+      assertNotNull(entities);
+      assertEquals(limit, entities.size());
+      assertEquals(entity1, entities.get(0));
+      TimelineEntity entity2 = entities.get(limit - 1);
+
+      uri = URI.create(resourceUri + queryParam + "&fromid="
+          + entity2.getInfo().get("SYSTEM_INFO_FLOW_RUN_ID"));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
+      });
+      assertNotNull(entities);
+      assertEquals(limit, entities.size());
+      assertEquals(entity2, entities.get(0));
+      assertEquals(entity3, entities.get(1));
+
+      uri = URI.create(resourceUri + queryParam + "&fromid="
+          + entity3.getInfo().get("SYSTEM_INFO_FLOW_RUN_ID"));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
+      });
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      assertEquals(entity3, entities.get(0));
+    } finally {
+      client.destroy();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d27b1ca/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 994c276..df3ccab 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
@@ -1097,6 +1097,9 @@ public class TimelineReaderWebServices {
    *     METRICS makes sense for flow runs hence only ALL or METRICS are
    *     supported as fields for fetching flow runs. Other fields will lead to
    *     HTTP 400 (Bad Request) response. (Optional query param).
+   * @param fromId Defines the flow run id. If specified, retrieve the next
+   *     set of flow runs from the given id. The set of flow runs retrieved
+   *     is inclusive of specified fromId.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     set of <cite>FlowRunEntity</cite> instances for the given flow are
@@ -1118,7 +1121,8 @@ public class TimelineReaderWebServices {
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("fromid") String fromId) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -1140,11 +1144,12 @@ public class TimelineReaderWebServices {
       entities = timelineReaderManager.getEntities(context,
           TimelineReaderWebServicesUtils.createTimelineEntityFilters(
           limit, createdTimeStart, createdTimeEnd, null, null, null,
-              null, null, null, null, null),
+              null, null, null, null, fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
           null, metricsToRetrieve, fields, null));
     } catch (Exception e) {
-      handleException(e, url, startTime, "createdTime start/end or limit");
+      handleException(e, url, startTime,
+          "createdTime start/end or limit or fromId");
     }
     long endTime = Time.monotonicNow();
     if (entities == null) {
@@ -1182,6 +1187,9 @@ public class TimelineReaderWebServices {
    *     METRICS makes sense for flow runs hence only ALL or METRICS are
    *     supported as fields for fetching flow runs. Other fields will lead to
    *     HTTP 400 (Bad Request) response. (Optional query param).
+   * @param fromId Defines the flow run id. If specified, retrieve the next
+   *     set of flow runs from the given id. The set of flow runs retrieved
+   *     is inclusive of specified fromId.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     set of <cite>FlowRunEntity</cite> instances for the given flow are
@@ -1204,9 +1212,10 @@ public class TimelineReaderWebServices {
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("fromid") String fromId) {
     return getFlowRuns(req, res, null, userId, flowName, limit,
-        createdTimeStart, createdTimeEnd, metricsToRetrieve, fields);
+        createdTimeStart, createdTimeEnd, metricsToRetrieve, fields, fromId);
   }
 
   /**
@@ -1237,6 +1246,9 @@ public class TimelineReaderWebServices {
    *     METRICS makes sense for flow runs hence only ALL or METRICS are
    *     supported as fields for fetching flow runs. Other fields will lead to
    *     HTTP 400 (Bad Request) response. (Optional query param).
+   * @param fromId Defines the flow run id. If specified, retrieve the next
+   *     set of flow runs from the given id. The set of flow runs retrieved
+   *     is inclusive of specified fromId.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     set of <cite>FlowRunEntity</cite> instances for the given flow are
@@ -1260,7 +1272,8 @@ public class TimelineReaderWebServices {
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("fromid") String fromId) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -1279,11 +1292,12 @@ public class TimelineReaderWebServices {
               TimelineEntityType.YARN_FLOW_RUN.toString(), null, null),
           TimelineReaderWebServicesUtils.createTimelineEntityFilters(
           limit, createdTimeStart, createdTimeEnd, null, null, null,
-              null, null, null, null, null),
+              null, null, null, null, fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
           null, metricsToRetrieve, fields, null));
     } catch (Exception e) {
-      handleException(e, url, startTime, "createdTime start/end or limit");
+      handleException(e, url, startTime,
+          "createdTime start/end or limit or fromId");
     }
     long endTime = Time.monotonicNow();
     if (entities == null) {
@@ -1719,6 +1733,9 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param fromId Defines the application id. If specified, retrieve the next
+   *     set of applications from the given id. The set of applications
+   *     retrieved is inclusive of specified fromId.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -1748,7 +1765,8 @@ public class TimelineReaderWebServices {
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
-      @QueryParam("metricslimit") String metricsLimit) {
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("fromid") String fromId) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -1771,7 +1789,7 @@ public class TimelineReaderWebServices {
           TimelineReaderWebServicesUtils.createTimelineEntityFilters(
           limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
               infofilters, conffilters, metricfilters, eventfilters, null,
-              null),
+              fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
           confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
     } catch (Exception e) {
@@ -1847,6 +1865,9 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param fromId Defines the application id. If specified, retrieve the next
+   *     set of applications from the given id. The set of applications
+   *     retrieved is inclusive of specified fromId.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -1878,12 +1899,13 @@ public class TimelineReaderWebServices {
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
-      @QueryParam("metricslimit") String metricsLimit) {
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("fromid") String fromId) {
     return getEntities(req, res, null, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, null, null);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, null, fromId);
   }
 
   /**
@@ -1947,6 +1969,9 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param fromId Defines the application id. If specified, retrieve the next
+   *     set of applications from the given id. The set of applications
+   *     retrieved is inclusive of specified fromId.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -1980,12 +2005,13 @@ public class TimelineReaderWebServices {
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
-      @QueryParam("metricslimit") String metricsLimit) {
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("fromid") String fromId) {
     return getEntities(req, res, clusterId, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, null, null);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, null, fromId);
   }
 
   /**
@@ -2046,6 +2072,9 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param fromId Defines the application id. If specified, retrieve the next
+   *     set of applications from the given id. The set of applications
+   *     retrieved is inclusive of specified fromId.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -2076,12 +2105,13 @@ public class TimelineReaderWebServices {
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
-      @QueryParam("metricslimit") String metricsLimit) {
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("fromid") String fromId) {
     return getEntities(req, res, null, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         null, limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
         infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, null, null);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, null, fromId);
   }
 
   /**
@@ -2143,6 +2173,9 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param fromId Defines the application id. If specified, retrieve the next
+   *     set of applications from the given id. The set of applications
+   *     retrieved is inclusive of specified fromId.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -2174,12 +2207,13 @@ public class TimelineReaderWebServices {
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
-      @QueryParam("metricslimit") String metricsLimit) {
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("fromid") String fromId) {
     return getEntities(req, res, clusterId, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         null, limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
         infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, null, null);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, null, fromId);
   }
 
   /**
@@ -3107,4 +3141,4 @@ public class TimelineReaderWebServices {
         " (Took " + (endTime - startTime) + " ms.)");
     return results;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d27b1ca/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.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/storage/reader/ApplicationEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
index 1667f61..8a331c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
@@ -48,7 +48,9 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
 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;
 
@@ -359,13 +361,44 @@ class ApplicationEntityReader extends GenericEntityReader {
       Connection conn, FilterList filterList) throws IOException {
     Scan scan = new Scan();
     TimelineReaderContext context = getContext();
+    RowKeyPrefix<ApplicationRowKey> applicationRowKeyPrefix = null;
+
     // Whether or not flowRunID is null doesn't matter, the
     // ApplicationRowKeyPrefix will do the right thing.
-    RowKeyPrefix<ApplicationRowKey> applicationRowKeyPrefix =
-        new ApplicationRowKeyPrefix(context.getClusterId(),
-            context.getUserId(), context.getFlowName(),
-            context.getFlowRunId());
-    scan.setRowPrefixFilter(applicationRowKeyPrefix.getRowKeyPrefix());
+    // default mode, will always scans from beginning of entity type.
+    if (getFilters().getFromId() == null) {
+      applicationRowKeyPrefix = new ApplicationRowKeyPrefix(
+          context.getClusterId(), context.getUserId(), context.getFlowName(),
+          context.getFlowRunId());
+      scan.setRowPrefixFilter(applicationRowKeyPrefix.getRowKeyPrefix());
+    } else {
+      Long flowRunId = context.getFlowRunId();
+      if (flowRunId == null) {
+        AppToFlowRowKey appToFlowRowKey = new AppToFlowRowKey(
+            context.getClusterId(), getFilters().getFromId());
+        FlowContext flowContext =
+            lookupFlowContext(appToFlowRowKey, hbaseConf, conn);
+        flowRunId = flowContext.getFlowRunId();
+      }
+
+      ApplicationRowKey applicationRowKey =
+          new ApplicationRowKey(context.getClusterId(), context.getUserId(),
+              context.getFlowName(), flowRunId, getFilters().getFromId());
+
+      // set start row
+      scan.setStartRow(applicationRowKey.getRowKey());
+
+      // get the bytes for stop row
+      applicationRowKeyPrefix = new ApplicationRowKeyPrefix(
+          context.getClusterId(), context.getUserId(), context.getFlowName(),
+          context.getFlowRunId());
+
+      // set stop row
+      scan.setStopRow(
+          HBaseTimelineStorageUtils.calculateTheClosestNextRowKeyForPrefix(
+              applicationRowKeyPrefix.getRowKeyPrefix()));
+    }
+
     FilterList newList = new FilterList();
     newList.addFilter(new PageFilter(getFilters().getLimit()));
     if (filterList != null && !filterList.getFilters().isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d27b1ca/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.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/storage/reader/FlowRunEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
index 9b8482c..cedf96a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilte
 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.flow.FlowRunColumn;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnFamily;
@@ -210,10 +211,30 @@ class FlowRunEntityReader extends TimelineEntityReader {
       FilterList filterList) throws IOException {
     Scan scan = new Scan();
     TimelineReaderContext context = getContext();
-    RowKeyPrefix<FlowRunRowKey> flowRunRowKeyPrefix =
-        new FlowRunRowKeyPrefix(context.getClusterId(), context.getUserId(),
-            context.getFlowName());
-    scan.setRowPrefixFilter(flowRunRowKeyPrefix.getRowKeyPrefix());
+    RowKeyPrefix<FlowRunRowKey> flowRunRowKeyPrefix = null;
+    if (getFilters().getFromId() == null) {
+      flowRunRowKeyPrefix = new FlowRunRowKeyPrefix(context.getClusterId(),
+          context.getUserId(), context.getFlowName());
+      scan.setRowPrefixFilter(flowRunRowKeyPrefix.getRowKeyPrefix());
+    } else {
+
+      FlowRunRowKey flowRunRowKey =
+          new FlowRunRowKey(context.getClusterId(), context.getUserId(),
+              context.getFlowName(), Long.parseLong(getFilters().getFromId()));
+
+      // set start row
+      scan.setStartRow(flowRunRowKey.getRowKey());
+
+      // get the bytes for stop row
+      flowRunRowKeyPrefix = new FlowRunRowKeyPrefix(context.getClusterId(),
+          context.getUserId(), context.getFlowName());
+
+      // set stop row
+      scan.setStopRow(
+          HBaseTimelineStorageUtils.calculateTheClosestNextRowKeyForPrefix(
+              flowRunRowKeyPrefix.getRowKeyPrefix()));
+    }
+
     FilterList newList = new FilterList();
     newList.addFilter(new PageFilter(getFilters().getLimit()));
     if (filterList != null && !filterList.getFilters().isEmpty()) {


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


[42/50] [abbrv] hadoop git commit: Addendum patch for YARN-6102.

Posted by va...@apache.org.
Addendum patch for YARN-6102.

(cherry picked from commit 08d5f3acf6f283694f806b0d798dc10b515136ef)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 278ca2f246cdc521d9ef728101cdf252b5f98c2d
Parents: c602f05
Author: Sunil G <su...@apache.org>
Authored: Wed Jul 26 18:40:52 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Fri Jul 28 22:48:38 2017 +0530

----------------------------------------------------------------------
 .../apache/hadoop/yarn/server/resourcemanager/ResourceManager.java  | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/278ca2f2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index d8de137..d085fbc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -1417,7 +1417,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
     rmContextImpl.setDispatcher(dispatcher);
 
     rmContext = rmContextImpl;
-    rmContext.setDispatcher(rmDispatcher);
   }
 
   private void setSchedulerRecoveryStartAndWaitTime(RMState state,


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


[11/50] [abbrv] hadoop git commit: YARN-6027. Support fromid(offset) filter for /flows API (Rohith Sharma K S via Varun Saxena)

Posted by va...@apache.org.
YARN-6027. Support fromid(offset) filter for /flows API (Rohith Sharma K S via Varun Saxena)

(cherry picked from commit 63c06ec44e633567c378e28898e319143593ff30)

Conflicts:
	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


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

Branch: refs/heads/YARN-5355_branch2
Commit: 1862a03030a26876f701cd842fce17f272fbb8fb
Parents: 177a324
Author: Varun Saxena <va...@apache.org>
Authored: Thu Mar 2 01:49:34 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:44 2017 +0530

----------------------------------------------------------------------
 .../AbstractTimelineReaderHBaseTestBase.java    | 176 ++++++++
 ...stTimelineReaderWebServicesHBaseStorage.java | 411 ++++++++-----------
 .../storage/common/KeyConverterToString.java    |  38 ++
 .../storage/flow/FlowActivityRowKey.java        |  59 ++-
 .../reader/FlowActivityEntityReader.java        |  28 +-
 .../storage/common/TestRowKeys.java             |  21 +
 .../reader/TimelineReaderServer.java            |   2 +-
 .../reader/TimelineReaderUtils.java             |  28 +-
 .../reader/TimelineReaderWebServices.java       |  18 +-
 .../reader/TimelineUIDConverter.java            |  30 +-
 10 files changed, 549 insertions(+), 262 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1862a030/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.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/AbstractTimelineReaderHBaseTestBase.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/AbstractTimelineReaderHBaseTestBase.java
new file mode 100644
index 0000000..7853c94
--- /dev/null
+++ 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/AbstractTimelineReaderHBaseTestBase.java
@@ -0,0 +1,176 @@
+/**
+ * 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.reader;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URL;
+import java.util.List;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timelineservice.storage.DataGeneratorForTest;
+import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
+import org.junit.Assert;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.ClientResponse.Status;
+import com.sun.jersey.api.client.GenericType;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+
+/**
+ * Test Base for TimelineReaderServer HBase tests.
+ */
+public abstract class AbstractTimelineReaderHBaseTestBase {
+  private static int serverPort;
+  private static TimelineReaderServer server;
+  private static HBaseTestingUtility util;
+
+  public static void setup() throws Exception {
+    util = new HBaseTestingUtility();
+    Configuration conf = util.getConfiguration();
+    conf.setInt("hfile.format.version", 3);
+    util.startMiniCluster();
+    DataGeneratorForTest.createSchema(util.getConfiguration());
+  }
+
+  public static void tearDown() throws Exception {
+    if (server != null) {
+      server.stop();
+      server = null;
+    }
+    if (util != null) {
+      util.shutdownMiniCluster();
+    }
+  }
+
+  protected static void initialize() throws Exception {
+    try {
+      Configuration config = util.getConfiguration();
+      config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+      config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+      config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+          "localhost:0");
+      config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
+      config.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
+          "org.apache.hadoop.yarn.server.timelineservice.storage."
+              + "HBaseTimelineReaderImpl");
+      config.setInt("hfile.format.version", 3);
+      server = new TimelineReaderServer() {
+        @Override
+        protected void setupOptions(Configuration conf) {
+          // The parent code tries to use HttpServer2 from this version of
+          // Hadoop, but the tests are loading in HttpServer2 from
+          // ${hbase-compatible-hadoop.version}. This version uses Jetty 9
+          // while ${hbase-compatible-hadoop.version} uses Jetty 6, and there
+          // are many differences, including classnames and packages.
+          // We do nothing here, so that we don't cause a NoSuchMethodError.
+          // Once ${hbase-compatible-hadoop.version} is changed to Hadoop 3,
+          // we should be able to remove this @Override.
+        }
+      };
+      server.init(config);
+      server.start();
+      serverPort = server.getWebServerPort();
+    } catch (Exception e) {
+      Assert.fail("Web server failed to start");
+    }
+  }
+
+  protected Client createClient() {
+    ClientConfig cfg = new DefaultClientConfig();
+    cfg.getClasses().add(YarnJacksonJaxbJsonProvider.class);
+    return new Client(
+        new URLConnectionClientHandler(new DummyURLConnectionFactory()), cfg);
+  }
+
+  protected ClientResponse getResponse(Client client, URI uri)
+      throws Exception {
+    ClientResponse resp =
+        client.resource(uri).accept(MediaType.APPLICATION_JSON)
+            .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    if (resp == null || resp.getStatusInfo()
+        .getStatusCode() != ClientResponse.Status.OK.getStatusCode()) {
+      String msg = "";
+      if (resp != null) {
+        msg = String.valueOf(resp.getStatusInfo().getStatusCode());
+      }
+      throw new IOException(
+          "Incorrect response from timeline reader. " + "Status=" + msg);
+    }
+    return resp;
+  }
+
+  protected void verifyHttpResponse(Client client, URI uri, Status status) {
+    ClientResponse resp =
+        client.resource(uri).accept(MediaType.APPLICATION_JSON)
+            .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    assertNotNull(resp);
+    assertTrue("Response from server should have been " + status,
+        resp.getStatusInfo().getStatusCode() == status.getStatusCode());
+    System.out.println("Response is: " + resp.getEntity(String.class));
+  }
+
+  protected List<FlowActivityEntity> verifyFlowEntites(Client client, URI uri,
+      int noOfEntities) throws Exception {
+    ClientResponse resp = getResponse(client, uri);
+    List<FlowActivityEntity> entities =
+        resp.getEntity(new GenericType<List<FlowActivityEntity>>() {
+        });
+    assertNotNull(entities);
+    assertEquals(noOfEntities, entities.size());
+    return entities;
+  }
+
+  protected static class DummyURLConnectionFactory
+      implements HttpURLConnectionFactory {
+
+    @Override
+    public HttpURLConnection getHttpURLConnection(final URL url)
+        throws IOException {
+      try {
+        return (HttpURLConnection) url.openConnection();
+      } catch (UndeclaredThrowableException e) {
+        throw new IOException(e.getCause());
+      }
+    }
+  }
+
+  protected static HBaseTestingUtility getHBaseTestingUtility() {
+    return util;
+  }
+
+  public static int getServerPort() {
+    return serverPort;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1862a030/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 d9a7078..6386183 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
@@ -24,10 +24,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
-import java.lang.reflect.UndeclaredThrowableException;
-import java.net.HttpURLConnection;
 import java.net.URI;
-import java.net.URL;
 import java.text.DateFormat;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -39,7 +36,6 @@ import java.util.Set;
 import javax.ws.rs.core.MediaType;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
@@ -50,17 +46,11 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
-import org.apache.hadoop.yarn.server.timelineservice.storage.DataGeneratorForTest;
 import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
-import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -70,27 +60,26 @@ import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
 import com.sun.jersey.api.client.GenericType;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
-import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
-
-public class TestTimelineReaderWebServicesHBaseStorage {
-  private int serverPort;
-  private TimelineReaderServer server;
-  private static HBaseTestingUtility util;
+
+/**
+ * Test TimelineReder Web Service REST API's.
+ */
+public class TestTimelineReaderWebServicesHBaseStorage
+    extends AbstractTimelineReaderHBaseTestBase {
   private static long ts = System.currentTimeMillis();
   private static long dayTs =
       HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts);
 
   @BeforeClass
-  public static void setup() throws Exception {
-    util = new HBaseTestingUtility();
-    Configuration conf = util.getConfiguration();
-    conf.setInt("hfile.format.version", 3);
-    util.startMiniCluster();
-    DataGeneratorForTest.createSchema(conf);
+  public static void setupBeforeClass() throws Exception {
+    setup();
     loadData();
+    initialize();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    tearDown();
   }
 
   private static void loadData() throws Exception {
@@ -344,7 +333,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     }
 
     HBaseTimelineWriterImpl hbi = null;
-    Configuration c1 = util.getConfiguration();
+    Configuration c1 = getHBaseTestingUtility().getConfiguration();
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
@@ -393,71 +382,6 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     }
   }
 
-  @AfterClass
-  public static void tearDown() throws Exception {
-    util.shutdownMiniCluster();
-  }
-
-  @Before
-  public void init() throws Exception {
-    try {
-      Configuration config = util.getConfiguration();
-      config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
-      config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-      config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
-          "localhost:0");
-      config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
-      config.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
-          "org.apache.hadoop.yarn.server.timelineservice.storage." +
-              "HBaseTimelineReaderImpl");
-      config.setInt("hfile.format.version", 3);
-      server = new TimelineReaderServer();
-      server.init(config);
-      server.start();
-      serverPort = server.getWebServerPort();
-    } catch (Exception e) {
-      Assert.fail("Web server failed to start");
-    }
-  }
-
-  private static Client createClient() {
-    ClientConfig cfg = new DefaultClientConfig();
-    cfg.getClasses().add(YarnJacksonJaxbJsonProvider.class);
-    return new Client(new URLConnectionClientHandler(
-        new DummyURLConnectionFactory()), cfg);
-  }
-
-  private static ClientResponse getResponse(Client client, URI uri)
-      throws Exception {
-    ClientResponse resp =
-        client.resource(uri).accept(MediaType.APPLICATION_JSON)
-        .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    if (resp == null ||
-        resp.getClientResponseStatus() != ClientResponse.Status.OK) {
-      String msg = "";
-      if (resp != null) {
-        msg = String.valueOf(resp.getClientResponseStatus());
-      }
-      throw new IOException("Incorrect response from timeline reader. " +
-          "Status=" + msg);
-    }
-    return resp;
-  }
-
-  private static class DummyURLConnectionFactory
-      implements HttpURLConnectionFactory {
-
-    @Override
-    public HttpURLConnection getHttpURLConnection(final URL url)
-        throws IOException {
-      try {
-        return (HttpURLConnection)url.openConnection();
-      } catch (UndeclaredThrowableException e) {
-        throw new IOException(e.getCause());
-      }
-    }
-  }
-
   private static TimelineEntity newEntity(String type, String id) {
     TimelineEntity entity = new TimelineEntity();
     entity.setIdentifier(new TimelineEntity.Identifier(type, id));
@@ -499,22 +423,11 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     return false;
   }
 
-  private static void verifyHttpResponse(Client client, URI uri,
-      Status status) {
-    ClientResponse resp =
-        client.resource(uri).accept(MediaType.APPLICATION_JSON)
-        .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    assertNotNull(resp);
-    assertTrue("Response from server should have been " + status,
-        resp.getClientResponseStatus() == status);
-    System.out.println("Response is: " + resp.getEntity(String.class));
-  }
-
   @Test
   public void testGetFlowRun() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" +
           "1002345678919");
       ClientResponse resp = getResponse(client, uri);
@@ -534,7 +447,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
 
       // Query without specifying cluster ID.
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/users/user1/flows/flow_name/runs/1002345678919");
       resp = getResponse(client, uri);
       entity = resp.getEntity(FlowRunEntity.class);
@@ -559,7 +472,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetFlowRuns() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/runs");
       ClientResponse resp = getResponse(client, uri);
       Set<FlowRunEntity> entities =
@@ -578,8 +491,9 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         assertEquals(0, entity.getMetrics().size());
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "clusters/cluster1/users/user1/flows/flow_name/runs?limit=1");
+      uri =
+          URI.create("http://localhost:" + getServerPort() + "/ws/v2/timeline/"
+              + "clusters/cluster1/users/user1/flows/flow_name/runs?limit=1");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
       assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
@@ -593,7 +507,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         assertEquals(0, entity.getMetrics().size());
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
           "createdtimestart=1425016501030");
       resp = getResponse(client, uri);
@@ -609,7 +523,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         assertEquals(0, entity.getMetrics().size());
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
           "createdtimestart=1425016500999&createdtimeend=1425016501035");
       resp = getResponse(client, uri);
@@ -628,7 +542,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         assertEquals(0, entity.getMetrics().size());
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
           "createdtimeend=1425016501030");
       resp = getResponse(client, uri);
@@ -644,7 +558,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         assertEquals(0, entity.getMetrics().size());
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
           "fields=metrics");
       resp = getResponse(client, uri);
@@ -666,7 +580,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // fields as CONFIGS will lead to a HTTP 400 as it makes no sense for
       // flow runs.
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
           "fields=CONFIGS");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
@@ -679,7 +593,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetFlowRunsMetricsToRetrieve() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
           "metricstoretrieve=MAP_,HDFS_");
       ClientResponse resp = getResponse(client, uri);
@@ -698,7 +612,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
       assertEquals(3, metricCnt);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
           "metricstoretrieve=!(MAP_,HDFS_)");
       resp = getResponse(client, uri);
@@ -724,7 +638,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     Client client = createClient();
     try {
       // Query all flows.
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/flows");
       ClientResponse resp = getResponse(client, uri);
       Set<FlowActivityEntity> flowEntities =
@@ -750,7 +664,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       // Query flowruns based on UID returned in query above.
       List<String> listFlowRunUIDs = new ArrayList<String>();
       for (String flowUID : listFlowUIDs) {
-        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+        uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
             "timeline/flow-uid/" + flowUID + "/runs");
         resp = getResponse(client, uri);
         Set<FlowRunEntity> frEntities =
@@ -770,7 +684,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // Query single flowrun based on UIDs' returned in query to get flowruns.
       for (String flowRunUID : listFlowRunUIDs) {
-        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+        uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
             "timeline/run-uid/" + flowRunUID);
         resp = getResponse(client, uri);
         FlowRunEntity entity = resp.getEntity(FlowRunEntity.class);
@@ -782,7 +696,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       for (String flowRunUID : listFlowRunUIDs) {
         TimelineReaderContext context =
             TimelineUIDConverter.FLOWRUN_UID.decodeUID(flowRunUID);
-        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+        uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
             "timeline/run-uid/" + flowRunUID + "/apps");
         resp = getResponse(client, uri);
         Set<TimelineEntity> appEntities =
@@ -802,7 +716,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // Query single app based on UIDs' returned in query to get apps.
       for (String appUID : listAppUIDs) {
-        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+        uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
             "timeline/app-uid/" + appUID);
         resp = getResponse(client, uri);
         TimelineEntity entity = resp.getEntity(TimelineEntity.class);
@@ -815,7 +729,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       for (String appUID : listAppUIDs) {
         TimelineReaderContext context =
             TimelineUIDConverter.APPLICATION_UID.decodeUID(appUID);
-        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+        uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
             "timeline/app-uid/" + appUID + "/entities/type1");
         resp = getResponse(client, uri);
         Set<TimelineEntity> entities =
@@ -837,39 +751,39 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // Query single entity based on UIDs' returned in query to get entities.
       for (String entityUID : listEntityUIDs) {
-        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+        uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
             "timeline/entity-uid/" + entityUID);
         resp = getResponse(client, uri);
         TimelineEntity entity = resp.getEntity(TimelineEntity.class);
         assertNotNull(entity);
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/flow-uid/dummy:flow/runs");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/run-uid/dummy:flowrun");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
 
       // Run Id is not a numerical value.
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/run-uid/some:dummy:flow:123v456");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/run-uid/dummy:flowrun/apps");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/app-uid/dummy:app");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/app-uid/dummy:app/entities/type1");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/entity-uid/dummy:entity");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
     } finally {
@@ -883,7 +797,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     try {
       String appUIDWithFlowInfo =
           "cluster1!user1!flow_name!1002345678919!application_1111111111_1111";
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/"+
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/"+
           "timeline/app-uid/" + appUIDWithFlowInfo);
       ClientResponse resp = getResponse(client, uri);
       TimelineEntity appEntity1 = resp.getEntity(TimelineEntity.class);
@@ -892,8 +806,9 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           TimelineEntityType.YARN_APPLICATION.toString(), appEntity1.getType());
       assertEquals("application_1111111111_1111", appEntity1.getId());
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "app-uid/" + appUIDWithFlowInfo + "/entities/type1");
+      uri =
+          URI.create("http://localhost:" + getServerPort() + "/ws/v2/timeline/"
+              + "app-uid/" + appUIDWithFlowInfo + "/entities/type1");
       resp = getResponse(client, uri);
       Set<TimelineEntity> entities1 =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -910,8 +825,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
 
       String appUIDWithoutFlowInfo = "cluster1!application_1111111111_1111";
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+
-          "app-uid/" + appUIDWithoutFlowInfo);
+      uri = URI.create("http://localhost:" + getServerPort()
+          + "/ws/v2/timeline/" + "app-uid/" + appUIDWithoutFlowInfo);
       resp = getResponse(client, uri);
       TimelineEntity appEntity2 = resp.getEntity(TimelineEntity.class);
       assertNotNull(appEntity2);
@@ -919,8 +834,9 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           TimelineEntityType.YARN_APPLICATION.toString(), appEntity2.getType());
       assertEquals("application_1111111111_1111", appEntity2.getId());
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "app-uid/" + appUIDWithoutFlowInfo + "/entities/type1");
+      uri =
+          URI.create("http://localhost:" + getServerPort() + "/ws/v2/timeline/"
+              + "app-uid/" + appUIDWithoutFlowInfo + "/entities/type1");
       resp = getResponse(client, uri);
       Set<TimelineEntity> entities2 =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -937,8 +853,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
 
       String entityUIDWithFlowInfo = appUIDWithFlowInfo + "!type1!0!entity1";
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+
-          "entity-uid/" + entityUIDWithFlowInfo);
+      uri = URI.create("http://localhost:" + getServerPort()
+          + "/ws/v2/timeline/" + "entity-uid/" + entityUIDWithFlowInfo);
       resp = getResponse(client, uri);
       TimelineEntity singleEntity1 = resp.getEntity(TimelineEntity.class);
       assertNotNull(singleEntity1);
@@ -947,8 +863,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       String entityUIDWithoutFlowInfo =
           appUIDWithoutFlowInfo + "!type1!0!entity1";
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+
-          "entity-uid/" + entityUIDWithoutFlowInfo);
+      uri = URI.create("http://localhost:" + getServerPort()
+          + "/ws/v2/timeline/" + "entity-uid/" + entityUIDWithoutFlowInfo);
       resp = getResponse(client, uri);
       TimelineEntity singleEntity2 = resp.getEntity(TimelineEntity.class);
       assertNotNull(singleEntity2);
@@ -965,7 +881,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     try {
       String appUID =
           "cluster1!user*1!flow_name!1002345678919!application_1111111111_1111";
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/"+
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/"+
           "timeline/app-uid/" + appUID);
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
     } finally {
@@ -977,19 +893,19 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetFlows() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/flows");
 
       verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1},
           new String[] {"flow1", "flow_name", "flow_name2"});
 
       // Query without specifying cluster ID.
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/flows/");
       verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1},
           new String[] {"flow1", "flow_name", "flow_name2"});
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
               "timeline/clusters/cluster1/flows?limit=1");
       verifyFlowEntites(client, uri, 1, new int[] {3},
           new String[] {"flow1"});
@@ -998,43 +914,43 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(1425016501000L);
 
       DateFormat fmt = TimelineReaderWebServices.DATE_FORMAT.get();
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/flows?daterange="
           + fmt.format(firstFlowActivity) + "-"
           + fmt.format(dayTs));
       verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1},
           new String[] {"flow1", "flow_name", "flow_name2"});
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/flows?daterange=" +
           fmt.format(dayTs + (4*86400000L)));
       verifyFlowEntites(client, uri, 0, new int[] {}, new String[] {});
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/flows?daterange=-" +
           fmt.format(dayTs));
       verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1},
           new String[] {"flow1", "flow_name", "flow_name2"});
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/flows?daterange=" +
            fmt.format(firstFlowActivity) + "-");
       verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1},
           new String[] {"flow1", "flow_name", "flow_name2"});
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/flows?daterange=20150711:20150714");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/flows?daterange=20150714-20150711");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/flows?daterange=2015071129-20150712");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/flows?daterange=20150711-2015071243");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
     } finally {
@@ -1043,10 +959,47 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   }
 
   @Test
+  public void testGetFlowsForPagination() throws Exception {
+    Client client = createClient();
+    int noOfEntities = 3;
+    int limit = 2;
+    try {
+      String flowURI = "http://localhost:" + getServerPort() + "/ws/v2/"
+          + "timeline/clusters/cluster1/flows";
+      URI uri = URI.create(flowURI);
+      List<FlowActivityEntity> flowEntites =
+          verifyFlowEntites(client, uri, 3, new int[] {3, 2, 1},
+              new String[] {"flow1", "flow_name", "flow_name2"});
+      FlowActivityEntity fEntity1 = flowEntites.get(0);
+      FlowActivityEntity fEntity3 = flowEntites.get(noOfEntities - 1);
+
+      uri = URI.create(flowURI + "?limit=" + limit);
+      flowEntites = verifyFlowEntites(client, uri, limit);
+      assertEquals(fEntity1, flowEntites.get(0));
+      FlowActivityEntity fEntity2 = flowEntites.get(limit - 1);
+
+      uri = URI
+          .create(flowURI + "?limit=" + limit + "&fromid="
+              + fEntity2.getInfo().get(TimelineReaderUtils.FROMID_KEY));
+      flowEntites = verifyFlowEntites(client, uri, noOfEntities - limit + 1);
+      assertEquals(fEntity2, flowEntites.get(0));
+      assertEquals(fEntity3, flowEntites.get(noOfEntities - limit));
+
+      uri = URI
+          .create(flowURI + "?limit=" + limit + "&fromid="
+              + fEntity3.getInfo().get(TimelineReaderUtils.FROMID_KEY));
+      flowEntites = verifyFlowEntites(client, uri, 1);
+      assertEquals(fEntity3, flowEntites.get(0));
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
   public void testGetApp() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111?" +
           "userid=user1&fields=ALL&flowname=flow_name&flowrunid=1002345678919");
       ClientResponse resp = getResponse(client, uri);
@@ -1064,7 +1017,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         assertTrue(verifyMetrics(metric, m1, m2, m3));
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
               "timeline/apps/application_1111111111_2222?userid=user1" +
               "&fields=metrics&flowname=flow_name&flowrunid=1002345678919");
       resp = getResponse(client, uri);
@@ -1086,7 +1039,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetAppWithoutFlowInfo() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111?" +
           "fields=ALL");
       ClientResponse resp = getResponse(client, uri);
@@ -1105,7 +1058,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         assertTrue(verifyMetrics(metric, m1, m2, m3));
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111?" +
           "fields=ALL&metricslimit=10");
       resp = getResponse(client, uri);
@@ -1135,7 +1088,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetEntityWithoutFlowInfo() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1/entity1");
       ClientResponse resp = getResponse(client, uri);
@@ -1152,7 +1105,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetEntitiesWithoutFlowInfo() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1");
       ClientResponse resp = getResponse(client, uri);
@@ -1176,7 +1129,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetEntitiesDataToRetrieve() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?confstoretrieve=cfg_");
       ClientResponse resp = getResponse(client, uri);
@@ -1193,7 +1146,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
       assertEquals(2, cfgCnt);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?confstoretrieve=cfg_,config_");
       resp = getResponse(client, uri);
@@ -1210,7 +1163,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
       assertEquals(5, cfgCnt);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?confstoretrieve=!(cfg_,config_)");
       resp = getResponse(client, uri);
@@ -1226,7 +1179,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
       assertEquals(1, cfgCnt);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?metricstoretrieve=MAP_");
       resp = getResponse(client, uri);
@@ -1242,7 +1195,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
       assertEquals(1, metricCnt);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?metricstoretrieve=MAP1_,HDFS_");
       resp = getResponse(client, uri);
@@ -1259,7 +1212,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
       assertEquals(3, metricCnt);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?metricstoretrieve=!(MAP1_,HDFS_)");
       resp = getResponse(client, uri);
@@ -1284,7 +1237,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetEntitiesConfigFilters() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?conffilters=config_param1%20eq%20value1%20OR%20" +
           "config_param1%20eq%20value3");
@@ -1298,7 +1251,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
             entity.getId().equals("entity2"));
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?conffilters=config_param1%20eq%20value1%20AND" +
           "%20configuration_param2%20eq%20value2");
@@ -1309,7 +1262,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // conffilters=(config_param1 eq value1 AND configuration_param2 eq
       // value2) OR (config_param1 eq value3 AND cfg_param3 eq value1)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" +
           "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" +
@@ -1327,7 +1280,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // conffilters=(config_param1 eq value1 AND configuration_param2 eq
       // value2) OR (config_param1 eq value3 AND cfg_param3 eq value1)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" +
           "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" +
@@ -1343,7 +1296,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
       assertEquals(3, cfgCnt);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" +
           "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" +
@@ -1369,7 +1322,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       // entity1. For ne, both entity1 and entity2 will be returned. For ene,
       // only entity2 will be returned as we are checking for existence too.
       // conffilters=configuration_param2 ne value3
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?conffilters=configuration_param2%20ne%20value3");
       resp = getResponse(client, uri);
@@ -1381,7 +1334,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
             entity.getId().equals("entity2"));
       }
       // conffilters=configuration_param2 ene value3
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?conffilters=configuration_param2%20ene%20value3");
       resp = getResponse(client, uri);
@@ -1401,7 +1354,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     Client client = createClient();
     try {
       // infofilters=info1 eq cluster1 OR info1 eq cluster2
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?infofilters=info1%20eq%20cluster1%20OR%20info1%20eq" +
           "%20cluster2");
@@ -1416,7 +1369,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
 
       // infofilters=info1 eq cluster1 AND info4 eq 35000
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?infofilters=info1%20eq%20cluster1%20AND%20info4%20" +
           "eq%2035000");
@@ -1426,7 +1379,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertEquals(0, entities.size());
 
       // infofilters=info4 eq 35000 OR info4 eq 36000
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?infofilters=info4%20eq%2035000%20OR%20info4%20eq" +
           "%2036000");
@@ -1441,7 +1394,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // infofilters=(info1 eq cluster1 AND info4 eq 35000) OR
       // (info1 eq cluster2 AND info2 eq 2.0)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?infofilters=(info1%20eq%20cluster1%20AND%20info4%20" +
           "eq%2035000)%20OR%20(info1%20eq%20cluster2%20AND%20info2%20eq%202.0" +
@@ -1460,7 +1413,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // infofilters=(info1 eq cluster1 AND info4 eq 35000) OR
       // (info1 eq cluster2 AND info2 eq 2.0)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?infofilters=(info1%20eq%20cluster1%20AND%20info4%20" +
           "eq%2035000)%20OR%20(info1%20eq%20cluster2%20AND%20info2%20eq%20" +
@@ -1482,7 +1435,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       // both entity1 and entity2 will be returned. For ene, only entity2 will
       // be returned as we are checking for existence too.
       // infofilters=info3 ne 39000
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?infofilters=info3%20ne%2039000");
       resp = getResponse(client, uri);
@@ -1494,7 +1447,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
             entity.getId().equals("entity2"));
       }
       // infofilters=info3 ene 39000
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?infofilters=info3%20ene%2039000");
       resp = getResponse(client, uri);
@@ -1514,7 +1467,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     Client client = createClient();
     try {
       // metricfilters=HDFS_BYTES_READ lt 60 OR HDFS_BYTES_READ eq 157
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?metricfilters=HDFS_BYTES_READ%20lt%2060%20OR%20" +
           "HDFS_BYTES_READ%20eq%20157");
@@ -1529,7 +1482,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
 
       // metricfilters=HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?metricfilters=HDFS_BYTES_READ%20lt%2060%20AND%20" +
           "MAP_SLOT_MILLIS%20gt%2040");
@@ -1540,7 +1493,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR
       // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
           "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
@@ -1558,7 +1511,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR
       // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
           "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
@@ -1576,7 +1529,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR
       // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
           "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
@@ -1597,7 +1550,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
       assertEquals(2, metricCnt);
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
           "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
@@ -1630,7 +1583,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       // entity1. For ne, both entity1 and entity2 will be returned. For ene,
       // only entity2 will be returned as we are checking for existence too.
       // metricfilters=MAP11_SLOT_MILLIS ne 100
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?metricfilters=MAP11_SLOT_MILLIS%20ne%20100");
       resp = getResponse(client, uri);
@@ -1642,7 +1595,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
             entity.getId().equals("entity2"));
       }
       // metricfilters=MAP11_SLOT_MILLIS ene 100
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?metricfilters=MAP11_SLOT_MILLIS%20ene%20100");
       resp = getResponse(client, uri);
@@ -1661,7 +1614,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetEntitiesEventFilters() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?eventfilters=event1,event3");
       ClientResponse resp = getResponse(client, uri);
@@ -1674,7 +1627,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
             entity.getId().equals("entity2"));
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?eventfilters=!(event1,event3)");
       resp = getResponse(client, uri);
@@ -1683,7 +1636,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertEquals(0, entities.size());
 
       // eventfilters=!(event1,event3) OR event5,event6
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?eventfilters=!(event1,event3)%20OR%20event5,event6");
       resp = getResponse(client, uri);
@@ -1696,7 +1649,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       //  eventfilters=(!(event1,event3) OR event5,event6) OR
       // (event1,event2 AND (event3,event4))
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?eventfilters=(!(event1,event3)%20OR%20event5," +
           "event6)%20OR%20(event1,event2%20AND%20(event3,event4))");
@@ -1717,7 +1670,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetEntitiesRelationFilters() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1?isrelatedto=type3:entity31,type2:entity21:entity22");
       ClientResponse resp = getResponse(client, uri);
@@ -1730,7 +1683,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
             entity.getId().equals("entity2"));
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+      uri = URI.create("http://localhost:" + getServerPort() +
+          "/ws/v2/timeline/" +
           "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
           "?isrelatedto=!(type3:entity31,type2:entity21:entity22)");
       resp = getResponse(client, uri);
@@ -1740,7 +1694,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // isrelatedto=!(type3:entity31,type2:entity21:entity22)OR type5:entity51,
       // type6:entity61:entity66
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+      uri = URI.create("http://localhost:" + getServerPort() +
+          "/ws/v2/timeline/" +
           "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
           "?isrelatedto=!(type3:entity31,type2:entity21:entity22)%20OR%20" +
           "type5:entity51,type6:entity61:entity66");
@@ -1755,7 +1710,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       // isrelatedto=(!(type3:entity31,type2:entity21:entity22)OR type5:
       // entity51,type6:entity61:entity66) OR (type1:entity14,type2:entity21:
       // entity22 AND (type3:entity32:entity35,type4:entity42))
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+      uri = URI.create("http://localhost:" + getServerPort() +
+          "/ws/v2/timeline/" +
           "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
           "?isrelatedto=(!(type3:entity31,type2:entity21:entity22)%20OR%20" +
           "type5:entity51,type6:entity61:entity66)%20OR%20(type1:entity14," +
@@ -1772,7 +1728,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // relatesto=!(type3:entity31,type2:entity21:entity22)OR type5:entity51,
       // type6:entity61:entity66
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+      uri = URI.create("http://localhost:" + getServerPort() +
+          "/ws/v2/timeline/" +
           "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
           "?relatesto=!%20(type3:entity31,type2:entity21:entity22%20)%20OR%20" +
           "type5:entity51,type6:entity61:entity66");
@@ -1787,7 +1744,9 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       // relatesto=(!(type3:entity31,type2:entity21:entity22)OR type5:entity51,
       // type6:entity61:entity66) OR (type1:entity14,type2:entity21:entity22 AND
       // (type3:entity32:entity35 , type4:entity42))
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+      uri =
+          URI.create("http://localhost:" + getServerPort() +
+          "/ws/v2/timeline/" +
           "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
           "?relatesto=(!(%20type3:entity31,type2:entity21:entity22)%20OR%20" +
           "type5:entity51,type6:entity61:entity66%20)%20OR%20(type1:entity14," +
@@ -1813,7 +1772,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetEntityDataToRetrieve() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1/entity2?confstoretrieve=cfg_,configuration_");
       ClientResponse resp = getResponse(client, uri);
@@ -1827,7 +1786,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
             configKey.startsWith("cfg_"));
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1/entity2?confstoretrieve=!(cfg_,configuration_)");
       resp = getResponse(client, uri);
@@ -1840,7 +1799,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         assertTrue(configKey.startsWith("config_"));
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1/entity2?metricstoretrieve=MAP1_,HDFS_");
       resp = getResponse(client, uri);
@@ -1854,7 +1813,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
             metric.getId().startsWith("HDFS_"));
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1/entity2?metricstoretrieve=!(MAP1_,HDFS_)");
       resp = getResponse(client, uri);
@@ -1869,7 +1828,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         assertEquals(1, metric.getValues().size());
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
           "entities/type1/entity2?metricstoretrieve=!(MAP1_,HDFS_)&" +
           "metricslimit=5");
@@ -1892,7 +1851,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetFlowRunApps() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" +
           "1002345678919/apps?fields=ALL");
       ClientResponse resp = getResponse(client, uri);
@@ -1912,7 +1871,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         }
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
               "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" +
               "1002345678919/apps?fields=ALL&metricslimit=2");
       resp = getResponse(client, uri);
@@ -1932,14 +1891,14 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
 
       // Query without specifying cluster ID.
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/users/user1/flows/flow_name/runs/1002345678919/apps");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
       assertNotNull(entities);
       assertEquals(2, entities.size());
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/users/user1/flows/flow_name/runs/1002345678919/" +
           "apps?limit=1");
       resp = getResponse(client, uri);
@@ -1955,7 +1914,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetFlowApps() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" +
           "fields=ALL");
       ClientResponse resp = getResponse(client, uri);
@@ -1994,7 +1953,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         }
       }
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" +
           "fields=ALL&metricslimit=6");
       resp = getResponse(client, uri);
@@ -2038,14 +1997,14 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
 
       // Query without specifying cluster ID.
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/users/user1/flows/flow_name/apps");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
       assertNotNull(entities);
       assertEquals(3, entities.size());
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/users/user1/flows/flow_name/apps?limit=1");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -2061,7 +2020,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     Client client = createClient();
     try {
       String entityType = TimelineEntityType.YARN_APPLICATION.toString();
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" +
           "eventfilters=" + ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
       ClientResponse resp = getResponse(client, uri);
@@ -2072,7 +2031,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertTrue("Unexpected app in result", entities.contains(
           newEntity(entityType, "application_1111111111_1111")));
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" +
           "metricfilters=HDFS_BYTES_READ%20ge%200");
       resp = getResponse(client, uri);
@@ -2082,7 +2041,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertTrue("Unexpected app in result", entities.contains(
           newEntity(entityType, "application_1111111111_1111")));
 
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" +
           "conffilters=cfg1%20eq%20value1");
       resp = getResponse(client, uri);
@@ -2100,7 +2059,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetFlowRunNotPresent() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" +
           "1002345678929");
       verifyHttpResponse(client, uri, Status.NOT_FOUND);
@@ -2113,7 +2072,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetFlowsNotPresent() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster2/flows");
       ClientResponse resp = getResponse(client, uri);
       Set<FlowActivityEntity> entities =
@@ -2130,7 +2089,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetAppNotPresent() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/application_1111111111_1378");
       verifyHttpResponse(client, uri, Status.NOT_FOUND);
     } finally {
@@ -2142,7 +2101,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetFlowRunAppsNotPresent() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster2/users/user1/flows/flow_name/runs/" +
           "1002345678919/apps");
       ClientResponse resp = getResponse(client, uri);
@@ -2160,7 +2119,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   public void testGetFlowAppsNotPresent() throws Exception {
     Client client = createClient();
     try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
           "timeline/clusters/cluster2/users/user1/flows/flow_name55/apps");
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
@@ -2173,21 +2132,13 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     }
   }
 
-  @After
-  public void stop() throws Exception {
-    if (server != null) {
-      server.stop();
-      server = null;
-    }
-  }
-
   @Test
   public void testGenericEntitiesForPagination() throws Exception {
     Client client = createClient();
     try {
       int limit = 10;
       String queryParam = "?limit=" + limit;
-      String resourceUri = "http://localhost:" + serverPort + "/ws/v2/"
+      String resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/"
           + "timeline/clusters/cluster1/apps/application_1111111111_1111/"
           + "entities/entitytype";
       URI uri = URI.create(resourceUri + queryParam);
@@ -2251,7 +2202,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     return entity;
   }
 
-  private void verifyFlowEntites(Client client, URI uri, int noOfEntities,
+  private List<FlowActivityEntity> verifyFlowEntites(Client client, URI uri,
+      int noOfEntities,
       int[] a, String[] flowsInSequence) throws Exception {
     ClientResponse resp = getResponse(client, uri);
     List<FlowActivityEntity> entities =
@@ -2267,6 +2219,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           timelineEntity.getInfo().get("SYSTEM_INFO_FLOW_NAME"));
       assertEquals(a[count++], timelineEntity.getFlowRuns().size());
     }
+    return entities;
   }
 
   @Test
@@ -2275,7 +2228,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     try {
       // app entities stored is 15 during initialization.
       int totalAppEntities = 15;
-      String resourceUri = "http://localhost:" + serverPort + "/ws/v2/"
+      String resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/"
           + "timeline/clusters/cluster1/users/user1/flows/flow1/apps";
       URI uri = URI.create(resourceUri);
       ClientResponse resp = getResponse(client, uri);
@@ -2319,7 +2272,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     try {
       // app entities stored is 15 during initialization.
       int totalAppEntities = 5;
-      String resourceUri = "http://localhost:" + serverPort + "/ws/v2/"
+      String resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/"
           + "timeline/clusters/cluster1/users/user1/flows/flow1/runs/1/apps";
       URI uri = URI.create(resourceUri);
       ClientResponse resp = getResponse(client, uri);
@@ -2363,7 +2316,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     try {
       // app entities stored is 15 during initialization.
       int totalRuns = 3;
-      String resourceUri = "http://localhost:" + serverPort + "/ws/v2/"
+      String resourceUri = "http://localhost:" + getServerPort() + "/ws/v2/"
           + "timeline/clusters/cluster1/users/user1/flows/flow1/runs";
       URI uri = URI.create(resourceUri);
       ClientResponse resp = getResponse(client, uri);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1862a030/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverterToString.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/common/KeyConverterToString.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/common/KeyConverterToString.java
new file mode 100644
index 0000000..1f52a7b
--- /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/common/KeyConverterToString.java
@@ -0,0 +1,38 @@
+/**
+ * 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.common;
+
+/**
+ * Interface which has to be implemented for encoding and decoding row keys or
+ * column qualifiers as string.
+ */
+public interface KeyConverterToString<T> {
+  /**
+   * Encode key as string.
+   * @param key of type T to be encoded as string.
+   * @return encoded value as string.
+   */
+  String encodeAsString(T key);
+
+  /**
+   * Decode row key from string to a key of type T.
+   * @param encodedKey string representation of row key
+   * @return type T which has been constructed after decoding string.
+   */
+  T decodeFromString(String encodedKey);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1862a030/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.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/flow/FlowActivityRowKey.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/flow/FlowActivityRowKey.java
index bb77e36..b8a5dba 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/flow/FlowActivityRowKey.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/flow/FlowActivityRowKey.java
@@ -17,10 +17,14 @@
  */
 package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 
+import java.util.List;
+
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverterToString;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 
 /**
@@ -32,8 +36,8 @@ public class FlowActivityRowKey {
   private final Long dayTs;
   private final String userId;
   private final String flowName;
-  private final KeyConverter<FlowActivityRowKey> flowActivityRowKeyConverter =
-      new FlowActivityRowKeyConverter();
+  private final FlowActivityRowKeyConverter
+      flowActivityRowKeyConverter = new FlowActivityRowKeyConverter();
 
   /**
    * @param clusterId identifying the cluster
@@ -104,13 +108,32 @@ public class FlowActivityRowKey {
   }
 
   /**
+   * Constructs a row key for the flow activity table as follows:
+   * {@code clusterId!dayTimestamp!user!flowName}.
+   * @return String representation of row key
+   */
+  public String getRowKeyAsString() {
+    return flowActivityRowKeyConverter.encodeAsString(this);
+  }
+
+  /**
+   * Given the raw row key as string, returns the row key as an object.
+   * @param encodedRowKey String representation of row key.
+   * @return A <cite>FlowActivityRowKey</cite> object.
+   */
+  public static FlowActivityRowKey parseRowKeyFromString(String encodedRowKey) {
+    return new FlowActivityRowKeyConverter().decodeFromString(encodedRowKey);
+  }
+
+  /**
    * Encodes and decodes row key for flow activity table. The row key is of the
    * form : clusterId!dayTimestamp!user!flowName. dayTimestamp(top of the day
    * timestamp) is a long and rest are strings.
    * <p>
    */
-  final private static class FlowActivityRowKeyConverter implements
-      KeyConverter<FlowActivityRowKey> {
+  final private static class FlowActivityRowKeyConverter
+      implements KeyConverter<FlowActivityRowKey>,
+      KeyConverterToString<FlowActivityRowKey> {
 
     private FlowActivityRowKeyConverter() {
     }
@@ -192,5 +215,33 @@ public class FlowActivityRowKey {
               Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
       return new FlowActivityRowKey(clusterId, dayTs, userId, flowName);
     }
+
+    @Override
+    public String encodeAsString(FlowActivityRowKey key) {
+      if (key.getDayTimestamp() == null) {
+        return TimelineReaderUtils
+            .joinAndEscapeStrings(new String[] {key.clusterId});
+      } else if (key.getUserId() == null) {
+        return TimelineReaderUtils.joinAndEscapeStrings(
+            new String[] {key.clusterId, key.dayTs.toString()});
+      } else if (key.getFlowName() == null) {
+        return TimelineReaderUtils.joinAndEscapeStrings(
+            new String[] {key.clusterId, key.dayTs.toString(), key.userId});
+      }
+      return TimelineReaderUtils.joinAndEscapeStrings(new String[] {
+          key.clusterId, key.dayTs.toString(), key.userId, key.flowName});
+    }
+
+    @Override
+    public FlowActivityRowKey decodeFromString(String encodedRowKey) {
+      List<String> split = TimelineReaderUtils.split(encodedRowKey);
+      if (split == null || split.size() != 4) {
+        throw new IllegalArgumentException(
+            "Invalid row key for flow activity.");
+      }
+      Long dayTs = Long.valueOf(split.get(1));
+      return new FlowActivityRowKey(split.get(0), dayTs, split.get(2),
+          split.get(3));
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1862a030/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.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/FlowActivityEntityReader.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/FlowActivityEntityReader.java
index c741d0e..a1cdb29 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/FlowActivityEntityReader.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/FlowActivityEntityReader.java
@@ -34,6 +34,7 @@ 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.storage.common.BaseTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongKeyConverter;
@@ -41,6 +42,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityCo
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
+import org.apache.hadoop.yarn.webapp.BadRequestException;
 
 import com.google.common.base.Preconditions;
 
@@ -110,11 +112,30 @@ class FlowActivityEntityReader extends TimelineEntityReader {
       Connection conn, FilterList filterList) throws IOException {
     Scan scan = new Scan();
     String clusterId = getContext().getClusterId();
-    if (getFilters().getCreatedTimeBegin() == 0L &&
-        getFilters().getCreatedTimeEnd() == Long.MAX_VALUE) {
+    if (getFilters().getFromId() == null
+        && getFilters().getCreatedTimeBegin() == 0L
+        && getFilters().getCreatedTimeEnd() == Long.MAX_VALUE) {
        // All records have to be chosen.
       scan.setRowPrefixFilter(new FlowActivityRowKeyPrefix(clusterId)
           .getRowKeyPrefix());
+    } else if (getFilters().getFromId() != null) {
+      FlowActivityRowKey key = null;
+      try {
+        key =
+            FlowActivityRowKey.parseRowKeyFromString(getFilters().getFromId());
+      } catch (IllegalArgumentException e) {
+        throw new BadRequestException("Invalid filter fromid is provided.");
+      }
+      if (!clusterId.equals(key.getClusterId())) {
+        throw new BadRequestException(
+            "fromid doesn't belong to clusterId=" + clusterId);
+      }
+      scan.setStartRow(key.getRowKey());
+      scan.setStopRow(
+          new FlowActivityRowKeyPrefix(clusterId,
+              (getFilters().getCreatedTimeBegin() <= 0 ? 0
+                  : (getFilters().getCreatedTimeBegin() - 1)))
+                      .getRowKeyPrefix());
     } else {
       scan.setStartRow(new FlowActivityRowKeyPrefix(clusterId, getFilters()
           .getCreatedTimeEnd()).getRowKeyPrefix());
@@ -157,7 +178,8 @@ class FlowActivityEntityReader extends TimelineEntityReader {
       flowRun.setId(flowRun.getId());
       flowActivity.addFlowRun(flowRun);
     }
-
+    flowActivity.getInfo().put(TimelineReaderUtils.FROMID_KEY,
+        rowKey.getRowKeyAsString());
     return flowActivity;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1862a030/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
index cbd2273..bac5f85 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
@@ -225,6 +226,26 @@ public class TestRowKeys {
   }
 
   @Test
+  public void testFlowActivityRowKeyAsString() {
+    String cluster = "cl" + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR + "uster"
+        + TimelineReaderUtils.DEFAULT_ESCAPE_CHAR;
+    String user = TimelineReaderUtils.DEFAULT_ESCAPE_CHAR + "user";
+    String fName = "dummy_" + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR
+        + TimelineReaderUtils.DEFAULT_ESCAPE_CHAR + "flow"
+        + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR;
+    Long ts = 1459900830000L;
+    Long dayTimestamp = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts);
+    String rowKeyAsString =
+        new FlowActivityRowKey(cluster, ts, user, fName).getRowKeyAsString();
+    FlowActivityRowKey rowKey =
+        FlowActivityRowKey.parseRowKeyFromString(rowKeyAsString);
+    assertEquals(cluster, rowKey.getClusterId());
+    assertEquals(dayTimestamp, rowKey.getDayTimestamp());
+    assertEquals(user, rowKey.getUserId());
+    assertEquals(fName, rowKey.getFlowName());
+  }
+
+  @Test
   public void testFlowRunRowKey() {
     byte[] byteRowKey =
         new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID).getRowKey();


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


[13/50] [abbrv] hadoop git commit: YARN-6069. CORS support in timeline v2 (Rohith Sharma K S via Varun Saxena)

Posted by va...@apache.org.
YARN-6069. CORS support in timeline v2 (Rohith Sharma K S via Varun Saxena)

(cherry picked from commit ab192fd58358faff6880f3e0e867d7bdd4bb043a)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 70b788a3190ccf4dd5626b9699ddba52473ef33c
Parents: 4aad668
Author: Varun Saxena <va...@apache.org>
Authored: Thu Feb 23 11:15:51 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:44 2017 +0530

----------------------------------------------------------------------
 .../src/main/resources/yarn-default.xml                  | 11 +++++++++++
 .../timelineservice/reader/TimelineReaderServer.java     |  9 +++++++++
 .../src/site/markdown/TimelineServiceV2.md               |  9 +++++++++
 3 files changed, 29 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/70b788a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 77f7914..4fd10ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -3099,5 +3099,16 @@
     <name>yarn.app.attempt.diagnostics.limit.kc</name>
     <value>64</value>
   </property>
+  
+  <property>
+    <description>
+      Flag to enable cross-origin (CORS) support for timeline service v1.x or
+      Timeline Reader in timeline service v2. For timeline service v2, also add
+      org.apache.hadoop.security.HttpCrossOriginFilterInitializer to the
+      configuration hadoop.http.filter.initializers in core-site.xml.
+    </description>
+    <name>yarn.timeline-service.http-cross-origin.enabled</name>
+    <value>false</value>
+  </property>
 
 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70b788a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.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/TimelineReaderServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
index 8c5e72d..b45fd36 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.http.lib.StaticUserWebFilter;
+import org.apache.hadoop.security.HttpCrossOriginFilterInitializer;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -134,6 +135,14 @@ public class TimelineReaderServer extends CompositeService {
         YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
         WebAppUtils.getTimelineReaderWebAppURL(conf));
     LOG.info("Instantiating TimelineReaderWebApp at " + bindAddress);
+    boolean enableCorsFilter = conf.getBoolean(
+        YarnConfiguration.TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED,
+        YarnConfiguration.TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT);
+    // setup CORS
+    if (enableCorsFilter) {
+      conf.setBoolean(HttpCrossOriginFilterInitializer.PREFIX
+          + HttpCrossOriginFilterInitializer.ENABLED_SUFFIX, true);
+    }
     try {
       HttpServer2.Builder builder = new HttpServer2.Builder()
             .setName("timeline")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70b788a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
index 7d36a4a..8b18474 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
@@ -160,6 +160,15 @@ New configuration parameters that are introduced with v.2 are marked bold.
 | **`yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds`** | The setting that controls how long the final value of a metric of a completed app is retained before merging into the flow sum. Defaults to `259200000` (3 days). This should be set in the HBase cluster. |
 | **`yarn.rm.system-metrics-publisher.emit-container-events`** | The setting that controls whether yarn container metrics is published to the timeline server or not by RM. This configuration setting is for ATS V2. Defaults to `false`. |
 
+#### Enabling CORS support
+To enable cross-origin support (CORS) for the Timeline Service v.2, please set the following configuration parameters:
+
+In yarn-site.xml, set yarn.timeline-service.http-cross-origin.enabled to true.
+
+In core-site.xml, add org.apache.hadoop.security.HttpCrossOriginFilterInitializer to hadoop.http.filter.initializers.
+
+For more configurations used for cross-origin support, refer to [HttpAuthentication](../../hadoop-project-dist/hadoop-common/HttpAuthentication.html#CORS). Please note that yarn.timeline-service.http-cross-origin.enabled, if set to true, overrides hadoop.http.cross-origin.enabled.
+
 
 
 ### <a name="Enabling_Timeline_Service_v2"></a>Enabling Timeline Service v.2


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


[29/50] [abbrv] hadoop git commit: YARN-6561. Update exception message during timeline collector aux service initialization. (Vrushali C via Haibo Chen)

Posted by va...@apache.org.
YARN-6561. Update exception message during timeline collector aux service initialization. (Vrushali C via Haibo Chen)

(cherry picked from commit 4c8b9d23a1b4f92bf370617a0b42ac68bdcf33ac)


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

Branch: refs/heads/YARN-5355_branch2
Commit: f1f7d6534d8f4a9a4f9bf4b1c83b2dce463e40aa
Parents: f07a97c
Author: Haibo Chen <ha...@cloudera.com>
Authored: Tue May 9 21:37:30 2017 -0700
Committer: Haibo Chen <ha...@cloudera.com>
Committed: Tue May 9 22:04:30 2017 -0700

----------------------------------------------------------------------
 .../collector/PerNodeTimelineCollectorsAuxService.java        | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f1f7d653/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.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/collector/PerNodeTimelineCollectorsAuxService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
index 041e7c2..266bd04 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
@@ -73,7 +73,12 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
     if (!YarnConfiguration.timelineServiceV2Enabled(conf)) {
-      throw new YarnException("Timeline service v2 is not enabled");
+      throw new YarnException(
+          "Looks like timeline_collector is set as an auxillary service in "
+              + YarnConfiguration.NM_AUX_SERVICES
+              + ". But Timeline service v2 is not enabled,"
+              + " so timeline_collector needs to be removed"
+              + " from that list of auxillary services.");
     }
     collectorLingerPeriod =
         conf.getLong(YarnConfiguration.ATS_APP_COLLECTOR_LINGER_PERIOD_IN_MS,


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


[33/50] [abbrv] hadoop git commit: YARN-6604. Allow metric TTL for Application table to be specified through cmd (Haibo Chen via Varun Saxena)

Posted by va...@apache.org.
YARN-6604. Allow metric TTL for Application table to be specified through cmd (Haibo Chen via Varun Saxena)

(cherry picked from commit 0b7bff706e9a5c4a17e0f46deceb2521168d25b9)


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

Branch: refs/heads/YARN-5355_branch2
Commit: fe6884d3694a39f187ba0a7b6bcb76eec00294f3
Parents: 274c22f
Author: Varun Saxena <va...@apache.org>
Authored: Wed Jun 7 21:51:07 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Jun 7 21:57:55 2017 +0530

----------------------------------------------------------------------
 .../storage/TimelineSchemaCreator.java          | 36 ++++++++++++++------
 1 file changed, 26 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe6884d3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.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/TimelineSchemaCreator.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/TimelineSchemaCreator.java
index a9c74d2..f93c977 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/TimelineSchemaCreator.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/TimelineSchemaCreator.java
@@ -61,9 +61,10 @@ public final class TimelineSchemaCreator {
   final static String NAME = TimelineSchemaCreator.class.getSimpleName();
   private static final Log LOG = LogFactory.getLog(TimelineSchemaCreator.class);
   private static final String SKIP_EXISTING_TABLE_OPTION_SHORT = "s";
+  private static final String APP_METRICS_TTL_OPTION_SHORT = "ma";
   private static final String APP_TABLE_NAME_SHORT = "a";
   private static final String APP_TO_FLOW_TABLE_NAME_SHORT = "a2f";
-  private static final String TTL_OPTION_SHORT = "m";
+  private static final String ENTITY_METRICS_TTL_OPTION_SHORT = "me";
   private static final String ENTITY_TABLE_NAME_SHORT = "e";
   private static final String HELP_SHORT = "h";
   private static final String CREATE_TABLES_SHORT = "c";
@@ -91,12 +92,12 @@ public final class TimelineSchemaCreator {
       if (StringUtils.isNotBlank(entityTableName)) {
         hbaseConf.set(EntityTable.TABLE_NAME_CONF_NAME, entityTableName);
       }
-      // Grab the TTL argument
-      String entityTableTTLMetrics =commandLine.getOptionValue(
-          TTL_OPTION_SHORT);
-      if (StringUtils.isNotBlank(entityTableTTLMetrics)) {
-        int metricsTTL = Integer.parseInt(entityTableTTLMetrics);
-        new EntityTable().setMetricsTTL(metricsTTL, hbaseConf);
+      // Grab the entity metrics TTL
+      String entityTableMetricsTTL = commandLine.getOptionValue(
+          ENTITY_METRICS_TTL_OPTION_SHORT);
+      if (StringUtils.isNotBlank(entityTableMetricsTTL)) {
+        int entityMetricsTTL = Integer.parseInt(entityTableMetricsTTL);
+        new EntityTable().setMetricsTTL(entityMetricsTTL, hbaseConf);
       }
       // Grab the appToflowTableName argument
       String appToflowTableName = commandLine.getOptionValue(
@@ -111,6 +112,13 @@ public final class TimelineSchemaCreator {
         hbaseConf.set(ApplicationTable.TABLE_NAME_CONF_NAME,
             applicationTableName);
       }
+      // Grab the application metrics TTL
+      String applicationTableMetricsTTL = commandLine.getOptionValue(
+          APP_METRICS_TTL_OPTION_SHORT);
+      if (StringUtils.isNotBlank(applicationTableMetricsTTL)) {
+        int appMetricsTTL = Integer.parseInt(applicationTableMetricsTTL);
+        new ApplicationTable().setMetricsTTL(appMetricsTTL, hbaseConf);
+      }
 
       // create all table schemas in hbase
       final boolean skipExisting = commandLine.hasOption(
@@ -149,9 +157,9 @@ public final class TimelineSchemaCreator {
     o.setRequired(false);
     options.addOption(o);
 
-    o = new Option(TTL_OPTION_SHORT, "metricsTTL", true,
+    o = new Option(ENTITY_METRICS_TTL_OPTION_SHORT, "entityMetricsTTL", true,
         "TTL for metrics column family");
-    o.setArgName("metricsTTL");
+    o.setArgName("entityMetricsTTL");
     o.setRequired(false);
     options.addOption(o);
 
@@ -167,6 +175,12 @@ public final class TimelineSchemaCreator {
     o.setRequired(false);
     options.addOption(o);
 
+    o = new Option(APP_METRICS_TTL_OPTION_SHORT, "applicationMetricsTTL", true,
+        "TTL for metrics column family");
+    o.setArgName("applicationMetricsTTL");
+    o.setRequired(false);
+    options.addOption(o);
+
     // Options without an argument
     // No need to set arg name since we do not need an argument here
     o = new Option(SKIP_EXISTING_TABLE_OPTION_SHORT, "skipExistingTable",
@@ -197,12 +211,14 @@ public final class TimelineSchemaCreator {
     usage.append("The Optional options for creating tables include: \n");
     usage.append("[-entityTableName <Entity Table Name>] " +
         "The name of the Entity table\n");
-    usage.append("[-metricsTTL <Entity Table Metrics TTL>]" +
+    usage.append("[-entityMetricsTTL <Entity Table Metrics TTL>]" +
         " TTL for metrics in the Entity table\n");
     usage.append("[-appToflowTableName <AppToflow Table Name>]" +
         " The name of the AppToFlow table\n");
     usage.append("[-applicationTableName <Application Table Name>]" +
         " The name of the Application table\n");
+    usage.append("[-applicationMetricsTTL <Application Table Metrics TTL>]" +
+        " TTL for metrics in the Application table\n");
     usage.append("[-skipExistingTable] Whether to skip existing" +
         " hbase tables\n");
     System.out.println(usage.toString());


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


[40/50] [abbrv] hadoop git commit: YARN-6888. Refactor AppLevelTimelineCollector such that RM does not have aggregator threads created. Contributed by Vrushali C.

Posted by va...@apache.org.
YARN-6888. Refactor AppLevelTimelineCollector such that RM does not have aggregator threads created. Contributed by Vrushali C.

(cherry picked from commit 20dd6d18b7787e67ef96f3b6b92ea8415a8650fd)


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

Branch: refs/heads/YARN-5355_branch2
Commit: bb4f44038c6244116d27e0b0c43f8fd79d0063e7
Parents: 1ee2a98
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Fri Jul 28 11:47:16 2017 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Fri Jul 28 11:51:23 2017 +0530

----------------------------------------------------------------------
 .../collector/AppLevelTimelineCollector.java    |  87 +----------
 .../AppLevelTimelineCollectorWithAgg.java       | 150 +++++++++++++++++++
 .../PerNodeTimelineCollectorsAuxService.java    |   2 +-
 .../TestNMTimelineCollectorManager.java         |   4 +-
 4 files changed, 155 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bb4f4403/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.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/collector/AppLevelTimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
index 0b05309..c481dbe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.collector;
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -26,19 +25,10 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 import com.google.common.base.Preconditions;
 
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
 /**
  * Service that handles writes to the timeline service and writes them to the
  * backing storage for a given YARN application.
@@ -50,15 +40,8 @@ import java.util.concurrent.TimeUnit;
 public class AppLevelTimelineCollector extends TimelineCollector {
   private static final Log LOG = LogFactory.getLog(TimelineCollector.class);
 
-  private final static int AGGREGATION_EXECUTOR_NUM_THREADS = 1;
-  private final static int AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS = 15;
-  private static Set<String> entityTypesSkipAggregation
-      = initializeSkipSet();
-
   private final ApplicationId appId;
   private final TimelineCollectorContext context;
-  private ScheduledThreadPoolExecutor appAggregationExecutor;
-  private AppLevelAggregator appAggregator;
   private UserGroupInformation currentUser;
 
   public AppLevelTimelineCollector(ApplicationId appId) {
@@ -68,12 +51,8 @@ public class AppLevelTimelineCollector extends TimelineCollector {
     context = new TimelineCollectorContext();
   }
 
-  private static Set<String> initializeSkipSet() {
-    Set<String> result = new HashSet<>();
-    result.add(TimelineEntityType.YARN_APPLICATION.toString());
-    result.add(TimelineEntityType.YARN_FLOW_RUN.toString());
-    result.add(TimelineEntityType.YARN_FLOW_ACTIVITY.toString());
-    return result;
+  public UserGroupInformation getCurrentUser() {
+    return currentUser;
   }
 
   @Override
@@ -91,29 +70,11 @@ public class AppLevelTimelineCollector extends TimelineCollector {
 
   @Override
   protected void serviceStart() throws Exception {
-    // Launch the aggregation thread
-    appAggregationExecutor = new ScheduledThreadPoolExecutor(
-        AppLevelTimelineCollector.AGGREGATION_EXECUTOR_NUM_THREADS,
-        new ThreadFactoryBuilder()
-            .setNameFormat("TimelineCollector Aggregation thread #%d")
-            .build());
-    appAggregator = new AppLevelAggregator();
-    appAggregationExecutor.scheduleAtFixedRate(appAggregator,
-        AppLevelTimelineCollector.AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS,
-        AppLevelTimelineCollector.AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS,
-        TimeUnit.SECONDS);
     super.serviceStart();
   }
 
   @Override
   protected void serviceStop() throws Exception {
-    appAggregationExecutor.shutdown();
-    if (!appAggregationExecutor.awaitTermination(10, TimeUnit.SECONDS)) {
-      LOG.info("App-level aggregator shutdown timed out, shutdown now. ");
-      appAggregationExecutor.shutdownNow();
-    }
-    // Perform one round of aggregation after the aggregation executor is done.
-    appAggregator.aggregate();
     super.serviceStop();
   }
 
@@ -122,48 +83,4 @@ public class AppLevelTimelineCollector extends TimelineCollector {
     return context;
   }
 
-  @Override
-  protected Set<String> getEntityTypesSkipAggregation() {
-    return entityTypesSkipAggregation;
-  }
-
-  private class AppLevelAggregator implements Runnable {
-
-    private void aggregate() {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("App-level real-time aggregating");
-      }
-      if (!isReadyToAggregate()) {
-        LOG.warn("App-level collector is not ready, skip aggregation. ");
-        return;
-      }
-      try {
-        TimelineCollectorContext currContext = getTimelineEntityContext();
-        Map<String, AggregationStatusTable> aggregationGroups
-            = getAggregationGroups();
-        if (aggregationGroups == null
-            || aggregationGroups.isEmpty()) {
-          LOG.debug("App-level collector is empty, skip aggregation. ");
-          return;
-        }
-        TimelineEntity resultEntity = TimelineCollector.aggregateWithoutGroupId(
-            aggregationGroups, currContext.getAppId(),
-            TimelineEntityType.YARN_APPLICATION.toString());
-        TimelineEntities entities = new TimelineEntities();
-        entities.addEntity(resultEntity);
-        putEntitiesAsync(entities, currentUser);
-      } catch (Exception e) {
-        LOG.error("Error aggregating timeline metrics", e);
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("App-level real-time aggregation complete");
-      }
-    }
-
-    @Override
-    public void run() {
-      aggregate();
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bb4f4403/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollectorWithAgg.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/collector/AppLevelTimelineCollectorWithAgg.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollectorWithAgg.java
new file mode 100644
index 0000000..ac91275
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollectorWithAgg.java
@@ -0,0 +1,150 @@
+/**
+ * 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.collector;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+import org.apache.hadoop.conf.Configuration;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Service that handles aggregations for applications
+ * and makes use of {@link AppLevelTimelineCollector} class for
+ * writes to Timeline Service.
+ *
+ * App-related lifecycle management is handled by this service.
+ */
+@Private
+@Unstable
+public class AppLevelTimelineCollectorWithAgg
+    extends AppLevelTimelineCollector {
+  private static final Log LOG = LogFactory.getLog(TimelineCollector.class);
+
+  private final static int AGGREGATION_EXECUTOR_NUM_THREADS = 1;
+  private final static int AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS = 15;
+  private static Set<String> entityTypesSkipAggregation
+      = initializeSkipSet();
+
+  private ScheduledThreadPoolExecutor appAggregationExecutor;
+  private AppLevelAggregator appAggregator;
+
+  public AppLevelTimelineCollectorWithAgg(ApplicationId appId) {
+    super(appId);
+  }
+
+  private static Set<String> initializeSkipSet() {
+    Set<String> result = new HashSet<>();
+    result.add(TimelineEntityType.YARN_APPLICATION.toString());
+    result.add(TimelineEntityType.YARN_FLOW_RUN.toString());
+    result.add(TimelineEntityType.YARN_FLOW_ACTIVITY.toString());
+    return result;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    // Launch the aggregation thread
+    appAggregationExecutor = new ScheduledThreadPoolExecutor(
+        AppLevelTimelineCollectorWithAgg.AGGREGATION_EXECUTOR_NUM_THREADS,
+        new ThreadFactoryBuilder()
+            .setNameFormat("TimelineCollector Aggregation thread #%d")
+            .build());
+    appAggregator = new AppLevelAggregator();
+    appAggregationExecutor.scheduleAtFixedRate(appAggregator,
+        AppLevelTimelineCollectorWithAgg.
+          AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS,
+        AppLevelTimelineCollectorWithAgg.
+          AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS,
+        TimeUnit.SECONDS);
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    appAggregationExecutor.shutdown();
+    if (!appAggregationExecutor.awaitTermination(10, TimeUnit.SECONDS)) {
+      LOG.info("App-level aggregator shutdown timed out, shutdown now. ");
+      appAggregationExecutor.shutdownNow();
+    }
+    // Perform one round of aggregation after the aggregation executor is done.
+    appAggregator.aggregate();
+    super.serviceStop();
+  }
+
+  @Override
+  protected Set<String> getEntityTypesSkipAggregation() {
+    return entityTypesSkipAggregation;
+  }
+
+  private class AppLevelAggregator implements Runnable {
+
+    private void aggregate() {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("App-level real-time aggregating");
+      }
+      if (!isReadyToAggregate()) {
+        LOG.warn("App-level collector is not ready, skip aggregation. ");
+        return;
+      }
+      try {
+        TimelineCollectorContext currContext = getTimelineEntityContext();
+        Map<String, AggregationStatusTable> aggregationGroups
+            = getAggregationGroups();
+        if (aggregationGroups == null
+            || aggregationGroups.isEmpty()) {
+          LOG.debug("App-level collector is empty, skip aggregation. ");
+          return;
+        }
+        TimelineEntity resultEntity = TimelineCollector.aggregateWithoutGroupId(
+            aggregationGroups, currContext.getAppId(),
+            TimelineEntityType.YARN_APPLICATION.toString());
+        TimelineEntities entities = new TimelineEntities();
+        entities.addEntity(resultEntity);
+        putEntitiesAsync(entities, getCurrentUser());
+      } catch (Exception e) {
+        LOG.error("Error aggregating timeline metrics", e);
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("App-level real-time aggregation complete");
+      }
+    }
+
+    @Override
+    public void run() {
+      aggregate();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bb4f4403/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.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/collector/PerNodeTimelineCollectorsAuxService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
index 266bd04..93e5666 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
@@ -118,7 +118,7 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
    */
   public boolean addApplication(ApplicationId appId) {
     AppLevelTimelineCollector collector =
-        new AppLevelTimelineCollector(appId);
+        new AppLevelTimelineCollectorWithAgg(appId);
     return (collectorManager.putIfAbsent(appId, collector)
         == collector);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bb4f4403/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestNMTimelineCollectorManager.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/collector/TestNMTimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestNMTimelineCollectorManager.java
index 7bc89c5..a59f8c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestNMTimelineCollectorManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestNMTimelineCollectorManager.java
@@ -95,7 +95,7 @@ public class TestNMTimelineCollectorManager {
       Callable<Boolean> task = new Callable<Boolean>() {
         public Boolean call() {
           AppLevelTimelineCollector collector =
-              new AppLevelTimelineCollector(appId);
+              new AppLevelTimelineCollectorWithAgg(appId);
           return (collectorManager.putIfAbsent(appId, collector) == collector);
         }
       };
@@ -126,7 +126,7 @@ public class TestNMTimelineCollectorManager {
       Callable<Boolean> task = new Callable<Boolean>() {
         public Boolean call() {
           AppLevelTimelineCollector collector =
-              new AppLevelTimelineCollector(appId);
+              new AppLevelTimelineCollectorWithAgg(appId);
           boolean successPut =
               (collectorManager.putIfAbsent(appId, collector) == collector);
           return successPut && collectorManager.remove(appId);


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


[04/50] [abbrv] hadoop git commit: YARN-6256. Add FROM_ID info key for timeline entities in reader response (Rohith Sharma K S via Varun Saxena)

Posted by va...@apache.org.
YARN-6256. Add FROM_ID info key for timeline entities in reader response (Rohith Sharma K S via Varun Saxena)

(cherry picked from commit 5d9ad152a1082a7c9b8edaf57a88ae471a537599)


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

Branch: refs/heads/YARN-5355_branch2
Commit: a391f54f7729c92d06d6c37370035d4442ce59c7
Parents: 6c993a3
Author: Varun Saxena <va...@apache.org>
Authored: Tue Mar 7 23:54:38 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:44 2017 +0530

----------------------------------------------------------------------
 ...stTimelineReaderWebServicesHBaseStorage.java |  33 +--
 .../storage/TestHBaseTimelineStorageApps.java   |  19 +-
 .../TestHBaseTimelineStorageEntities.java       |  21 +-
 .../storage/application/ApplicationRowKey.java  |  49 +++-
 .../storage/entity/EntityRowKey.java            |  56 ++++-
 .../storage/flow/FlowRunRowKey.java             |  47 +++-
 .../storage/reader/ApplicationEntityReader.java |  28 ++-
 .../storage/reader/FlowRunEntityReader.java     |  32 +--
 .../storage/reader/GenericEntityReader.java     |  25 +-
 .../storage/common/TestRowKeys.java             |  21 --
 .../storage/common/TestRowKeysAsString.java     | 115 ++++++++++
 .../reader/TimelineEntityFilters.java           |  29 +--
 .../reader/TimelineReaderWebServices.java       | 227 +++++++------------
 .../reader/TimelineReaderWebServicesUtils.java  |   4 +-
 14 files changed, 445 insertions(+), 261 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/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 6386183..6e416c8 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
@@ -816,7 +816,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
       assertEquals(2, entities1.size());
       for (TimelineEntity entity : entities1) {
         assertNotNull(entity.getInfo());
-        assertEquals(1, entity.getInfo().size());
+        assertEquals(2, entity.getInfo().size());
         String uid =
             (String) entity.getInfo().get(TimelineReaderManager.UID_KEY);
         assertNotNull(uid);
@@ -844,7 +844,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
       assertEquals(2, entities2.size());
       for (TimelineEntity entity : entities2) {
         assertNotNull(entity.getInfo());
-        assertEquals(1, entity.getInfo().size());
+        assertEquals(2, entity.getInfo().size());
         String uid =
             (String) entity.getInfo().get(TimelineReaderManager.UID_KEY);
         assertNotNull(uid);
@@ -1408,8 +1408,9 @@ public class TestTimelineReaderWebServicesHBaseStorage
         infoCnt += entity.getInfo().size();
         assertTrue(entity.getId().equals("entity2"));
       }
-      // Includes UID in info field even if fields not specified as INFO.
-      assertEquals(1, infoCnt);
+      // Includes UID and FROM_ID in info field even if fields not specified as
+      // INFO.
+      assertEquals(2, infoCnt);
 
       // infofilters=(info1 eq cluster1 AND info4 eq 35000) OR
       // (info1 eq cluster2 AND info2 eq 2.0)
@@ -1427,8 +1428,8 @@ public class TestTimelineReaderWebServicesHBaseStorage
         infoCnt += entity.getInfo().size();
         assertTrue(entity.getId().equals("entity2"));
       }
-      // Includes UID in info field.
-      assertEquals(4, infoCnt);
+      // Includes UID and FROM_ID in info field.
+      assertEquals(5, infoCnt);
 
       // Test for behavior when compare op is ne(not equals) vs ene
       // (exists and not equals). info3 does not exist for entity2. For ne,
@@ -2159,8 +2160,8 @@ public class TestTimelineReaderWebServicesHBaseStorage
       // verify for entity-10 to entity-7 in descending order.
       TimelineEntity entity = verifyPaginatedEntites(entities, limit, 10);
 
-      queryParam = "?limit=" + limit + "&&fromidprefix=" + entity.getIdPrefix()
-          + "&&fromid=" + entity.getId();
+      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>>() {
@@ -2168,7 +2169,8 @@ public class TestTimelineReaderWebServicesHBaseStorage
       // verify for entity-7 to entity-4 in descending order.
       entity = verifyPaginatedEntites(entities, limit, 7);
 
-      queryParam = "?limit=" + limit + "&&fromidprefix=" + entity.getIdPrefix();
+      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>>() {
@@ -2176,7 +2178,8 @@ public class TestTimelineReaderWebServicesHBaseStorage
       // verify for entity-4 to entity-1 in descending order.
       entity = verifyPaginatedEntites(entities, limit, 4);
 
-      queryParam = "?limit=" + limit + "&&fromidprefix=" + entity.getIdPrefix();
+      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>>() {
@@ -2252,7 +2255,8 @@ public class TestTimelineReaderWebServicesHBaseStorage
       TimelineEntity entity10 = entities.get(limit - 1);
 
       uri =
-          URI.create(resourceUri + queryParam + "&fromid=" + entity10.getId());
+          URI.create(resourceUri + queryParam + "&fromid="
+              + entity10.getInfo().get(TimelineReaderUtils.FROMID_KEY));
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
       });
@@ -2296,7 +2300,8 @@ public class TestTimelineReaderWebServicesHBaseStorage
       TimelineEntity entity3 = entities.get(limit - 1);
 
       uri =
-          URI.create(resourceUri + queryParam + "&fromid=" + entity3.getId());
+          URI.create(resourceUri + queryParam + "&fromid="
+              + entity3.getInfo().get(TimelineReaderUtils.FROMID_KEY));
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
       });
@@ -2340,7 +2345,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
       TimelineEntity entity2 = entities.get(limit - 1);
 
       uri = URI.create(resourceUri + queryParam + "&fromid="
-          + entity2.getInfo().get("SYSTEM_INFO_FLOW_RUN_ID"));
+          + entity2.getInfo().get(TimelineReaderUtils.FROMID_KEY));
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
       });
@@ -2350,7 +2355,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
       assertEquals(entity3, entities.get(1));
 
       uri = URI.create(resourceUri + queryParam + "&fromid="
-          + entity3.getInfo().get("SYSTEM_INFO_FLOW_RUN_ID"));
+          + entity3.getInfo().get(TimelineReaderUtils.FROMID_KEY));
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<List<TimelineEntity>>() {
       });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.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/storage/TestHBaseTimelineStorageApps.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/storage/TestHBaseTimelineStorageApps.java
index 31d56b2..6bb8de1 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/storage/TestHBaseTimelineStorageApps.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/storage/TestHBaseTimelineStorageApps.java
@@ -388,6 +388,8 @@ public class TestHBaseTimelineStorageApps {
           e1.getType());
       assertEquals(cTime, e1.getCreatedTime());
       Map<String, Object> infoMap2 = e1.getInfo();
+      // fromid key is added by storage. Remove it for comparision.
+      infoMap2.remove("FROM_ID");
       assertEquals(infoMap, infoMap2);
 
       Map<String, Set<String>> isRelatedTo2 = e1.getIsRelatedToEntities();
@@ -447,6 +449,9 @@ public class TestHBaseTimelineStorageApps {
       assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
           e1.getType());
       assertEquals(cTime, e1.getCreatedTime());
+      infoMap2 = e1.getInfo();
+      // fromid key is added by storage. Remove it for comparison.
+      infoMap2.remove("FROM_ID");
       assertEquals(infoMap, e1.getInfo());
       assertEquals(isRelatedTo, e1.getIsRelatedToEntities());
       assertEquals(relatesTo, e1.getRelatesToEntities());
@@ -678,7 +683,7 @@ public class TestHBaseTimelineStorageApps {
     }
     assertEquals(5, cfgCnt);
     assertEquals(3, metricCnt);
-    assertEquals(5, infoCnt);
+    assertEquals(8, infoCnt);
     assertEquals(4, eventCnt);
     assertEquals(4, relatesToCnt);
     assertEquals(4, isRelatedToCnt);
@@ -742,7 +747,8 @@ public class TestHBaseTimelineStorageApps {
         TimelineEntityType.YARN_APPLICATION.toString(), null),
         new TimelineDataToRetrieve());
     assertNotNull(e1);
-    assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() &&
+    assertEquals(1, e1.getInfo().size());
+    assertTrue(e1.getConfigs().isEmpty() &&
         e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() &&
         e1.getRelatesToEntities().isEmpty());
     Set<TimelineEntity> es1 = reader.getEntities(
@@ -753,7 +759,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineDataToRetrieve());
     assertEquals(3, es1.size());
     for (TimelineEntity e : es1) {
-      assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() &&
+      assertEquals(1, e1.getInfo().size());
+      assertTrue(e.getConfigs().isEmpty() &&
           e.getMetrics().isEmpty() && e.getIsRelatedToEntities().isEmpty() &&
           e.getRelatesToEntities().isEmpty());
     }
@@ -786,7 +793,7 @@ public class TestHBaseTimelineStorageApps {
       isRelatedToCnt += entity.getIsRelatedToEntities().size();
       infoCnt += entity.getInfo().size();
     }
-    assertEquals(0, infoCnt);
+    assertEquals(3, infoCnt);
     assertEquals(4, isRelatedToCnt);
     assertEquals(3, metricsCnt);
   }
@@ -1768,7 +1775,7 @@ public class TestHBaseTimelineStorageApps {
     for (TimelineEntity entity : entities) {
       infoCnt += entity.getInfo().size();
     }
-    assertEquals(5, infoCnt);
+    assertEquals(7, infoCnt);
 
     TimelineFilterList infoFilterList1 = new TimelineFilterList(
         new TimelineKeyValueFilter(
@@ -1785,7 +1792,7 @@ public class TestHBaseTimelineStorageApps {
     for (TimelineEntity entity : entities) {
       infoCnt += entity.getInfo().size();
     }
-    assertEquals(3, infoCnt);
+    assertEquals(4, infoCnt);
 
     TimelineFilterList infoFilterList2 = new TimelineFilterList(
         new TimelineKeyValueFilter(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.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/storage/TestHBaseTimelineStorageEntities.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/storage/TestHBaseTimelineStorageEntities.java
index b188802..7211450 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/storage/TestHBaseTimelineStorageEntities.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/storage/TestHBaseTimelineStorageEntities.java
@@ -309,6 +309,8 @@ public class TestHBaseTimelineStorageEntities {
       assertEquals(type, e1.getType());
       assertEquals(cTime, e1.getCreatedTime());
       Map<String, Object> infoMap2 = e1.getInfo();
+      // fromid key is added by storage. Remove it for comparison.
+      infoMap2.remove("FROM_ID");
       assertEquals(infoMap, infoMap2);
 
       Map<String, Set<String>> isRelatedTo2 = e1.getIsRelatedToEntities();
@@ -334,7 +336,10 @@ public class TestHBaseTimelineStorageEntities {
       assertEquals(id, e1.getId());
       assertEquals(type, e1.getType());
       assertEquals(cTime, e1.getCreatedTime());
-      assertEquals(infoMap, e1.getInfo());
+      infoMap2 = e1.getInfo();
+      // fromid key is added by storage. Remove it for comparision.
+      infoMap2.remove("FROM_ID");
+      assertEquals(infoMap, infoMap2);
       assertEquals(isRelatedTo, e1.getIsRelatedToEntities());
       assertEquals(relatesTo, e1.getRelatesToEntities());
       assertEquals(conf, e1.getConfigs());
@@ -569,7 +574,7 @@ public class TestHBaseTimelineStorageEntities {
     }
     assertEquals(5, cfgCnt);
     assertEquals(3, metricCnt);
-    assertEquals(5, infoCnt);
+    assertEquals(8, infoCnt);
     assertEquals(4, eventCnt);
     assertEquals(4, relatesToCnt);
     assertEquals(4, isRelatedToCnt);
@@ -1122,7 +1127,8 @@ public class TestHBaseTimelineStorageEntities {
         1002345678919L, "application_1231111111_1111", "world", "hello"),
         new TimelineDataToRetrieve());
     assertNotNull(e1);
-    assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() &&
+    assertEquals(1, e1.getInfo().size());
+    assertTrue(e1.getConfigs().isEmpty() &&
         e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() &&
         e1.getRelatesToEntities().isEmpty());
     Set<TimelineEntity> es1 = reader.getEntities(
@@ -1132,9 +1138,10 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineDataToRetrieve());
     assertEquals(3, es1.size());
     for (TimelineEntity e : es1) {
-      assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() &&
+      assertTrue(e.getConfigs().isEmpty() &&
           e.getMetrics().isEmpty() && e.getIsRelatedToEntities().isEmpty() &&
           e.getRelatesToEntities().isEmpty());
+      assertEquals(1, e.getInfo().size());
     }
   }
 
@@ -1163,7 +1170,7 @@ public class TestHBaseTimelineStorageEntities {
       isRelatedToCnt += entity.getIsRelatedToEntities().size();
       infoCnt += entity.getInfo().size();
     }
-    assertEquals(0, infoCnt);
+    assertEquals(3, infoCnt);
     assertEquals(4, isRelatedToCnt);
     assertEquals(3, metricsCnt);
   }
@@ -1599,7 +1606,7 @@ public class TestHBaseTimelineStorageEntities {
     for (TimelineEntity entity : entities) {
       infoCnt += entity.getInfo().size();
     }
-    assertEquals(5, infoCnt);
+    assertEquals(7, infoCnt);
 
     TimelineFilterList infoFilterList1 = new TimelineFilterList(
         new TimelineKeyValueFilter(
@@ -1615,7 +1622,7 @@ public class TestHBaseTimelineStorageEntities {
     for (TimelineEntity entity : entities) {
       infoCnt += entity.getInfo().size();
     }
-    assertEquals(3, infoCnt);
+    assertEquals(4, infoCnt);
 
     TimelineFilterList infoFilterList2 = new TimelineFilterList(
         new TimelineKeyValueFilter(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.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/application/ApplicationRowKey.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/application/ApplicationRowKey.java
index da62fdf..e89a6a7 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/application/ApplicationRowKey.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/application/ApplicationRowKey.java
@@ -18,9 +18,13 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.storage.application;
 
+import java.util.List;
+
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverterToString;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 
@@ -33,7 +37,7 @@ public class ApplicationRowKey {
   private final String flowName;
   private final Long flowRunId;
   private final String appId;
-  private final KeyConverter<ApplicationRowKey> appRowKeyConverter =
+  private final ApplicationRowKeyConverter appRowKeyConverter =
       new ApplicationRowKeyConverter();
 
   public ApplicationRowKey(String clusterId, String userId, String flowName,
@@ -86,6 +90,24 @@ public class ApplicationRowKey {
   }
 
   /**
+   * Constructs a row key for the application table as follows:
+   * {@code clusterId!userName!flowName!flowRunId!AppId}.
+   * @return String representation of row key.
+   */
+  public String getRowKeyAsString() {
+    return appRowKeyConverter.encodeAsString(this);
+  }
+
+  /**
+   * Given the encoded row key as string, returns the row key as an object.
+   * @param encodedRowKey String representation of row key.
+   * @return A <cite>ApplicationRowKey</cite> object.
+   */
+  public static ApplicationRowKey parseRowKeyFromString(String encodedRowKey) {
+    return new ApplicationRowKeyConverter().decodeFromString(encodedRowKey);
+  }
+
+  /**
    * Encodes and decodes row key for application table. The row key is of the
    * form: clusterId!userName!flowName!flowRunId!appId. flowRunId is a long,
    * appId is encoded and decoded using {@link AppIdKeyConverter} and rest are
@@ -93,7 +115,7 @@ public class ApplicationRowKey {
    * <p>
    */
   final private static class ApplicationRowKeyConverter implements
-      KeyConverter<ApplicationRowKey> {
+      KeyConverter<ApplicationRowKey>, KeyConverterToString<ApplicationRowKey> {
 
     private final KeyConverter<String> appIDKeyConverter =
         new AppIdKeyConverter();
@@ -201,6 +223,29 @@ public class ApplicationRowKey {
       return new ApplicationRowKey(clusterId, userId, flowName, flowRunId,
           appId);
     }
+
+    @Override
+    public String encodeAsString(ApplicationRowKey key) {
+      if (key.clusterId == null || key.userId == null || key.flowName == null
+          || key.flowRunId == null || key.appId == null) {
+        throw new IllegalArgumentException();
+      }
+      return TimelineReaderUtils
+          .joinAndEscapeStrings(new String[] {key.clusterId, key.userId,
+              key.flowName, key.flowRunId.toString(), key.appId});
+    }
+
+    @Override
+    public ApplicationRowKey decodeFromString(String encodedRowKey) {
+      List<String> split = TimelineReaderUtils.split(encodedRowKey);
+      if (split == null || split.size() != 5) {
+        throw new IllegalArgumentException(
+            "Invalid row key for application table.");
+      }
+      Long flowRunId = Long.valueOf(split.get(3));
+      return new ApplicationRowKey(split.get(0), split.get(1), split.get(2),
+          flowRunId, split.get(4));
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.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/entity/EntityRowKey.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/entity/EntityRowKey.java
index a8f1d0c..7bf02f2 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/entity/EntityRowKey.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/entity/EntityRowKey.java
@@ -17,9 +17,13 @@
  */
 package org.apache.hadoop.yarn.server.timelineservice.storage.entity;
 
+import java.util.List;
+
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverterToString;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 
@@ -35,7 +39,7 @@ public class EntityRowKey {
   private final String entityType;
   private final Long entityIdPrefix;
   private final String entityId;
-  private final KeyConverter<EntityRowKey> entityRowKeyConverter =
+  private final EntityRowKeyConverter entityRowKeyConverter =
       new EntityRowKeyConverter();
 
   public EntityRowKey(String clusterId, String userId, String flowName,
@@ -96,7 +100,6 @@ public class EntityRowKey {
 
   /**
    * Given the raw row key as bytes, returns the row key as an object.
-   *
    * @param rowKey byte representation of row key.
    * @return An <cite>EntityRowKey</cite> object.
    */
@@ -105,6 +108,27 @@ public class EntityRowKey {
   }
 
   /**
+   * Constructs a row key for the entity table as follows:
+   * <p>
+   * {@code userName!clusterId!flowName!flowRunId!AppId!
+   * entityType!entityIdPrefix!entityId}.
+   * </p>
+   * @return String representation of row key.
+   */
+  public String getRowKeyAsString() {
+    return entityRowKeyConverter.encodeAsString(this);
+  }
+
+  /**
+   * Given the encoded row key as string, returns the row key as an object.
+   * @param encodedRowKey String representation of row key.
+   * @return A <cite>EntityRowKey</cite> object.
+   */
+  public static EntityRowKey parseRowKeyFromString(String encodedRowKey) {
+    return new EntityRowKeyConverter().decodeFromString(encodedRowKey);
+  }
+
+  /**
    * Encodes and decodes row key for entity table. The row key is of the form :
    * userName!clusterId!flowName!flowRunId!appId!entityType!entityId. flowRunId
    * is a long, appId is encoded/decoded using {@link AppIdKeyConverter} and
@@ -112,7 +136,7 @@ public class EntityRowKey {
    * <p>
    */
   final private static class EntityRowKeyConverter implements
-      KeyConverter<EntityRowKey> {
+      KeyConverter<EntityRowKey>, KeyConverterToString<EntityRowKey> {
 
     private final AppIdKeyConverter appIDKeyConverter = new AppIdKeyConverter();
 
@@ -245,5 +269,31 @@ public class EntityRowKey {
       return new EntityRowKey(clusterId, userId, flowName, flowRunId, appId,
           entityType, entityPrefixId, entityId);
     }
+
+    @Override
+    public String encodeAsString(EntityRowKey key) {
+      if (key.clusterId == null || key.userId == null || key.flowName == null
+          || key.flowRunId == null || key.appId == null
+          || key.entityType == null || key.entityIdPrefix == null
+          || key.entityId == null) {
+        throw new IllegalArgumentException();
+      }
+      return TimelineReaderUtils
+          .joinAndEscapeStrings(new String[] {key.clusterId, key.userId,
+              key.flowName, key.flowRunId.toString(), key.appId, key.entityType,
+              key.entityIdPrefix.toString(), key.entityId});
+    }
+
+    @Override
+    public EntityRowKey decodeFromString(String encodedRowKey) {
+      List<String> split = TimelineReaderUtils.split(encodedRowKey);
+      if (split == null || split.size() != 8) {
+        throw new IllegalArgumentException("Invalid row key for entity table.");
+      }
+      Long flowRunId = Long.valueOf(split.get(3));
+      Long entityIdPrefix = Long.valueOf(split.get(6));
+      return new EntityRowKey(split.get(0), split.get(1), split.get(2),
+          flowRunId, split.get(4), split.get(5), entityIdPrefix, split.get(7));
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.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/flow/FlowRunRowKey.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/flow/FlowRunRowKey.java
index 8fda9a8..7ce91cf 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/flow/FlowRunRowKey.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/flow/FlowRunRowKey.java
@@ -17,8 +17,12 @@
  */
 package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 
+import java.util.List;
+
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverterToString;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 
@@ -70,7 +74,6 @@ public class FlowRunRowKey {
 
   /**
    * Given the raw row key as bytes, returns the row key as an object.
-   *
    * @param rowKey Byte representation of row key.
    * @return A <cite>FlowRunRowKey</cite> object.
    */
@@ -79,6 +82,24 @@ public class FlowRunRowKey {
   }
 
   /**
+   * Constructs a row key for the flow run table as follows:
+   * {@code clusterId!userId!flowName!Flow Run Id}.
+   * @return String representation of row key
+   */
+  public String getRowKeyAsString() {
+    return flowRunRowKeyConverter.encodeAsString(this);
+  }
+
+  /**
+   * Given the encoded row key as string, returns the row key as an object.
+   * @param encodedRowKey String representation of row key.
+   * @return A <cite>FlowRunRowKey</cite> object.
+   */
+  public static FlowRunRowKey parseRowKeyFromString(String encodedRowKey) {
+    return new FlowRunRowKeyConverter().decodeFromString(encodedRowKey);
+  }
+
+  /**
    * returns the Flow Key as a verbose String output.
    * @return String
    */
@@ -101,7 +122,7 @@ public class FlowRunRowKey {
    * <p>
    */
   final private static class FlowRunRowKeyConverter implements
-      KeyConverter<FlowRunRowKey> {
+      KeyConverter<FlowRunRowKey>, KeyConverterToString<FlowRunRowKey> {
 
     private FlowRunRowKeyConverter() {
     }
@@ -186,5 +207,27 @@ public class FlowRunRowKey {
           LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3]));
       return new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
     }
+
+    @Override
+    public String encodeAsString(FlowRunRowKey key) {
+      if (key.clusterId == null || key.userId == null || key.flowName == null
+          || key.flowRunId == null) {
+        throw new IllegalArgumentException();
+      }
+      return TimelineReaderUtils.joinAndEscapeStrings(new String[] {
+          key.clusterId, key.userId, key.flowName, key.flowRunId.toString()});
+    }
+
+    @Override
+    public FlowRunRowKey decodeFromString(String encodedRowKey) {
+      List<String> split = TimelineReaderUtils.split(encodedRowKey);
+      if (split == null || split.size() != 4) {
+        throw new IllegalArgumentException(
+            "Invalid row key for flow run table.");
+      }
+      Long flowRunId = Long.valueOf(split.get(3));
+      return new FlowRunRowKey(split.get(0), split.get(1), split.get(2),
+          flowRunId);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.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/ApplicationEntityReader.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/ApplicationEntityReader.java
index 4e8286d..b4bb005 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/ApplicationEntityReader.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/ApplicationEntityReader.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 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;
@@ -48,11 +49,11 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
 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.webapp.BadRequestException;
 
 import com.google.common.base.Preconditions;
 
@@ -372,18 +373,17 @@ class ApplicationEntityReader extends GenericEntityReader {
           context.getFlowRunId());
       scan.setRowPrefixFilter(applicationRowKeyPrefix.getRowKeyPrefix());
     } else {
-      Long flowRunId = context.getFlowRunId();
-      if (flowRunId == null) {
-        AppToFlowRowKey appToFlowRowKey = new AppToFlowRowKey(
-            getFilters().getFromId());
-        FlowContext flowContext = lookupFlowContext(appToFlowRowKey,
-            context.getClusterId(), hbaseConf, conn);
-        flowRunId = flowContext.getFlowRunId();
+      ApplicationRowKey applicationRowKey = null;
+      try {
+        applicationRowKey =
+            ApplicationRowKey.parseRowKeyFromString(getFilters().getFromId());
+      } catch (IllegalArgumentException e) {
+        throw new BadRequestException("Invalid filter fromid is provided.");
+      }
+      if (!context.getClusterId().equals(applicationRowKey.getClusterId())) {
+        throw new BadRequestException(
+            "fromid doesn't belong to clusterId=" + context.getClusterId());
       }
-
-      ApplicationRowKey applicationRowKey =
-          new ApplicationRowKey(context.getClusterId(), context.getUserId(),
-              context.getFlowName(), flowRunId, getFilters().getFromId());
 
       // set start row
       scan.setStartRow(applicationRowKey.getRowKey());
@@ -497,6 +497,10 @@ class ApplicationEntityReader extends GenericEntityReader {
     if (hasField(fieldsToRetrieve, Field.METRICS)) {
       readMetrics(entity, result, ApplicationColumnPrefix.METRIC);
     }
+
+    ApplicationRowKey rowKey = ApplicationRowKey.parseRowKey(result.getRow());
+    entity.getInfo().put(TimelineReaderUtils.FROMID_KEY,
+        rowKey.getRowKeyAsString());
     return entity;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.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/FlowRunEntityReader.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/FlowRunEntityReader.java
index cedf96a..af043b3 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/FlowRunEntityReader.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/FlowRunEntityReader.java
@@ -39,6 +39,7 @@ 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;
@@ -217,11 +218,17 @@ class FlowRunEntityReader extends TimelineEntityReader {
           context.getUserId(), context.getFlowName());
       scan.setRowPrefixFilter(flowRunRowKeyPrefix.getRowKeyPrefix());
     } else {
-
-      FlowRunRowKey flowRunRowKey =
-          new FlowRunRowKey(context.getClusterId(), context.getUserId(),
-              context.getFlowName(), Long.parseLong(getFilters().getFromId()));
-
+      FlowRunRowKey flowRunRowKey = null;
+      try {
+        flowRunRowKey =
+            FlowRunRowKey.parseRowKeyFromString(getFilters().getFromId());
+      } catch (IllegalArgumentException e) {
+        throw new BadRequestException("Invalid filter fromid is provided.");
+      }
+      if (!context.getClusterId().equals(flowRunRowKey.getClusterId())) {
+        throw new BadRequestException(
+            "fromid doesn't belong to clusterId=" + context.getClusterId());
+      }
       // set start row
       scan.setStartRow(flowRunRowKey.getRowKey());
 
@@ -247,16 +254,11 @@ class FlowRunEntityReader extends TimelineEntityReader {
 
   @Override
   protected TimelineEntity parseEntity(Result result) throws IOException {
-    TimelineReaderContext context = getContext();
     FlowRunEntity flowRun = new FlowRunEntity();
-    flowRun.setUser(context.getUserId());
-    flowRun.setName(context.getFlowName());
-    if (isSingleEntityRead()) {
-      flowRun.setRunId(context.getFlowRunId());
-    } else {
-      FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(result.getRow());
-      flowRun.setRunId(rowKey.getFlowRunId());
-    }
+    FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(result.getRow());
+    flowRun.setRunId(rowKey.getFlowRunId());
+    flowRun.setUser(rowKey.getUserId());
+    flowRun.setName(rowKey.getFlowName());
 
     // read the start time
     Long startTime = (Long) FlowRunColumn.MIN_START_TIME.readResult(result);
@@ -285,6 +287,8 @@ class FlowRunEntityReader extends TimelineEntityReader {
 
     // set the id
     flowRun.setId(flowRun.getId());
+    flowRun.getInfo().put(TimelineReaderUtils.FROMID_KEY,
+        rowKey.getRowKeyAsString());
     return flowRun;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/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 f6904c5..39013d9 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
@@ -40,6 +40,7 @@ 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;
@@ -56,6 +57,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+import org.apache.hadoop.yarn.webapp.BadRequestException;
 
 import com.google.common.base.Preconditions;
 
@@ -475,19 +477,27 @@ class GenericEntityReader extends TimelineEntityReader {
     TimelineReaderContext context = getContext();
     RowKeyPrefix<EntityRowKey> entityRowKeyPrefix = null;
     // default mode, will always scans from beginning of entity type.
-    if (getFilters() == null || getFilters().getFromIdPrefix() == null) {
+    if (getFilters() == null || getFilters().getFromId() == null) {
       entityRowKeyPrefix = new EntityRowKeyPrefix(context.getClusterId(),
           context.getUserId(), context.getFlowName(), context.getFlowRunId(),
           context.getAppId(), context.getEntityType(), null, null);
       scan.setRowPrefixFilter(entityRowKeyPrefix.getRowKeyPrefix());
     } else { // pagination mode, will scan from given entityIdPrefix!enitityId
-      entityRowKeyPrefix = new EntityRowKeyPrefix(context.getClusterId(),
-          context.getUserId(), context.getFlowName(), context.getFlowRunId(),
-          context.getAppId(), context.getEntityType(),
-          getFilters().getFromIdPrefix(), getFilters().getFromId());
+
+      EntityRowKey entityRowKey = null;
+      try {
+        entityRowKey =
+            EntityRowKey.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(entityRowKeyPrefix.getRowKeyPrefix());
+      scan.setStartRow(entityRowKey.getRowKey());
 
       // get the bytes for stop row
       entityRowKeyPrefix = new EntityRowKeyPrefix(context.getClusterId(),
@@ -599,6 +609,9 @@ class GenericEntityReader extends TimelineEntityReader {
     if (hasField(fieldsToRetrieve, Field.METRICS)) {
       readMetrics(entity, result, EntityColumnPrefix.METRIC);
     }
+
+    entity.getInfo().put(TimelineReaderUtils.FROMID_KEY,
+        parseRowKey.getRowKeyAsString());
     return entity;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
index bac5f85..cbd2273 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
@@ -226,26 +225,6 @@ public class TestRowKeys {
   }
 
   @Test
-  public void testFlowActivityRowKeyAsString() {
-    String cluster = "cl" + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR + "uster"
-        + TimelineReaderUtils.DEFAULT_ESCAPE_CHAR;
-    String user = TimelineReaderUtils.DEFAULT_ESCAPE_CHAR + "user";
-    String fName = "dummy_" + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR
-        + TimelineReaderUtils.DEFAULT_ESCAPE_CHAR + "flow"
-        + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR;
-    Long ts = 1459900830000L;
-    Long dayTimestamp = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts);
-    String rowKeyAsString =
-        new FlowActivityRowKey(cluster, ts, user, fName).getRowKeyAsString();
-    FlowActivityRowKey rowKey =
-        FlowActivityRowKey.parseRowKeyFromString(rowKeyAsString);
-    assertEquals(cluster, rowKey.getClusterId());
-    assertEquals(dayTimestamp, rowKey.getDayTimestamp());
-    assertEquals(user, rowKey.getUserId());
-    assertEquals(fName, rowKey.getFlowName());
-  }
-
-  @Test
   public void testFlowRunRowKey() {
     byte[] byteRowKey =
         new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID).getRowKey();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java
new file mode 100644
index 0000000..f0ef720
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java
@@ -0,0 +1,115 @@
+/**
+ * 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.common;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
+import org.junit.Test;
+
+/**
+ * Test for row key as string.
+ */
+public class TestRowKeysAsString {
+
+  private final static String CLUSTER =
+      "cl" + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR + "uster"
+          + TimelineReaderUtils.DEFAULT_ESCAPE_CHAR;
+  private final static String USER =
+      TimelineReaderUtils.DEFAULT_ESCAPE_CHAR + "user";
+  private final static String FLOW_NAME =
+      "dummy_" + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR
+          + TimelineReaderUtils.DEFAULT_ESCAPE_CHAR + "flow"
+          + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR;
+  private final static Long FLOW_RUN_ID = System.currentTimeMillis();
+  private final static String APPLICATION_ID =
+      ApplicationId.newInstance(System.currentTimeMillis(), 1).toString();
+
+  @Test(timeout = 10000)
+  public void testApplicationRow() {
+    String rowKeyAsString = new ApplicationRowKey(CLUSTER, USER, FLOW_NAME,
+        FLOW_RUN_ID, APPLICATION_ID).getRowKeyAsString();
+    ApplicationRowKey rowKey =
+        ApplicationRowKey.parseRowKeyFromString(rowKeyAsString);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(USER, rowKey.getUserId());
+    assertEquals(FLOW_NAME, rowKey.getFlowName());
+    assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
+    assertEquals(APPLICATION_ID, rowKey.getAppId());
+  }
+
+  @Test(timeout = 10000)
+  public void testEntityRowKey() {
+    char del = TimelineReaderUtils.DEFAULT_DELIMITER_CHAR;
+    char esc = TimelineReaderUtils.DEFAULT_ESCAPE_CHAR;
+    String id = del + esc + "ent" + esc + del + "ity" + esc + del + esc + "id"
+        + esc + del + esc;
+    String type = "entity" + esc + del + esc + "Type";
+    TimelineEntity entity = new TimelineEntity();
+    entity.setId(id);
+    entity.setType(type);
+    entity.setIdPrefix(54321);
+
+    String rowKeyAsString =
+        new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID,
+            entity.getType(), entity.getIdPrefix(), entity.getId())
+                .getRowKeyAsString();
+    EntityRowKey rowKey = EntityRowKey.parseRowKeyFromString(rowKeyAsString);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(USER, rowKey.getUserId());
+    assertEquals(FLOW_NAME, rowKey.getFlowName());
+    assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
+    assertEquals(APPLICATION_ID, rowKey.getAppId());
+    assertEquals(entity.getType(), rowKey.getEntityType());
+    assertEquals(entity.getIdPrefix(), rowKey.getEntityIdPrefix().longValue());
+    assertEquals(entity.getId(), rowKey.getEntityId());
+
+  }
+
+  @Test(timeout = 10000)
+  public void testFlowActivityRowKey() {
+    Long ts = 1459900830000L;
+    Long dayTimestamp = HBaseTimelineStorageUtils.getTopOfTheDayTimestamp(ts);
+    String rowKeyAsString = new FlowActivityRowKey(CLUSTER, ts, USER, FLOW_NAME)
+        .getRowKeyAsString();
+    FlowActivityRowKey rowKey =
+        FlowActivityRowKey.parseRowKeyFromString(rowKeyAsString);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(dayTimestamp, rowKey.getDayTimestamp());
+    assertEquals(USER, rowKey.getUserId());
+    assertEquals(FLOW_NAME, rowKey.getFlowName());
+  }
+
+  @Test(timeout = 10000)
+  public void testFlowRunRowKey() {
+    String rowKeyAsString =
+        new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID)
+            .getRowKeyAsString();
+    FlowRunRowKey rowKey = FlowRunRowKey.parseRowKeyFromString(rowKeyAsString);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(USER, rowKey.getUserId());
+    assertEquals(FLOW_NAME, rowKey.getFlowName());
+    assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.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/TimelineEntityFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
index 79a83c6..dc3e3ec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
@@ -99,19 +99,10 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyVa
  * filter list, event filters can be evaluated with logical AND/OR and we can
  * create a hierarchy of these {@link TimelineExistsFilter} objects. If null or
  * empty, the filter is not applied.</li>
- * <li><b>fromIdPrefix</b> - If specified, retrieve entities with an id prefix
- * greater than or equal to the specified fromIdPrefix. If fromIdPrefix is same
- * for all entities of a given entity type, then the user must provide fromId as
- * a filter to denote the start entity from which further entities will be
- * fetched. fromIdPrefix is mandatory even in the case the entity id prefix is
- * not used and should be set to 0.</li>
- * <li><b>fromId</b> - If specified along with fromIdPrefix, retrieve entities
- * with an id prefix greater than or equal to specified id prefix in
- * fromIdPrefix and entity id lexicographically greater than or equal to entity
- * id specified in fromId. Please note than fromIdPrefix is mandatory if fromId
- * is specified, otherwise, the filter will be ignored. It is recommended to
- * provide both fromIdPrefix and fromId filters for more accurate results as id
- * prefix may not be unique for an entity.</li>
+ * <li><b>fromId</b> - If specified, retrieve the next set of entities from the
+ * given fromId. The set of entities retrieved is inclusive of specified fromId.
+ * fromId should be taken from the value associated with FROM_ID info key in
+ * entity response which was sent earlier.</li>
  * </ul>
  */
 @Private
@@ -126,7 +117,6 @@ public class TimelineEntityFilters {
   private TimelineFilterList configFilters;
   private TimelineFilterList metricFilters;
   private TimelineFilterList eventFilters;
-  private Long fromIdPrefix;
   private String fromId;
   private static final long DEFAULT_BEGIN_TIME = 0L;
   private static final long DEFAULT_END_TIME = Long.MAX_VALUE;
@@ -146,11 +136,10 @@ public class TimelineEntityFilters {
       TimelineFilterList entityInfoFilters,
       TimelineFilterList entityConfigFilters,
       TimelineFilterList entityMetricFilters,
-      TimelineFilterList entityEventFilters, Long fromidprefix, String fromid) {
+      TimelineFilterList entityEventFilters, String fromid) {
     this(entityLimit, timeBegin, timeEnd, entityRelatesTo, entityIsRelatedTo,
         entityInfoFilters, entityConfigFilters, entityMetricFilters,
         entityEventFilters);
-    this.fromIdPrefix = fromidprefix;
     this.fromId = fromid;
   }
 
@@ -276,12 +265,4 @@ public class TimelineEntityFilters {
   public void setFromId(String fromId) {
     this.fromId = fromId;
   }
-
-  public Long getFromIdPrefix() {
-    return fromIdPrefix;
-  }
-
-  public void setFromIdPrefix(Long fromIdPrefix) {
-    this.fromIdPrefix = fromIdPrefix;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/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 1f82d91..490a3cf 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
@@ -264,20 +264,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
-   * @param fromIdPrefix If specified, retrieve entities with an id prefix
-   *     greater than or equal to the specified fromIdPrefix. If fromIdPrefix
-   *     is same for all entities of a given entity type, then the user must
-   *     provide fromId as a filter to denote the start entity from which
-   *     further entities will be fetched. fromIdPrefix is mandatory even
-   *     in the case the entity id prefix is not used and should be set to 0.
-   * @param fromId If specified along with fromIdPrefix, retrieve entities with
-   *     an id prefix greater than or equal to specified id prefix in
-   *     fromIdPrefix and entity id lexicographically greater than or equal
-   *     to entity id specified in fromId. Please note than fromIdPrefix is
-   *     mandatory if fromId is specified, otherwise, the filter will be
-   *     ignored. It is recommended to provide both fromIdPrefix and fromId
-   *     filters for more accurate results as id prefix may not be unique
-   *     for an entity.
+   * @param fromId If specified, retrieve the next set of entities from the
+   *     given fromId. The set of entities retrieved is inclusive of specified
+   *     fromId. fromId should be taken from the value associated with FROM_ID
+   *     info key in entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances of the given entity type
@@ -309,7 +299,6 @@ public class TimelineReaderWebServices {
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
       @QueryParam("metricslimit") String metricsLimit,
-      @QueryParam("fromidprefix") String fromIdPrefix,
       @QueryParam("fromid") String fromId) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -334,7 +323,7 @@ public class TimelineReaderWebServices {
           TimelineReaderWebServicesUtils.createTimelineEntityFilters(
           limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
               infofilters, conffilters, metricfilters, eventfilters,
-              fromIdPrefix, fromId),
+              fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
           confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
     } catch (Exception e) {
@@ -417,20 +406,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
-   * @param fromIdPrefix If specified, retrieve entities with an id prefix
-   *     greater than or equal to the specified fromIdPrefix. If fromIdPrefix
-   *     is same for all entities of a given entity type, then the user must
-   *     provide fromId as a filter to denote the start entity from which
-   *     further entities will be fetched. fromIdPrefix is mandatory even
-   *     in the case the entity id prefix is not used and should be set to 0.
-   * @param fromId If specified along with fromIdPrefix, retrieve entities with
-   *     an id prefix greater than or equal to specified id prefix in
-   *     fromIdPrefix and entity id lexicographically greater than or equal
-   *     to entity id specified in fromId. Please note than fromIdPrefix is
-   *     mandatory if fromId is specified, otherwise, the filter will be
-   *     ignored. It is recommended to provide both fromIdPrefix and fromId
-   *     filters for more accurate results as id prefix may not be unique
-   *     for an entity.
+   * @param fromId If specified, retrieve the next set of entities from the
+   *     given fromId. The set of entities retrieved is inclusive of specified
+   *     fromId. fromId should be taken from the value associated with FROM_ID
+   *     info key in entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances of the given entity type
@@ -467,12 +446,11 @@ public class TimelineReaderWebServices {
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
       @QueryParam("metricslimit") String metricsLimit,
-      @QueryParam("fromidprefix") String fromIdPrefix,
       @QueryParam("fromid") String fromId) {
     return getEntities(req, res, null, appId, entityType, userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, fromIdPrefix,
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
         fromId);
   }
 
@@ -544,20 +522,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
-   * @param fromIdPrefix If specified, retrieve entities with an id prefix
-   *     greater than or equal to the specified fromIdPrefix. If fromIdPrefix
-   *     is same for all entities of a given entity type, then the user must
-   *     provide fromId as a filter to denote the start entity from which
-   *     further entities will be fetched. fromIdPrefix is mandatory even
-   *     in the case the entity id prefix is not used and should be set to 0.
-   * @param fromId If specified along with fromIdPrefix, retrieve entities with
-   *     an id prefix greater than or equal to specified id prefix in
-   *     fromIdPrefix and entity id lexicographically greater than or equal
-   *     to entity id specified in fromId. Please note than fromIdPrefix is
-   *     mandatory if fromId is specified, otherwise, the filter will be
-   *     ignored. It is recommended to provide both fromIdPrefix and fromId
-   *     filters for more accurate results as id prefix may not be unique
-   *     for an entity.
+   * @param fromId If specified, retrieve the next set of entities from the
+   *     given fromId. The set of entities retrieved is inclusive of specified
+   *     fromId. fromId should be taken from the value associated with FROM_ID
+   *     info key in entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances of the given entity type
@@ -595,7 +563,6 @@ public class TimelineReaderWebServices {
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
       @QueryParam("metricslimit") String metricsLimit,
-      @QueryParam("fromidprefix") String fromIdPrefix,
       @QueryParam("fromid") String fromId) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -616,7 +583,7 @@ public class TimelineReaderWebServices {
           TimelineReaderWebServicesUtils.createTimelineEntityFilters(
           limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
               infofilters, conffilters, metricfilters, eventfilters,
-              fromIdPrefix, fromId),
+              fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
           confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
     } catch (Exception e) {
@@ -1097,9 +1064,10 @@ public class TimelineReaderWebServices {
    *     METRICS makes sense for flow runs hence only ALL or METRICS are
    *     supported as fields for fetching flow runs. Other fields will lead to
    *     HTTP 400 (Bad Request) response. (Optional query param).
-   * @param fromId Defines the flow run id. If specified, retrieve the next
-   *     set of flow runs from the given id. The set of flow runs retrieved
-   *     is inclusive of specified fromId.
+   * @param fromId If specified, retrieve the next set of flow run entities
+   *     from the given fromId. The set of entities retrieved is inclusive of
+   *     specified fromId. fromId should be taken from the value associated
+   *     with FROM_ID info key in entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     set of <cite>FlowRunEntity</cite> instances for the given flow are
@@ -1144,7 +1112,7 @@ public class TimelineReaderWebServices {
       entities = timelineReaderManager.getEntities(context,
           TimelineReaderWebServicesUtils.createTimelineEntityFilters(
           limit, createdTimeStart, createdTimeEnd, null, null, null,
-              null, null, null, null, fromId),
+              null, null, null, fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
           null, metricsToRetrieve, fields, null));
     } catch (Exception e) {
@@ -1187,9 +1155,10 @@ public class TimelineReaderWebServices {
    *     METRICS makes sense for flow runs hence only ALL or METRICS are
    *     supported as fields for fetching flow runs. Other fields will lead to
    *     HTTP 400 (Bad Request) response. (Optional query param).
-   * @param fromId Defines the flow run id. If specified, retrieve the next
-   *     set of flow runs from the given id. The set of flow runs retrieved
-   *     is inclusive of specified fromId.
+   * @param fromId If specified, retrieve the next set of flow run entities
+   *     from the given fromId. The set of entities retrieved is inclusive of
+   *     specified fromId. fromId should be taken from the value associated
+   *     with FROM_ID info key in entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     set of <cite>FlowRunEntity</cite> instances for the given flow are
@@ -1246,9 +1215,10 @@ public class TimelineReaderWebServices {
    *     METRICS makes sense for flow runs hence only ALL or METRICS are
    *     supported as fields for fetching flow runs. Other fields will lead to
    *     HTTP 400 (Bad Request) response. (Optional query param).
-   * @param fromId Defines the flow run id. If specified, retrieve the next
-   *     set of flow runs from the given id. The set of flow runs retrieved
-   *     is inclusive of specified fromId.
+   * @param fromId If specified, retrieve the next set of flow run entities
+   *     from the given fromId. The set of entities retrieved is inclusive of
+   *     specified fromId. fromId should be taken from the value associated
+   *     with FROM_ID info key in entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     set of <cite>FlowRunEntity</cite> instances for the given flow are
@@ -1292,7 +1262,7 @@ public class TimelineReaderWebServices {
               TimelineEntityType.YARN_FLOW_RUN.toString(), null, null),
           TimelineReaderWebServicesUtils.createTimelineEntityFilters(
           limit, createdTimeStart, createdTimeEnd, null, null, null,
-              null, null, null, null, fromId),
+              null, null, null, fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
           null, metricsToRetrieve, fields, null));
     } catch (Exception e) {
@@ -1422,7 +1392,7 @@ public class TimelineReaderWebServices {
       DateRange range = parseDateRange(dateRange);
       TimelineEntityFilters entityFilters =
           TimelineReaderWebServicesUtils.createTimelineEntityFilters(
-              limit, null, null, null, null, null, null, null, null, null,
+              limit, null, null, null, null, null, null, null, null,
               fromId);
       entityFilters.setCreatedTimeBegin(range.dateStart);
       entityFilters.setCreatedTimeEnd(range.dateEnd);
@@ -1743,9 +1713,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
-   * @param fromId Defines the application id. If specified, retrieve the next
-   *     set of applications from the given id. The set of applications
-   *     retrieved is inclusive of specified fromId.
+   * @param fromId If specified, retrieve the next set of applications
+   *     from the given fromId. The set of applications retrieved is inclusive
+   *     of specified fromId. fromId should be taken from the value associated
+   *     with FROM_ID info key in entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -1798,7 +1769,7 @@ public class TimelineReaderWebServices {
       entities = timelineReaderManager.getEntities(context,
           TimelineReaderWebServicesUtils.createTimelineEntityFilters(
           limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
-              infofilters, conffilters, metricfilters, eventfilters, null,
+              infofilters, conffilters, metricfilters, eventfilters,
               fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
           confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
@@ -1875,9 +1846,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
-   * @param fromId Defines the application id. If specified, retrieve the next
-   *     set of applications from the given id. The set of applications
-   *     retrieved is inclusive of specified fromId.
+   * @param fromId If specified, retrieve the next set of applications
+   *     from the given fromId. The set of applications retrieved is inclusive
+   *     of specified fromId. fromId should be taken from the value associated
+   *     with FROM_ID info key in entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -1915,7 +1887,7 @@ public class TimelineReaderWebServices {
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, null, fromId);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, fromId);
   }
 
   /**
@@ -1979,9 +1951,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
-   * @param fromId Defines the application id. If specified, retrieve the next
-   *     set of applications from the given id. The set of applications
-   *     retrieved is inclusive of specified fromId.
+   * @param fromId If specified, retrieve the next set of applications
+   *     from the given fromId. The set of applications retrieved is inclusive
+   *     of specified fromId. fromId should be taken from the value associated
+   *     with FROM_ID info key in entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -2021,7 +1994,7 @@ public class TimelineReaderWebServices {
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, null, fromId);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, fromId);
   }
 
   /**
@@ -2082,9 +2055,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
-   * @param fromId Defines the application id. If specified, retrieve the next
-   *     set of applications from the given id. The set of applications
-   *     retrieved is inclusive of specified fromId.
+   * @param fromId If specified, retrieve the next set of applications
+   *     from the given fromId. The set of applications retrieved is inclusive
+   *     of specified fromId. fromId should be taken from the value associated
+   *     with FROM_ID info key in entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -2121,7 +2095,7 @@ public class TimelineReaderWebServices {
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         null, limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
         infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, null, fromId);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, fromId);
   }
 
   /**
@@ -2183,9 +2157,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
-   * @param fromId Defines the application id. If specified, retrieve the next
-   *     set of applications from the given id. The set of applications
-   *     retrieved is inclusive of specified fromId.
+   * @param fromId If specified, retrieve the next set of applications
+   *     from the given fromId. The set of applications retrieved is inclusive
+   *     of specified fromId. fromId should be taken from the value associated
+   *     with FROM_ID info key in entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -2223,7 +2198,7 @@ public class TimelineReaderWebServices {
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         null, limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
         infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, null, fromId);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, fromId);
   }
 
   /**
@@ -2294,21 +2269,11 @@ public class TimelineReaderWebServices {
    *          have to be retrieved, then metricsLimit will be considered as 1
    *          i.e. latest single value of metric(s) will be returned. (Optional
    *          query param).
-   * @param fromIdPrefix If specified, retrieve entities with an id prefix
-   *          greater than or equal to the specified fromIdPrefix. If
-   *          fromIdPrefix is same for all entities of a given entity type, then
-   *          the user must provide fromId as a filter to denote the start
-   *          entity from which further entities will be fetched. fromIdPrefix
-   *          is mandatory even in the case the entity id prefix is not used and
-   *          should be set to 0.
-   * @param fromId If specified along with fromIdPrefix, retrieve entities with
-   *          an id prefix greater than or equal to specified id prefix in
-   *          fromIdPrefix and entity id lexicographically greater than or equal
-   *          to entity id specified in fromId. Please note than fromIdPrefix is
-   *          mandatory if fromId is specified, otherwise, the filter will be
-   *          ignored. It is recommended to provide both fromIdPrefix and fromId
-   *          filters for more accurate results as id prefix may not be unique
-   *          for an entity.
+   * @param fromId If specified, retrieve the next set of application-attempt
+   *         entities from the given fromId. The set of application-attempt
+   *         entities retrieved is inclusive of specified fromId. fromId should
+   *         be taken from the value associated with FROM_ID info key in
+   *         entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *         set of <cite>TimelineEntity</cite> instances of the app-attempt
@@ -2342,13 +2307,12 @@ public class TimelineReaderWebServices {
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
       @QueryParam("metricslimit") String metricsLimit,
-      @QueryParam("fromidprefix") String fromIdPrefix,
       @QueryParam("fromid") String fromId) {
 
     return getAppAttempts(req, res, null, appId, userId, flowName, flowRunId,
         limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
         infofilters, conffilters, metricfilters, eventfilters, confsToRetrieve,
-        metricsToRetrieve, fields, metricsLimit, fromIdPrefix, fromId);
+        metricsToRetrieve, fields, metricsLimit, fromId);
   }
 
   /**
@@ -2420,21 +2384,11 @@ public class TimelineReaderWebServices {
    *          have to be retrieved, then metricsLimit will be considered as 1
    *          i.e. latest single value of metric(s) will be returned. (Optional
    *          query param).
-   * @param fromIdPrefix If specified, retrieve entities with an id prefix
-   *          greater than or equal to the specified fromIdPrefix. If
-   *          fromIdPrefix is same for all entities of a given entity type, then
-   *          the user must provide fromId as a filter to denote the start
-   *          entity from which further entities will be fetched. fromIdPrefix
-   *          is mandatory even in the case the entity id prefix is not used and
-   *          should be set to 0.
-   * @param fromId If specified along with fromIdPrefix, retrieve entities with
-   *          an id prefix greater than or equal to specified id prefix in
-   *          fromIdPrefix and entity id lexicographically greater than or equal
-   *          to entity id specified in fromId. Please note than fromIdPrefix is
-   *          mandatory if fromId is specified, otherwise, the filter will be
-   *          ignored. It is recommended to provide both fromIdPrefix and fromId
-   *          filters for more accurate results as id prefix may not be unique
-   *          for an entity.
+   * @param fromId If specified, retrieve the next set of application-attempt
+   *         entities from the given fromId. The set of application-attempt
+   *         entities retrieved is inclusive of specified fromId. fromId should
+   *         be taken from the value associated with FROM_ID info key in
+   *         entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *         set of <cite>TimelineEntity</cite> instances of the app-attempts
@@ -2469,7 +2423,6 @@ public class TimelineReaderWebServices {
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
       @QueryParam("metricslimit") String metricsLimit,
-      @QueryParam("fromidprefix") String fromIdPrefix,
       @QueryParam("fromid") String fromId) {
 
     return getEntities(req, res, clusterId, appId,
@@ -2477,7 +2430,7 @@ public class TimelineReaderWebServices {
         flowName, flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
         confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
-        fromIdPrefix, fromId);
+        fromId);
   }
 
   /**
@@ -2699,21 +2652,11 @@ public class TimelineReaderWebServices {
    *          have to be retrieved, then metricsLimit will be considered as 1
    *          i.e. latest single value of metric(s) will be returned. (Optional
    *          query param).
-   * @param fromIdPrefix If specified, retrieve entities with an id prefix
-   *          greater than or equal to the specified fromIdPrefix. If
-   *          fromIdPrefix is same for all entities of a given entity type, then
-   *          the user must provide fromId as a filter to denote the start
-   *          entity from which further entities will be fetched. fromIdPrefix
-   *          is mandatory even in the case the entity id prefix is not used and
-   *          should be set to 0.
-   * @param fromId If specified along with fromIdPrefix, retrieve entities with
-   *          an id prefix greater than or equal to specified id prefix in
-   *          fromIdPrefix and entity id lexicographically greater than or equal
-   *          to entity id specified in fromId. Please note than fromIdPrefix is
-   *          mandatory if fromId is specified, otherwise, the filter will be
-   *          ignored. It is recommended to provide both fromIdPrefix and fromId
-   *          filters for more accurate results as id prefix may not be unique
-   *          for an entity.
+   * @param fromId If specified, retrieve the next set of container
+   *         entities from the given fromId. The set of container
+   *         entities retrieved is inclusive of specified fromId. fromId should
+   *         be taken from the value associated with FROM_ID info key in
+   *         entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *         set of <cite>TimelineEntity</cite> instances of the containers
@@ -2748,12 +2691,11 @@ public class TimelineReaderWebServices {
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
       @QueryParam("metricslimit") String metricsLimit,
-      @QueryParam("fromidprefix") String fromIdPrefix,
       @QueryParam("fromid") String fromId) {
     return getContainers(req, res, null, appId, appattemptId, userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, fromIdPrefix,
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
         fromId);
   }
 
@@ -2828,21 +2770,11 @@ public class TimelineReaderWebServices {
    *          have to be retrieved, then metricsLimit will be considered as 1
    *          i.e. latest single value of metric(s) will be returned. (Optional
    *          query param).
-   * @param fromIdPrefix If specified, retrieve entities with an id prefix
-   *          greater than or equal to the specified fromIdPrefix. If
-   *          fromIdPrefix is same for all entities of a given entity type, then
-   *          the user must provide fromId as a filter to denote the start
-   *          entity from which further entities will be fetched. fromIdPrefix
-   *          is mandatory even in the case the entity id prefix is not used and
-   *          should be set to 0.
-   * @param fromId If specified along with fromIdPrefix, retrieve entities with
-   *          an id prefix greater than or equal to specified id prefix in
-   *          fromIdPrefix and entity id lexicographically greater than or equal
-   *          to entity id specified in fromId. Please note than fromIdPrefix is
-   *          mandatory if fromId is specified, otherwise, the filter will be
-   *          ignored. It is recommended to provide both fromIdPrefix and fromId
-   *          filters for more accurate results as id prefix may not be unique
-   *          for an entity.
+   * @param fromId If specified, retrieve the next set of container
+   *         entities from the given fromId. The set of container
+   *         entities retrieved is inclusive of specified fromId. fromId should
+   *         be taken from the value associated with FROM_ID info key in
+   *         entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *         set of <cite>TimelineEntity</cite> instances of the containers
@@ -2879,7 +2811,6 @@ public class TimelineReaderWebServices {
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
       @QueryParam("metricslimit") String metricsLimit,
-      @QueryParam("fromidprefix") String fromIdPrefix,
       @QueryParam("fromid") String fromId) {
 
     String entityType = TimelineEntityType.YARN_CONTAINER.toString();
@@ -2898,7 +2829,7 @@ public class TimelineReaderWebServices {
     return getEntities(req, res, clusterId, appId, entityType, userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilter, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, fromIdPrefix,
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
         fromId);
   }
 


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


[25/50] [abbrv] hadoop git commit: YARN-6455. Enhance the timelinewriter.flush() race condition fix (Haibo Chen via Varun Saxena)

Posted by va...@apache.org.
YARN-6455. Enhance the timelinewriter.flush() race condition fix (Haibo Chen via Varun Saxena)

(cherry picked from commit 793bbf216dc6d44ea57c778037bd461513fa6078)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 1a771bdec9ea6d46b4e9bbe2e38c8468d48232ae
Parents: 3f3e926
Author: Varun Saxena <va...@apache.org>
Authored: Thu Apr 27 15:01:58 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Thu Apr 27 15:17:46 2017 +0530

----------------------------------------------------------------------
 .../timelineservice/collector/AppLevelTimelineCollector.java | 8 ++++----
 .../server/timelineservice/collector/TimelineCollector.java  | 4 ----
 2 files changed, 4 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a771bde/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.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/collector/AppLevelTimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
index e62a436..0b05309 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
@@ -59,6 +59,7 @@ public class AppLevelTimelineCollector extends TimelineCollector {
   private final TimelineCollectorContext context;
   private ScheduledThreadPoolExecutor appAggregationExecutor;
   private AppLevelAggregator appAggregator;
+  private UserGroupInformation currentUser;
 
   public AppLevelTimelineCollector(ApplicationId appId) {
     super(AppLevelTimelineCollector.class.getName() + " - " + appId.toString());
@@ -82,7 +83,8 @@ public class AppLevelTimelineCollector extends TimelineCollector {
     // Set the default values, which will be updated with an RPC call to get the
     // context info from NM.
     // Current user usually is not the app user, but keep this field non-null
-    context.setUserId(UserGroupInformation.getCurrentUser().getShortUserName());
+    currentUser = UserGroupInformation.getCurrentUser();
+    context.setUserId(currentUser.getShortUserName());
     context.setAppId(appId.toString());
     super.serviceInit(conf);
   }
@@ -149,9 +151,7 @@ public class AppLevelTimelineCollector extends TimelineCollector {
             TimelineEntityType.YARN_APPLICATION.toString());
         TimelineEntities entities = new TimelineEntities();
         entities.addEntity(resultEntity);
-        getWriter().write(currContext.getClusterId(), currContext.getUserId(),
-            currContext.getFlowName(), currContext.getFlowVersion(),
-            currContext.getFlowRunId(), currContext.getAppId(), entities);
+        putEntitiesAsync(entities, currentUser);
       } catch (Exception e) {
         LOG.error("Error aggregating timeline metrics", e);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a771bde/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.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/collector/TimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
index 4c9e9f8..c94c505 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
@@ -85,10 +85,6 @@ public abstract class TimelineCollector extends CompositeService {
     this.writer = w;
   }
 
-  protected TimelineWriter getWriter() {
-    return writer;
-  }
-
   protected Map<String, AggregationStatusTable> getAggregationGroups() {
     return aggregationGroups;
   }


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


[14/50] [abbrv] hadoop git commit: YARN-6376. Exceptions caused by synchronous putEntities requests can be swallowed (Haibo Chen via Varun Saxena)

Posted by va...@apache.org.
YARN-6376. Exceptions caused by synchronous putEntities requests can be swallowed (Haibo Chen via Varun Saxena)

(cherry picked from commit b58777a9c9a5b6f2e4bcfd2b3bede33f25f80dec)


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

Branch: refs/heads/YARN-5355_branch2
Commit: fd2c8d2c6f53b335ce5e2ee5b160165119779a52
Parents: 739bf97
Author: Varun Saxena <va...@apache.org>
Authored: Fri Mar 31 02:17:20 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:45 2017 +0530

----------------------------------------------------------------------
 .../timelineservice/collector/TimelineCollector.java      | 10 ++++++++--
 .../collector/TimelineCollectorManager.java               |  7 ++++++-
 2 files changed, 14 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd2c8d2c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.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/collector/TimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
index 353066b..4c9e9f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
@@ -137,8 +137,14 @@ public abstract class TimelineCollector extends CompositeService {
           + callerUgi + ")");
     }
 
-    TimelineWriteResponse response = writeTimelineEntities(entities);
-    flushBufferedTimelineEntities();
+    TimelineWriteResponse response;
+    // synchronize on the writer object so that no other threads can
+    // flush the writer buffer concurrently and swallow any exception
+    // caused by the timeline enitites that are being put here.
+    synchronized (writer) {
+      response = writeTimelineEntities(entities);
+      flushBufferedTimelineEntities();
+    }
 
     return response;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fd2c8d2c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.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/collector/TimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
index 19896e8..8ef9b43 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
@@ -259,7 +259,12 @@ public class TimelineCollectorManager extends AbstractService {
 
     public void run() {
       try {
-        writer.flush();
+        // synchronize on the writer object to avoid flushing timeline
+        // entities placed on the buffer by synchronous putEntities
+        // requests.
+        synchronized (writer) {
+          writer.flush();
+        }
       } catch (Throwable th) {
         // we need to handle all exceptions or subsequent execution may be
         // suppressed


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


[19/50] [abbrv] hadoop git commit: YARN-6146. Add Builder methods for TimelineEntityFilters (Haibo Chen via Varun Saxena)

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/e59486a3/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 4a9e53e..4d3e769 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
@@ -76,12 +76,44 @@ final class TimelineReaderWebServicesUtils {
       String isRelatedTo, String infofilters, String conffilters,
       String metricfilters, String eventfilters,
       String fromid) throws TimelineParseException {
-    return new TimelineEntityFilters(parseLongStr(limit),
-        parseLongStr(createdTimeStart), parseLongStr(createdTimeEnd),
-        parseRelationFilters(relatesTo), parseRelationFilters(isRelatedTo),
-        parseKVFilters(infofilters, false), parseKVFilters(conffilters, true),
-        parseMetricFilters(metricfilters), parseEventFilters(eventfilters),
-        parseStr(fromid));
+    return createTimelineEntityFilters(
+        limit, parseLongStr(createdTimeStart),
+        parseLongStr(createdTimeEnd),
+        relatesTo, isRelatedTo, infofilters,
+        conffilters, metricfilters, eventfilters, fromid);
+  }
+
+  /**
+   * Parse the passed filters represented as strings and convert them into a
+   * {@link TimelineEntityFilters} object.
+   * @param limit Limit to number of entities to return.
+   * @param createdTimeStart Created time start for the entities to return.
+   * @param createdTimeEnd Created time end for the entities to return.
+   * @param relatesTo Entities to return must match relatesTo.
+   * @param isRelatedTo Entities to return must match isRelatedTo.
+   * @param infofilters Entities to return must match these info filters.
+   * @param conffilters Entities to return must match these metric filters.
+   * @param metricfilters Entities to return must match these metric filters.
+   * @param eventfilters Entities to return must match these event filters.
+   * @return a {@link TimelineEntityFilters} object.
+   * @throws TimelineParseException if any problem occurs during parsing.
+   */
+  static TimelineEntityFilters createTimelineEntityFilters(String limit,
+      Long createdTimeStart, Long createdTimeEnd, String relatesTo,
+      String isRelatedTo, String infofilters, String conffilters,
+      String metricfilters, String eventfilters,
+      String fromid) throws TimelineParseException {
+    return new TimelineEntityFilters.Builder()
+        .entityLimit(parseLongStr(limit))
+        .createdTimeBegin(createdTimeStart)
+        .createTimeEnd(createdTimeEnd)
+        .relatesTo(parseRelationFilters(relatesTo))
+        .isRelatedTo(parseRelationFilters(isRelatedTo))
+        .infoFilters(parseKVFilters(infofilters, false))
+        .configFilters(parseKVFilters(conffilters, true))
+        .metricFilters(parseMetricFilters(metricfilters))
+        .eventFilters(parseEventFilters(eventfilters))
+        .fromId(parseStr(fromid)).build();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e59486a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.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/storage/TestFileSystemTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java
index 35af169..1bc66db 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java
@@ -398,7 +398,7 @@ public class TestFileSystemTimelineReaderImpl {
   public void testGetAllEntities() throws Exception {
     Set<TimelineEntity> result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
-        "app", null), new TimelineEntityFilters(),
+        "app", null), new TimelineEntityFilters.Builder().build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     // All 4 entities will be returned
     Assert.assertEquals(4, result.size());
@@ -409,8 +409,8 @@ public class TestFileSystemTimelineReaderImpl {
     Set<TimelineEntity> result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(2L, null, null, null, null, null, null,
-        null, null), new TimelineDataToRetrieve());
+        new TimelineEntityFilters.Builder().entityLimit(2L).build(),
+        new TimelineDataToRetrieve());
     Assert.assertEquals(2, result.size());
     // Needs to be rewritten once hashcode and equals for
     // TimelineEntity is implemented
@@ -424,8 +424,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(3L, null, null, null, null, null, null,
-        null, null), new TimelineDataToRetrieve());
+        new TimelineEntityFilters.Builder().entityLimit(3L).build(),
+        new TimelineDataToRetrieve());
     // Even though 2 entities out of 4 have same created time, one entity
     // is left out due to limit
     Assert.assertEquals(3, result.size());
@@ -437,8 +437,8 @@ public class TestFileSystemTimelineReaderImpl {
     Set<TimelineEntity> result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, 1425016502030L, 1425016502060L, null,
-        null, null, null, null, null),
+        new TimelineEntityFilters.Builder().createdTimeBegin(1425016502030L)
+            .createTimeEnd(1425016502060L).build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(1, result.size());
     // Only one entity with ID id_4 should be returned.
@@ -452,9 +452,9 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
             "app", null),
-            new TimelineEntityFilters(null, null, 1425016502010L, null, null,
-            null, null, null, null),
-            new TimelineDataToRetrieve());
+        new TimelineEntityFilters.Builder().createTimeEnd(1425016502010L)
+            .build(),
+        new TimelineDataToRetrieve());
     Assert.assertEquals(3, result.size());
     for (TimelineEntity entity : result) {
       if (entity.getId().equals("id_4")) {
@@ -466,8 +466,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, 1425016502010L, null, null, null,
-        null, null, null, null),
+        new TimelineEntityFilters.Builder().createdTimeBegin(1425016502010L)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(1, result.size());
     for (TimelineEntity entity : result) {
@@ -486,8 +486,7 @@ public class TestFileSystemTimelineReaderImpl {
     Set<TimelineEntity> result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList).build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(1, result.size());
     // Only one entity with ID id_3 should be returned.
@@ -506,8 +505,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(1, result.size());
     for (TimelineEntity entity : result) {
@@ -525,8 +524,7 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, eventFilters),
+        new TimelineEntityFilters.Builder().eventFilters(eventFilters).build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(1, result.size());
     for (TimelineEntity entity : result) {
@@ -542,8 +540,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(2, result.size());
     // Two entities with IDs' id_1 and id_2 should be returned.
@@ -569,8 +567,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList1, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList1)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(2, result.size());
     for (TimelineEntity entity : result) {
@@ -592,8 +590,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList2, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList2)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(2, result.size());
     for (TimelineEntity entity : result) {
@@ -610,8 +608,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList3, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList3)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(1, result.size());
     for(TimelineEntity entity : result) {
@@ -628,8 +626,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList4, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList4)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(0, result.size());
 
@@ -641,8 +639,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList5, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList5)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(1, result.size());
     for (TimelineEntity entity : result) {
@@ -665,8 +663,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList1, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(2, result.size());
     // Two entities with IDs' id_2 and id_3 should be returned.
@@ -684,8 +682,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList2, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList2)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(1, result.size());
     for (TimelineEntity entity : result) {
@@ -702,8 +700,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList3, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList3)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(0, result.size());
 
@@ -715,8 +713,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList4, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList4)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(2, result.size());
     for (TimelineEntity entity : result) {
@@ -731,8 +729,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList5, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList5)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(2, result.size());
     for (TimelineEntity entity : result) {
@@ -749,8 +747,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList1,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList1)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(0, result.size());
 
@@ -762,8 +760,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList2,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList2)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(2, result.size());
     for (TimelineEntity entity : result) {
@@ -780,8 +778,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList3,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList3)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(0, result.size());
 
@@ -793,8 +791,8 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList4,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList4)
+            .build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(1, result.size());
     for (TimelineEntity entity : result) {
@@ -815,8 +813,7 @@ public class TestFileSystemTimelineReaderImpl {
     Set<TimelineEntity> result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, relatesTo, null, null,
-        null, null, null),
+        new TimelineEntityFilters.Builder().relatesTo(relatesTo).build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(1, result.size());
     // Only one entity with ID id_1 should be returned.
@@ -835,8 +832,7 @@ public class TestFileSystemTimelineReaderImpl {
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, isRelatedTo, null,
-        null, null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(isRelatedTo).build(),
         new TimelineDataToRetrieve());
     Assert.assertEquals(2, result.size());
     // Two entities with IDs' id_1 and id_3 should be returned.


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


[46/50] [abbrv] hadoop git commit: YARN-5647. [ATSv2 Security] Collector side changes for loading auth filters and principals. Contributed by Varun Saxena

Posted by va...@apache.org.
YARN-5647. [ATSv2 Security] Collector side changes for loading auth filters and principals. Contributed by Varun Saxena

(cherry picked from commit 79dae624793164cd4692c86992a511310145858a)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 9496748b43a17aef48d090ab31e5c4c4df806bae
Parents: 69d2c1e
Author: Jian He <ji...@apache.org>
Authored: Wed Jun 7 13:45:34 2017 -0700
Committer: Varun Saxena <va...@apache.org>
Committed: Tue Aug 1 03:04:26 2017 +0530

----------------------------------------------------------------------
 .../ApplicationHistoryServer.java               |  79 ++---
 .../security/TimelineAuthenticationFilter.java  |  49 ---
 ...TimelineAuthenticationFilterInitializer.java | 129 -------
 ...lineDelegationTokenSecretManagerService.java | 240 --------------
 ...neV1DelegationTokenSecretManagerService.java | 225 +++++++++++++
 .../TestTimelineAuthenticationFilter.java       | 323 ------------------
 .../TestTimelineAuthenticationFilterForV1.java  | 332 +++++++++++++++++++
 ...TimelineAuthenticationFilterInitializer.java |  76 -----
 .../security/TimelineAuthenticationFilter.java  |  55 +++
 ...TimelineAuthenticationFilterInitializer.java | 129 +++++++
 ...elineDelgationTokenSecretManagerService.java |  83 +++++
 .../server/timeline/security/package-info.java  |  26 ++
 .../util/timeline/TimelineServerUtils.java      |  92 +++++
 .../yarn/server/util/timeline/package-info.java |  25 ++
 ...TimelineAuthenticationFilterInitializer.java |  76 +++++
 .../collector/NodeTimelineCollectorManager.java |  66 +++-
 .../PerNodeTimelineCollectorsAuxService.java    |   5 +-
 .../collector/TimelineCollectorManager.java     |   6 +-
 ...neV2DelegationTokenSecretManagerService.java |  78 +++++
 .../timelineservice/security/package-info.java  |  25 ++
 20 files changed, 1227 insertions(+), 892 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
index fd63787..2c6e204 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
@@ -20,7 +20,8 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.ArrayList;
+import java.util.LinkedHashSet;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -29,7 +30,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
-import org.apache.hadoop.security.AuthenticationFilterInitializer;
 import org.apache.hadoop.security.HttpCrossOriginFilterInitializer;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.service.CompositeService;
@@ -49,10 +49,9 @@ import org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore;
 import org.apache.hadoop.yarn.server.timeline.TimelineDataManager;
 import org.apache.hadoop.yarn.server.timeline.TimelineStore;
 import org.apache.hadoop.yarn.server.timeline.security.TimelineACLsManager;
-import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilter;
-import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer;
-import org.apache.hadoop.yarn.server.timeline.security.TimelineDelegationTokenSecretManagerService;
+import org.apache.hadoop.yarn.server.timeline.security.TimelineV1DelegationTokenSecretManagerService;
 import org.apache.hadoop.yarn.server.timeline.webapp.CrossOriginFilterInitializer;
+import org.apache.hadoop.yarn.server.util.timeline.TimelineServerUtils;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebApps;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
@@ -75,7 +74,7 @@ public class ApplicationHistoryServer extends CompositeService {
   private ApplicationACLsManager aclsManager;
   private ApplicationHistoryManager historyManager;
   private TimelineStore timelineStore;
-  private TimelineDelegationTokenSecretManagerService secretManagerService;
+  private TimelineV1DelegationTokenSecretManagerService secretManagerService;
   private TimelineDataManager timelineDataManager;
   private WebApp webApp;
   private JvmPauseMonitor pauseMonitor;
@@ -223,9 +222,9 @@ public class ApplicationHistoryServer extends CompositeService {
         TimelineStore.class), conf);
   }
 
-  private TimelineDelegationTokenSecretManagerService
+  private TimelineV1DelegationTokenSecretManagerService
       createTimelineDelegationTokenSecretManagerService(Configuration conf) {
-    return new TimelineDelegationTokenSecretManagerService();
+    return new TimelineV1DelegationTokenSecretManagerService();
   }
 
   private TimelineDataManager createTimelineDataManager(Configuration conf) {
@@ -237,63 +236,33 @@ public class ApplicationHistoryServer extends CompositeService {
   @SuppressWarnings("unchecked")
   private void startWebApp() {
     Configuration conf = getConfig();
-    TimelineAuthenticationFilter.setTimelineDelegationTokenSecretManager(
-        secretManagerService.getTimelineDelegationTokenSecretManager());
     // Always load pseudo authentication filter to parse "user.name" in an URL
     // to identify a HTTP request's user in insecure mode.
     // When Kerberos authentication type is set (i.e., secure mode is turned on),
     // the customized filter will be loaded by the timeline server to do Kerberos
     // + DT authentication.
-    String initializers = conf.get("hadoop.http.filter.initializers");
-    boolean modifiedInitializers = false;
-
-    initializers =
-        initializers == null || initializers.length() == 0 ? "" : initializers;
-
+    String initializers = conf.get("hadoop.http.filter.initializers", "");
+    Set<String> defaultInitializers = new LinkedHashSet<String>();
+    // Add CORS filter
     if (!initializers.contains(CrossOriginFilterInitializer.class.getName())) {
-      if(conf.getBoolean(YarnConfiguration
-          .TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED, YarnConfiguration
-              .TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT)) {
-        if (initializers.contains(HttpCrossOriginFilterInitializer.class.getName())) {
-          initializers =
-            initializers.replaceAll(HttpCrossOriginFilterInitializer.class.getName(),
+      if(conf.getBoolean(YarnConfiguration.
+          TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED,
+          YarnConfiguration.
+          TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT)) {
+        if (initializers.contains(
+            HttpCrossOriginFilterInitializer.class.getName())) {
+          initializers = initializers.replaceAll(
+              HttpCrossOriginFilterInitializer.class.getName(),
               CrossOriginFilterInitializer.class.getName());
+        } else {
+          defaultInitializers.add(CrossOriginFilterInitializer.class.getName());
         }
-        else {
-          if (initializers.length() != 0) {
-            initializers += ",";
-          }
-          initializers += CrossOriginFilterInitializer.class.getName();
-        }
-        modifiedInitializers = true;
-      }
-    }
-
-    if (!initializers.contains(TimelineAuthenticationFilterInitializer.class
-      .getName())) {
-      if (initializers.length() != 0) {
-        initializers += ",";
       }
-      initializers += TimelineAuthenticationFilterInitializer.class.getName();
-      modifiedInitializers = true;
-    }
-
-    String[] parts = initializers.split(",");
-    ArrayList<String> target = new ArrayList<String>();
-    for (String filterInitializer : parts) {
-      filterInitializer = filterInitializer.trim();
-      if (filterInitializer.equals(AuthenticationFilterInitializer.class
-        .getName())) {
-        modifiedInitializers = true;
-        continue;
-      }
-      target.add(filterInitializer);
-    }
-    String actualInitializers =
-        org.apache.commons.lang.StringUtils.join(target, ",");
-    if (modifiedInitializers) {
-      conf.set("hadoop.http.filter.initializers", actualInitializers);
     }
+    TimelineServerUtils.addTimelineAuthFilter(
+        initializers, defaultInitializers, secretManagerService);
+    TimelineServerUtils.setTimelineFilters(
+        conf, initializers, defaultInitializers);
     String bindAddress = WebAppUtils.getWebAppBindURL(conf,
                           YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
                           WebAppUtils.getAHSWebAppURLWithoutScheme(conf));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java
deleted file mode 100644
index ad8dc2c..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * 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.timeline.security;
-
-import javax.servlet.FilterConfig;
-import javax.servlet.ServletException;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter;
-import org.apache.hadoop.yarn.server.timeline.security.TimelineDelegationTokenSecretManagerService.TimelineDelegationTokenSecretManager;
-
-@Private
-@Unstable
-public class TimelineAuthenticationFilter
-    extends DelegationTokenAuthenticationFilter {
-
-  private static TimelineDelegationTokenSecretManager secretManager;
-
-  @Override
-  public void init(FilterConfig filterConfig) throws ServletException {
-    filterConfig.getServletContext().setAttribute(
-        DelegationTokenAuthenticationFilter.DELEGATION_TOKEN_SECRET_MANAGER_ATTR,
-        secretManager);
-    super.init(filterConfig);
-  }
-
-  public static void setTimelineDelegationTokenSecretManager(
-      TimelineDelegationTokenSecretManager secretManager) {
-    TimelineAuthenticationFilter.secretManager = secretManager;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
deleted file mode 100644
index 4e7c29a..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * 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.timeline.security;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.http.FilterContainer;
-import org.apache.hadoop.http.FilterInitializer;
-import org.apache.hadoop.http.HttpServer2;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
-import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
-import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
-import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler;
-import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticationHandler;
-import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticationHandler;
-import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Initializes {@link TimelineAuthenticationFilter} which provides support for
- * Kerberos HTTP SPNEGO authentication.
- * <p>
- * It enables Kerberos HTTP SPNEGO plus delegation token authentication for the
- * timeline server.
- * <p>
- * Refer to the {@code core-default.xml} file, after the comment 'HTTP
- * Authentication' for details on the configuration options. All related
- * configuration properties have {@code hadoop.http.authentication.} as prefix.
- */
-public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
-
-  /**
-   * The configuration prefix of timeline HTTP authentication
-   */
-  public static final String PREFIX = "yarn.timeline-service.http-authentication.";
-
-  @VisibleForTesting
-  Map<String, String> filterConfig;
-
-  /**
-   * Initializes {@link TimelineAuthenticationFilter}
-   * <p>
-   * Propagates to {@link TimelineAuthenticationFilter} configuration all YARN
-   * configuration properties prefixed with {@value #PREFIX}
-   *
-   * @param container
-   *          The filter container
-   * @param conf
-   *          Configuration for run-time parameters
-   */
-  @Override
-  public void initFilter(FilterContainer container, Configuration conf) {
-    filterConfig = new HashMap<String, String>();
-
-    // setting the cookie path to root '/' so it is used for all resources.
-    filterConfig.put(TimelineAuthenticationFilter.COOKIE_PATH, "/");
-
-    for (Map.Entry<String, String> entry : conf) {
-      String name = entry.getKey();
-      if (name.startsWith(ProxyUsers.CONF_HADOOP_PROXYUSER)) {
-        String value = conf.get(name);
-        name = name.substring("hadoop.".length());
-        filterConfig.put(name, value);
-      }
-    }
-    for (Map.Entry<String, String> entry : conf) {
-      String name = entry.getKey();
-      if (name.startsWith(PREFIX)) {
-        // yarn.timeline-service.http-authentication.proxyuser will override
-        // hadoop.proxyuser
-        String value = conf.get(name);
-        name = name.substring(PREFIX.length());
-        filterConfig.put(name, value);
-      }
-    }
-
-    String authType = filterConfig.get(AuthenticationFilter.AUTH_TYPE);
-    if (authType.equals(PseudoAuthenticationHandler.TYPE)) {
-      filterConfig.put(AuthenticationFilter.AUTH_TYPE,
-          PseudoDelegationTokenAuthenticationHandler.class.getName());
-    } else if (authType.equals(KerberosAuthenticationHandler.TYPE)) {
-      filterConfig.put(AuthenticationFilter.AUTH_TYPE,
-          KerberosDelegationTokenAuthenticationHandler.class.getName());
-
-      // Resolve _HOST into bind address
-      String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
-      String principal =
-          filterConfig.get(KerberosAuthenticationHandler.PRINCIPAL);
-      if (principal != null) {
-        try {
-          principal = SecurityUtil.getServerPrincipal(principal, bindAddress);
-        } catch (IOException ex) {
-          throw new RuntimeException(
-              "Could not resolve Kerberos principal name: " + ex.toString(), ex);
-        }
-        filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL,
-            principal);
-      }
-    }
-
-    filterConfig.put(DelegationTokenAuthenticationHandler.TOKEN_KIND,
-        TimelineDelegationTokenIdentifier.KIND_NAME.toString());
-
-    container.addGlobalFilter("Timeline Authentication Filter",
-        TimelineAuthenticationFilter.class.getName(),
-        filterConfig);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
deleted file mode 100644
index 60a0348..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
+++ /dev/null
@@ -1,240 +0,0 @@
-/**
- * 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.timeline.security;
-
-import java.io.IOException;
-import java.util.Map.Entry;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
-import org.apache.hadoop.security.token.delegation.DelegationKey;
-import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
-import org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore;
-import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore;
-import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore.TimelineServiceState;
-
-/**
- * The service wrapper of {@link TimelineDelegationTokenSecretManager}
- */
-@Private
-@Unstable
-public class TimelineDelegationTokenSecretManagerService extends
-    AbstractService {
-
-  private TimelineDelegationTokenSecretManager secretManager = null;
-  private TimelineStateStore stateStore = null;
-
-  public TimelineDelegationTokenSecretManagerService() {
-    super(TimelineDelegationTokenSecretManagerService.class.getName());
-  }
-
-  @Override
-  protected void serviceInit(Configuration conf) throws Exception {
-    if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_RECOVERY_ENABLED,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_RECOVERY_ENABLED)) {
-      stateStore = createStateStore(conf);
-      stateStore.init(conf);
-    }
-
-    long secretKeyInterval =
-        conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL,
-            YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL);
-    long tokenMaxLifetime =
-        conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME,
-            YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME);
-    long tokenRenewInterval =
-        conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL,
-            YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL);
-    secretManager = new TimelineDelegationTokenSecretManager(secretKeyInterval,
-        tokenMaxLifetime, tokenRenewInterval, 3600000, stateStore);
-    super.init(conf);
-  }
-
-  @Override
-  protected void serviceStart() throws Exception {
-    if (stateStore != null) {
-      stateStore.start();
-      TimelineServiceState state = stateStore.loadState();
-      secretManager.recover(state);
-    }
-
-    secretManager.startThreads();
-    super.serviceStart();
-  }
-
-  @Override
-  protected void serviceStop() throws Exception {
-    if (stateStore != null) {
-      stateStore.stop();
-    }
-
-    secretManager.stopThreads();
-    super.stop();
-  }
-
-  protected TimelineStateStore createStateStore(
-      Configuration conf) {
-    return ReflectionUtils.newInstance(
-        conf.getClass(YarnConfiguration.TIMELINE_SERVICE_STATE_STORE_CLASS,
-            LeveldbTimelineStateStore.class,
-            TimelineStateStore.class), conf);
-  }
-
-  /**
-   * Ge the instance of {link #TimelineDelegationTokenSecretManager}
-   *
-   * @return the instance of {link #TimelineDelegationTokenSecretManager}
-   */
-  public TimelineDelegationTokenSecretManager
-  getTimelineDelegationTokenSecretManager() {
-    return secretManager;
-  }
-
-  @Private
-  @Unstable
-  public static class TimelineDelegationTokenSecretManager extends
-      AbstractDelegationTokenSecretManager<TimelineDelegationTokenIdentifier> {
-
-    public static final Log LOG =
-        LogFactory.getLog(TimelineDelegationTokenSecretManager.class);
-
-    private TimelineStateStore stateStore;
-
-    /**
-     * Create a timeline secret manager
-     * @param delegationKeyUpdateInterval the number of milliseconds for rolling
-     *        new secret keys.
-     * @param delegationTokenMaxLifetime the maximum lifetime of the delegation
-     *        tokens in milliseconds
-     * @param delegationTokenRenewInterval how often the tokens must be renewed
-     *        in milliseconds
-     * @param delegationTokenRemoverScanInterval how often the tokens are
-     *        scanned for expired tokens in milliseconds
-     * @param stateStore timeline service state store
-     */
-    public TimelineDelegationTokenSecretManager(
-        long delegationKeyUpdateInterval,
-        long delegationTokenMaxLifetime,
-        long delegationTokenRenewInterval,
-        long delegationTokenRemoverScanInterval,
-        TimelineStateStore stateStore) {
-      super(delegationKeyUpdateInterval, delegationTokenMaxLifetime,
-          delegationTokenRenewInterval, delegationTokenRemoverScanInterval);
-      this.stateStore = stateStore;
-    }
-
-    @Override
-    public TimelineDelegationTokenIdentifier createIdentifier() {
-      return new TimelineDelegationTokenIdentifier();
-    }
-
-    @Override
-    protected void storeNewMasterKey(DelegationKey key) throws IOException {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Storing master key " + key.getKeyId());
-      }
-      try {
-        if (stateStore != null) {
-          stateStore.storeTokenMasterKey(key);
-        }
-      } catch (IOException e) {
-        LOG.error("Unable to store master key " + key.getKeyId(), e);
-      }
-    }
-
-    @Override
-    protected void removeStoredMasterKey(DelegationKey key) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Removing master key " + key.getKeyId());
-      }
-      try {
-        if (stateStore != null) {
-          stateStore.removeTokenMasterKey(key);
-        }
-      } catch (IOException e) {
-        LOG.error("Unable to remove master key " + key.getKeyId(), e);
-      }
-    }
-
-    @Override
-    protected void storeNewToken(TimelineDelegationTokenIdentifier tokenId,
-        long renewDate) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Storing token " + tokenId.getSequenceNumber());
-      }
-      try {
-        if (stateStore != null) {
-          stateStore.storeToken(tokenId, renewDate);
-        }
-      } catch (IOException e) {
-        LOG.error("Unable to store token " + tokenId.getSequenceNumber(), e);
-      }
-    }
-
-    @Override
-    protected void removeStoredToken(TimelineDelegationTokenIdentifier tokenId)
-        throws IOException {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Storing token " + tokenId.getSequenceNumber());
-      }
-      try {
-        if (stateStore != null) {
-          stateStore.removeToken(tokenId);
-        }
-      } catch (IOException e) {
-        LOG.error("Unable to remove token " + tokenId.getSequenceNumber(), e);
-      }
-    }
-
-    @Override
-    protected void updateStoredToken(TimelineDelegationTokenIdentifier tokenId,
-        long renewDate) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Updating token " + tokenId.getSequenceNumber());
-      }
-      try {
-        if (stateStore != null) {
-          stateStore.updateToken(tokenId, renewDate);
-        }
-      } catch (IOException e) {
-        LOG.error("Unable to update token " + tokenId.getSequenceNumber(), e);
-      }
-    }
-
-    public void recover(TimelineServiceState state) throws IOException {
-      LOG.info("Recovering " + getClass().getSimpleName());
-      for (DelegationKey key : state.getTokenMasterKeyState()) {
-        addKey(key);
-      }
-      this.delegationTokenSequenceNumber = state.getLatestSequenceNumber();
-      for (Entry<TimelineDelegationTokenIdentifier, Long> entry :
-          state.getTokenState().entrySet()) {
-        addPersistedDelegationToken(entry.getKey(), entry.getValue());
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineV1DelegationTokenSecretManagerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineV1DelegationTokenSecretManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineV1DelegationTokenSecretManagerService.java
new file mode 100644
index 0000000..433be36
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineV1DelegationTokenSecretManagerService.java
@@ -0,0 +1,225 @@
+/**
+ * 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.timeline.security;
+
+import java.io.IOException;
+import java.util.Map.Entry;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.hadoop.security.token.delegation.DelegationKey;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore;
+import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore;
+import org.apache.hadoop.yarn.server.timeline.recovery.TimelineStateStore.TimelineServiceState;
+
+/**
+ * The service wrapper of {@link TimelineV1DelegationTokenSecretManager}.
+ */
+@Private
+@Unstable
+public class TimelineV1DelegationTokenSecretManagerService extends
+    TimelineDelgationTokenSecretManagerService {
+  private TimelineStateStore stateStore = null;
+
+  public TimelineV1DelegationTokenSecretManagerService() {
+    super(TimelineV1DelegationTokenSecretManagerService.class.getName());
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_RECOVERY_ENABLED,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_RECOVERY_ENABLED)) {
+      stateStore = createStateStore(conf);
+      stateStore.init(conf);
+    }
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    if (stateStore != null) {
+      stateStore.start();
+      TimelineServiceState state = stateStore.loadState();
+      ((TimelineV1DelegationTokenSecretManager)
+          getTimelineDelegationTokenSecretManager()).recover(state);
+    }
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (stateStore != null) {
+      stateStore.stop();
+    }
+    super.serviceStop();
+  }
+
+  @Override
+  protected AbstractDelegationTokenSecretManager
+      <TimelineDelegationTokenIdentifier>
+      createTimelineDelegationTokenSecretManager(long secretKeyInterval,
+          long tokenMaxLifetime, long tokenRenewInterval,
+          long tokenRemovalScanInterval) {
+    return new TimelineV1DelegationTokenSecretManager(secretKeyInterval,
+        tokenMaxLifetime, tokenRenewInterval, tokenRemovalScanInterval,
+        stateStore);
+  }
+
+  protected TimelineStateStore createStateStore(
+      Configuration conf) {
+    return ReflectionUtils.newInstance(
+        conf.getClass(YarnConfiguration.TIMELINE_SERVICE_STATE_STORE_CLASS,
+            LeveldbTimelineStateStore.class,
+            TimelineStateStore.class), conf);
+  }
+
+  /**
+   * Delegation token secret manager for ATSv1 and ATSv1.5.
+   */
+  @Private
+  @Unstable
+  public static class TimelineV1DelegationTokenSecretManager extends
+      AbstractDelegationTokenSecretManager<TimelineDelegationTokenIdentifier> {
+
+    public static final Log LOG =
+        LogFactory.getLog(TimelineV1DelegationTokenSecretManager.class);
+
+    private TimelineStateStore stateStore;
+
+    /**
+     * Create a timeline v1 secret manager.
+     * @param delegationKeyUpdateInterval the number of milliseconds for rolling
+     *        new secret keys.
+     * @param delegationTokenMaxLifetime the maximum lifetime of the delegation
+     *        tokens in milliseconds
+     * @param delegationTokenRenewInterval how often the tokens must be renewed
+     *        in milliseconds
+     * @param delegationTokenRemoverScanInterval how often the tokens are
+     *        scanned for expired tokens in milliseconds
+     * @param stateStore timeline service state store
+     */
+    public TimelineV1DelegationTokenSecretManager(
+        long delegationKeyUpdateInterval,
+        long delegationTokenMaxLifetime,
+        long delegationTokenRenewInterval,
+        long delegationTokenRemoverScanInterval,
+        TimelineStateStore stateStore) {
+      super(delegationKeyUpdateInterval, delegationTokenMaxLifetime,
+          delegationTokenRenewInterval, delegationTokenRemoverScanInterval);
+      this.stateStore = stateStore;
+    }
+
+    @Override
+    public TimelineDelegationTokenIdentifier createIdentifier() {
+      return new TimelineDelegationTokenIdentifier();
+    }
+
+    @Override
+    protected void storeNewMasterKey(DelegationKey key) throws IOException {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Storing master key " + key.getKeyId());
+      }
+      try {
+        if (stateStore != null) {
+          stateStore.storeTokenMasterKey(key);
+        }
+      } catch (IOException e) {
+        LOG.error("Unable to store master key " + key.getKeyId(), e);
+      }
+    }
+
+    @Override
+    protected void removeStoredMasterKey(DelegationKey key) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Removing master key " + key.getKeyId());
+      }
+      try {
+        if (stateStore != null) {
+          stateStore.removeTokenMasterKey(key);
+        }
+      } catch (IOException e) {
+        LOG.error("Unable to remove master key " + key.getKeyId(), e);
+      }
+    }
+
+    @Override
+    protected void storeNewToken(TimelineDelegationTokenIdentifier tokenId,
+        long renewDate) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Storing token " + tokenId.getSequenceNumber());
+      }
+      try {
+        if (stateStore != null) {
+          stateStore.storeToken(tokenId, renewDate);
+        }
+      } catch (IOException e) {
+        LOG.error("Unable to store token " + tokenId.getSequenceNumber(), e);
+      }
+    }
+
+    @Override
+    protected void removeStoredToken(TimelineDelegationTokenIdentifier tokenId)
+        throws IOException {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Storing token " + tokenId.getSequenceNumber());
+      }
+      try {
+        if (stateStore != null) {
+          stateStore.removeToken(tokenId);
+        }
+      } catch (IOException e) {
+        LOG.error("Unable to remove token " + tokenId.getSequenceNumber(), e);
+      }
+    }
+
+    @Override
+    protected void updateStoredToken(TimelineDelegationTokenIdentifier tokenId,
+        long renewDate) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Updating token " + tokenId.getSequenceNumber());
+      }
+      try {
+        if (stateStore != null) {
+          stateStore.updateToken(tokenId, renewDate);
+        }
+      } catch (IOException e) {
+        LOG.error("Unable to update token " + tokenId.getSequenceNumber(), e);
+      }
+    }
+
+    public void recover(TimelineServiceState state) throws IOException {
+      LOG.info("Recovering " + getClass().getSimpleName());
+      for (DelegationKey key : state.getTokenMasterKeyState()) {
+        addKey(key);
+      }
+      this.delegationTokenSequenceNumber = state.getLatestSequenceNumber();
+      for (Entry<TimelineDelegationTokenIdentifier, Long> entry :
+          state.getTokenState().entrySet()) {
+        addPersistedDelegationToken(entry.getKey(), entry.getValue());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilter.java
deleted file mode 100644
index 063f512..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilter.java
+++ /dev/null
@@ -1,323 +0,0 @@
-/**
- * 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.timeline.security;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.security.PrivilegedExceptionAction;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.concurrent.Callable;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.http.HttpConfig;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.minikdc.MiniKdc;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.KerberosTestUtils;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
-import org.apache.hadoop.security.authorize.AuthorizationException;
-import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
-import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
-import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
-import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer;
-import org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore;
-import org.apache.hadoop.yarn.server.timeline.TimelineStore;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class TestTimelineAuthenticationFilter {
-
-  private static final String FOO_USER = "foo";
-  private static final String BAR_USER = "bar";
-  private static final String HTTP_USER = "HTTP";
-
-  private static final File testRootDir = new File(
-      System.getProperty("test.build.dir", "target/test-dir"),
-      TestTimelineAuthenticationFilter.class.getName() + "-root");
-  private static File httpSpnegoKeytabFile = new File(
-      KerberosTestUtils.getKeytabFile());
-  private static String httpSpnegoPrincipal =
-      KerberosTestUtils.getServerPrincipal();
-  private static final String BASEDIR =
-      System.getProperty("test.build.dir", "target/test-dir") + "/"
-          + TestTimelineAuthenticationFilter.class.getSimpleName();
-
-  @Parameterized.Parameters
-  public static Collection<Object[]> withSsl() {
-    return Arrays.asList(new Object[][] { { false }, { true } });
-  }
-
-  private static MiniKdc testMiniKDC;
-  private static String keystoresDir;
-  private static String sslConfDir;
-  private static ApplicationHistoryServer testTimelineServer;
-  private static Configuration conf;
-  private static boolean withSsl;
-
-  public TestTimelineAuthenticationFilter(boolean withSsl) {
-    TestTimelineAuthenticationFilter.withSsl = withSsl;
-  }
-
-  @BeforeClass
-  public static void setup() {
-    try {
-      testMiniKDC = new MiniKdc(MiniKdc.createConf(), testRootDir);
-      testMiniKDC.start();
-      testMiniKDC.createPrincipal(
-          httpSpnegoKeytabFile, HTTP_USER + "/localhost");
-    } catch (Exception e) {
-      assertTrue("Couldn't setup MiniKDC", false);
-    }
-
-    try {
-      testTimelineServer = new ApplicationHistoryServer();
-      conf = new Configuration(false);
-      conf.setStrings(TimelineAuthenticationFilterInitializer.PREFIX + "type",
-          "kerberos");
-      conf.set(TimelineAuthenticationFilterInitializer.PREFIX +
-          KerberosAuthenticationHandler.PRINCIPAL, httpSpnegoPrincipal);
-      conf.set(TimelineAuthenticationFilterInitializer.PREFIX +
-          KerberosAuthenticationHandler.KEYTAB,
-          httpSpnegoKeytabFile.getAbsolutePath());
-      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
-        "kerberos");
-      conf.set(YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL,
-        httpSpnegoPrincipal);
-      conf.set(YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
-        httpSpnegoKeytabFile.getAbsolutePath());
-      conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
-      conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
-          MemoryTimelineStore.class, TimelineStore.class);
-      conf.set(YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
-          "localhost:10200");
-      conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
-          "localhost:8188");
-      conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
-          "localhost:8190");
-      conf.set("hadoop.proxyuser.HTTP.hosts", "*");
-      conf.set("hadoop.proxyuser.HTTP.users", FOO_USER);
-      conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, 1);
-
-      if (withSsl) {
-        conf.set(YarnConfiguration.YARN_HTTP_POLICY_KEY,
-            HttpConfig.Policy.HTTPS_ONLY.name());
-        File base = new File(BASEDIR);
-        FileUtil.fullyDelete(base);
-        base.mkdirs();
-        keystoresDir = new File(BASEDIR).getAbsolutePath();
-        sslConfDir =
-            KeyStoreTestUtil.getClasspathDir(TestTimelineAuthenticationFilter.class);
-        KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
-      }
-
-      UserGroupInformation.setConfiguration(conf);
-      testTimelineServer.init(conf);
-      testTimelineServer.start();
-    } catch (Exception e) {
-      assertTrue("Couldn't setup TimelineServer", false);
-    }
-  }
-
-  private TimelineClient createTimelineClientForUGI() {
-    TimelineClient client = TimelineClient.createTimelineClient();
-    client.init(conf);
-    client.start();
-    return client;
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    if (testMiniKDC != null) {
-      testMiniKDC.stop();
-    }
-
-    if (testTimelineServer != null) {
-      testTimelineServer.stop();
-    }
-
-    if (withSsl) {
-      KeyStoreTestUtil.cleanupSSLConfig(keystoresDir, sslConfDir);
-      File base = new File(BASEDIR);
-      FileUtil.fullyDelete(base);
-    }
-  }
-
-  @Test
-  public void testPutTimelineEntities() throws Exception {
-    KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        TimelineClient client = createTimelineClientForUGI();
-        TimelineEntity entityToStore = new TimelineEntity();
-        entityToStore.setEntityType(
-            TestTimelineAuthenticationFilter.class.getName());
-        entityToStore.setEntityId("entity1");
-        entityToStore.setStartTime(0L);
-        TimelinePutResponse putResponse = client.putEntities(entityToStore);
-        Assert.assertEquals(0, putResponse.getErrors().size());
-        TimelineEntity entityToRead =
-            testTimelineServer.getTimelineStore().getEntity(
-                "entity1", TestTimelineAuthenticationFilter.class.getName(), null);
-        Assert.assertNotNull(entityToRead);
-        return null;
-      }
-    });
-  }
-
-  @Test
-  public void testPutDomains() throws Exception {
-    KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<Void>() {
-      @Override
-      public Void call() throws Exception {
-        TimelineClient client = createTimelineClientForUGI();
-        TimelineDomain domainToStore = new TimelineDomain();
-        domainToStore.setId(TestTimelineAuthenticationFilter.class.getName());
-        domainToStore.setReaders("*");
-        domainToStore.setWriters("*");
-        client.putDomain(domainToStore);
-        TimelineDomain domainToRead =
-            testTimelineServer.getTimelineStore().getDomain(
-                TestTimelineAuthenticationFilter.class.getName());
-        Assert.assertNotNull(domainToRead);
-        return null;
-      }
-    });
-  }
-
-  @Test
-  public void testDelegationTokenOperations() throws Exception {
-    TimelineClient httpUserClient =
-      KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<TimelineClient>() {
-        @Override
-        public TimelineClient call() throws Exception {
-          return createTimelineClientForUGI();
-        }
-      });
-    UserGroupInformation httpUser =
-      KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<UserGroupInformation>() {
-        @Override
-        public UserGroupInformation call() throws Exception {
-          return UserGroupInformation.getCurrentUser();
-        }
-      });
-    // Let HTTP user to get the delegation for itself
-    Token<TimelineDelegationTokenIdentifier> token =
-      httpUserClient.getDelegationToken(httpUser.getShortUserName());
-    Assert.assertNotNull(token);
-    TimelineDelegationTokenIdentifier tDT = token.decodeIdentifier();
-    Assert.assertNotNull(tDT);
-    Assert.assertEquals(new Text(HTTP_USER), tDT.getOwner());
-
-    // Renew token
-    Assert.assertFalse(token.getService().toString().isEmpty());
-    // Renew the token from the token service address
-    long renewTime1 = httpUserClient.renewDelegationToken(token);
-    Thread.sleep(100);
-    token.setService(new Text());
-    Assert.assertTrue(token.getService().toString().isEmpty());
-    // If the token service address is not avaiable, it still can be renewed
-    // from the configured address
-    long renewTime2 = httpUserClient.renewDelegationToken(token);
-    Assert.assertTrue(renewTime1 < renewTime2);
-
-    // Cancel token
-    Assert.assertTrue(token.getService().toString().isEmpty());
-    // If the token service address is not avaiable, it still can be canceled
-    // from the configured address
-    httpUserClient.cancelDelegationToken(token);
-    // Renew should not be successful because the token is canceled
-    try {
-      httpUserClient.renewDelegationToken(token);
-      Assert.fail();
-    } catch (Exception e) {
-      Assert.assertTrue(e.getMessage().contains(
-            "Renewal request for unknown token"));
-    }
-
-    // Let HTTP user to get the delegation token for FOO user
-    UserGroupInformation fooUgi = UserGroupInformation.createProxyUser(
-        FOO_USER, httpUser);
-    TimelineClient fooUserClient = fooUgi.doAs(
-        new PrivilegedExceptionAction<TimelineClient>() {
-          @Override
-          public TimelineClient run() throws Exception {
-            return createTimelineClientForUGI();
-          }
-        });
-    token = fooUserClient.getDelegationToken(httpUser.getShortUserName());
-    Assert.assertNotNull(token);
-    tDT = token.decodeIdentifier();
-    Assert.assertNotNull(tDT);
-    Assert.assertEquals(new Text(FOO_USER), tDT.getOwner());
-    Assert.assertEquals(new Text(HTTP_USER), tDT.getRealUser());
-
-    // Renew token as the renewer
-    final Token<TimelineDelegationTokenIdentifier> tokenToRenew = token;
-    renewTime1 = httpUserClient.renewDelegationToken(tokenToRenew);
-    renewTime2 = httpUserClient.renewDelegationToken(tokenToRenew);
-    Assert.assertTrue(renewTime1 < renewTime2);
-
-    // Cancel token
-    Assert.assertFalse(tokenToRenew.getService().toString().isEmpty());
-    // Cancel the token from the token service address
-    fooUserClient.cancelDelegationToken(tokenToRenew);
-
-    // Renew should not be successful because the token is canceled
-    try {
-      httpUserClient.renewDelegationToken(tokenToRenew);
-      Assert.fail();
-    } catch (Exception e) {
-      Assert.assertTrue(
-          e.getMessage().contains("Renewal request for unknown token"));
-    }
-
-    // Let HTTP user to get the delegation token for BAR user
-    UserGroupInformation barUgi = UserGroupInformation.createProxyUser(
-        BAR_USER, httpUser);
-    TimelineClient barUserClient = barUgi.doAs(
-        new PrivilegedExceptionAction<TimelineClient>() {
-          @Override
-          public TimelineClient run() {
-            return createTimelineClientForUGI();
-          }
-        });
-
-    try {
-      barUserClient.getDelegationToken(httpUser.getShortUserName());
-      Assert.fail();
-    } catch (Exception e) {
-      Assert.assertTrue(e.getCause() instanceof AuthorizationException || e.getCause() instanceof AuthenticationException);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterForV1.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterForV1.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterForV1.java
new file mode 100644
index 0000000..d918e8d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterForV1.java
@@ -0,0 +1,332 @@
+/**
+ * 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.timeline.security;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.KerberosTestUtils;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
+import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer;
+import org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore;
+import org.apache.hadoop.yarn.server.timeline.TimelineStore;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Test cases for authentication via TimelineAuthenticationFilter while
+ * publishing entities for ATSv1.
+ */
+@RunWith(Parameterized.class)
+public class TestTimelineAuthenticationFilterForV1 {
+
+  private static final String FOO_USER = "foo";
+  private static final String BAR_USER = "bar";
+  private static final String HTTP_USER = "HTTP";
+
+  private static final File TEST_ROOT_DIR = new File(
+      System.getProperty("test.build.dir", "target/test-dir"),
+          TestTimelineAuthenticationFilterForV1.class.getName() + "-root");
+  private static File httpSpnegoKeytabFile = new File(
+      KerberosTestUtils.getKeytabFile());
+  private static String httpSpnegoPrincipal =
+      KerberosTestUtils.getServerPrincipal();
+  private static final String BASEDIR =
+      System.getProperty("test.build.dir", "target/test-dir") + "/"
+          + TestTimelineAuthenticationFilterForV1.class.getSimpleName();
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> withSsl() {
+    return Arrays.asList(new Object[][] {{false}, {true}});
+  }
+
+  private static MiniKdc testMiniKDC;
+  private static String keystoresDir;
+  private static String sslConfDir;
+  private static ApplicationHistoryServer testTimelineServer;
+  private static Configuration conf;
+  private static boolean withSsl;
+
+  public TestTimelineAuthenticationFilterForV1(boolean withSsl) {
+    TestTimelineAuthenticationFilterForV1.withSsl = withSsl;
+  }
+
+  @BeforeClass
+  public static void setup() {
+    try {
+      testMiniKDC = new MiniKdc(MiniKdc.createConf(), TEST_ROOT_DIR);
+      testMiniKDC.start();
+      testMiniKDC.createPrincipal(
+          httpSpnegoKeytabFile, HTTP_USER + "/localhost");
+    } catch (Exception e) {
+      assertTrue("Couldn't setup MiniKDC", false);
+    }
+
+    try {
+      testTimelineServer = new ApplicationHistoryServer();
+      conf = new Configuration(false);
+      conf.setStrings(TimelineAuthenticationFilterInitializer.PREFIX + "type",
+          "kerberos");
+      conf.set(TimelineAuthenticationFilterInitializer.PREFIX +
+          KerberosAuthenticationHandler.PRINCIPAL, httpSpnegoPrincipal);
+      conf.set(TimelineAuthenticationFilterInitializer.PREFIX +
+          KerberosAuthenticationHandler.KEYTAB,
+          httpSpnegoKeytabFile.getAbsolutePath());
+      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+          "kerberos");
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL,
+          httpSpnegoPrincipal);
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
+          httpSpnegoKeytabFile.getAbsolutePath());
+      conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+      conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
+          MemoryTimelineStore.class, TimelineStore.class);
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
+          "localhost:10200");
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+          "localhost:8188");
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
+          "localhost:8190");
+      conf.set("hadoop.proxyuser.HTTP.hosts", "*");
+      conf.set("hadoop.proxyuser.HTTP.users", FOO_USER);
+      conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, 1);
+
+      if (withSsl) {
+        conf.set(YarnConfiguration.YARN_HTTP_POLICY_KEY,
+            HttpConfig.Policy.HTTPS_ONLY.name());
+        File base = new File(BASEDIR);
+        FileUtil.fullyDelete(base);
+        base.mkdirs();
+        keystoresDir = new File(BASEDIR).getAbsolutePath();
+        sslConfDir = KeyStoreTestUtil.getClasspathDir(
+            TestTimelineAuthenticationFilterForV1.class);
+        KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
+      }
+
+      UserGroupInformation.setConfiguration(conf);
+      testTimelineServer.init(conf);
+      testTimelineServer.start();
+    } catch (Exception e) {
+      e.printStackTrace();
+      assertTrue("Couldn't setup TimelineServer", false);
+    }
+  }
+
+  private TimelineClient createTimelineClientForUGI() {
+    TimelineClient client = TimelineClient.createTimelineClient();
+    client.init(conf);
+    client.start();
+    return client;
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if (testMiniKDC != null) {
+      testMiniKDC.stop();
+    }
+
+    if (testTimelineServer != null) {
+      testTimelineServer.stop();
+    }
+
+    if (withSsl) {
+      KeyStoreTestUtil.cleanupSSLConfig(keystoresDir, sslConfDir);
+      File base = new File(BASEDIR);
+      FileUtil.fullyDelete(base);
+    }
+  }
+
+  @Test
+  public void testPutTimelineEntities() throws Exception {
+    KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        TimelineClient client = createTimelineClientForUGI();
+        TimelineEntity entityToStore = new TimelineEntity();
+        entityToStore.setEntityType(
+            TestTimelineAuthenticationFilterForV1.class.getName());
+        entityToStore.setEntityId("entity1");
+        entityToStore.setStartTime(0L);
+        TimelinePutResponse putResponse = client.putEntities(entityToStore);
+        Assert.assertEquals(0, putResponse.getErrors().size());
+        TimelineEntity entityToRead =
+            testTimelineServer.getTimelineStore().getEntity("entity1",
+                TestTimelineAuthenticationFilterForV1.class.getName(), null);
+        Assert.assertNotNull(entityToRead);
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void testPutDomains() throws Exception {
+    KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        TimelineClient client = createTimelineClientForUGI();
+        TimelineDomain domainToStore = new TimelineDomain();
+        domainToStore.setId(
+            TestTimelineAuthenticationFilterForV1.class.getName());
+        domainToStore.setReaders("*");
+        domainToStore.setWriters("*");
+        client.putDomain(domainToStore);
+        TimelineDomain domainToRead =
+            testTimelineServer.getTimelineStore().getDomain(
+                TestTimelineAuthenticationFilterForV1.class.getName());
+        Assert.assertNotNull(domainToRead);
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void testDelegationTokenOperations() throws Exception {
+    TimelineClient httpUserClient =
+        KerberosTestUtils.doAs(HTTP_USER + "/localhost",
+            new Callable<TimelineClient>() {
+            @Override
+            public TimelineClient call() throws Exception {
+              return createTimelineClientForUGI();
+            }
+          });
+    UserGroupInformation httpUser =
+        KerberosTestUtils.doAs(HTTP_USER + "/localhost",
+            new Callable<UserGroupInformation>() {
+            @Override
+            public UserGroupInformation call() throws Exception {
+              return UserGroupInformation.getCurrentUser();
+            }
+          });
+    // Let HTTP user to get the delegation for itself
+    Token<TimelineDelegationTokenIdentifier> token =
+        httpUserClient.getDelegationToken(httpUser.getShortUserName());
+    Assert.assertNotNull(token);
+    TimelineDelegationTokenIdentifier tDT = token.decodeIdentifier();
+    Assert.assertNotNull(tDT);
+    Assert.assertEquals(new Text(HTTP_USER), tDT.getOwner());
+
+    // Renew token
+    Assert.assertFalse(token.getService().toString().isEmpty());
+    // Renew the token from the token service address
+    long renewTime1 = httpUserClient.renewDelegationToken(token);
+    Thread.sleep(100);
+    token.setService(new Text());
+    Assert.assertTrue(token.getService().toString().isEmpty());
+    // If the token service address is not avaiable, it still can be renewed
+    // from the configured address
+    long renewTime2 = httpUserClient.renewDelegationToken(token);
+    Assert.assertTrue(renewTime1 < renewTime2);
+
+    // Cancel token
+    Assert.assertTrue(token.getService().toString().isEmpty());
+    // If the token service address is not avaiable, it still can be canceled
+    // from the configured address
+    httpUserClient.cancelDelegationToken(token);
+    // Renew should not be successful because the token is canceled
+    try {
+      httpUserClient.renewDelegationToken(token);
+      Assert.fail();
+    } catch (Exception e) {
+      Assert.assertTrue(e.getMessage().contains(
+            "Renewal request for unknown token"));
+    }
+
+    // Let HTTP user to get the delegation token for FOO user
+    UserGroupInformation fooUgi = UserGroupInformation.createProxyUser(
+        FOO_USER, httpUser);
+    TimelineClient fooUserClient = fooUgi.doAs(
+        new PrivilegedExceptionAction<TimelineClient>() {
+          @Override
+          public TimelineClient run() throws Exception {
+            return createTimelineClientForUGI();
+          }
+        });
+    token = fooUserClient.getDelegationToken(httpUser.getShortUserName());
+    Assert.assertNotNull(token);
+    tDT = token.decodeIdentifier();
+    Assert.assertNotNull(tDT);
+    Assert.assertEquals(new Text(FOO_USER), tDT.getOwner());
+    Assert.assertEquals(new Text(HTTP_USER), tDT.getRealUser());
+
+    // Renew token as the renewer
+    final Token<TimelineDelegationTokenIdentifier> tokenToRenew = token;
+    renewTime1 = httpUserClient.renewDelegationToken(tokenToRenew);
+    renewTime2 = httpUserClient.renewDelegationToken(tokenToRenew);
+    Assert.assertTrue(renewTime1 < renewTime2);
+
+    // Cancel token
+    Assert.assertFalse(tokenToRenew.getService().toString().isEmpty());
+    // Cancel the token from the token service address
+    fooUserClient.cancelDelegationToken(tokenToRenew);
+
+    // Renew should not be successful because the token is canceled
+    try {
+      httpUserClient.renewDelegationToken(tokenToRenew);
+      Assert.fail();
+    } catch (Exception e) {
+      Assert.assertTrue(
+          e.getMessage().contains("Renewal request for unknown token"));
+    }
+
+    // Let HTTP user to get the delegation token for BAR user
+    UserGroupInformation barUgi = UserGroupInformation.createProxyUser(
+        BAR_USER, httpUser);
+    TimelineClient barUserClient = barUgi.doAs(
+        new PrivilegedExceptionAction<TimelineClient>() {
+          @Override
+          public TimelineClient run() {
+            return createTimelineClientForUGI();
+          }
+        });
+
+    try {
+      barUserClient.getDelegationToken(httpUser.getShortUserName());
+      Assert.fail();
+    } catch (Exception e) {
+      Assert.assertTrue(e.getCause() instanceof AuthorizationException ||
+          e.getCause() instanceof AuthenticationException);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java
deleted file mode 100644
index 430911e..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/**
- * 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.timeline.security;
-
-import org.junit.Assert;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.http.FilterContainer;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import static org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer.PREFIX;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-
-public class TestTimelineAuthenticationFilterInitializer {
-
-  @Test
-  public void testProxyUserConfiguration() {
-    FilterContainer container = Mockito.mock(FilterContainer.class);
-    for (int i = 0; i < 3; ++i) {
-      Configuration conf = new YarnConfiguration();
-      switch (i) {
-      case 0:
-        // hadoop.proxyuser prefix
-        conf.set("hadoop.proxyuser.foo.hosts", "*");
-        conf.set("hadoop.proxyuser.foo.users", "*");
-        conf.set("hadoop.proxyuser.foo.groups", "*");
-        break;
-      case 1:
-        // yarn.timeline-service.http-authentication.proxyuser prefix
-        conf.set(PREFIX + "proxyuser.foo.hosts", "*");
-        conf.set(PREFIX + "proxyuser.foo.users", "*");
-        conf.set(PREFIX + "proxyuser.foo.groups", "*");
-        break;
-      case 2:
-        // hadoop.proxyuser prefix has been overwritten by
-        // yarn.timeline-service.http-authentication.proxyuser prefix
-        conf.set("hadoop.proxyuser.foo.hosts", "bar");
-        conf.set("hadoop.proxyuser.foo.users", "bar");
-        conf.set("hadoop.proxyuser.foo.groups", "bar");
-        conf.set(PREFIX + "proxyuser.foo.hosts", "*");
-        conf.set(PREFIX + "proxyuser.foo.users", "*");
-        conf.set(PREFIX + "proxyuser.foo.groups", "*");
-        break;
-      default:
-        break;
-      }
-
-      TimelineAuthenticationFilterInitializer initializer =
-          new TimelineAuthenticationFilterInitializer();
-      initializer.initFilter(container, conf);
-      Assert.assertEquals(
-          "*", initializer.filterConfig.get("proxyuser.foo.hosts"));
-      Assert.assertEquals(
-          "*", initializer.filterConfig.get("proxyuser.foo.users"));
-      Assert.assertEquals(
-          "*", initializer.filterConfig.get("proxyuser.foo.groups"));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java
new file mode 100644
index 0000000..f6d1863
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilter.java
@@ -0,0 +1,55 @@
+/**
+ * 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.timeline.security;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+
+/**
+ * Timeline authentication filter provides delegation token support for ATSv1
+ * and ATSv2.
+ */
+@Private
+@Unstable
+public class TimelineAuthenticationFilter
+    extends DelegationTokenAuthenticationFilter {
+
+  private static AbstractDelegationTokenSecretManager
+      <TimelineDelegationTokenIdentifier> secretManager;
+
+  @Override
+  public void init(FilterConfig filterConfig) throws ServletException {
+    filterConfig.getServletContext().setAttribute(
+        DelegationTokenAuthenticationFilter.
+            DELEGATION_TOKEN_SECRET_MANAGER_ATTR, secretManager);
+    super.init(filterConfig);
+  }
+
+  public static void setTimelineDelegationTokenSecretManager(
+      AbstractDelegationTokenSecretManager
+          <TimelineDelegationTokenIdentifier> secretMgr) {
+    TimelineAuthenticationFilter.secretManager = secretMgr;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
new file mode 100644
index 0000000..4e7c29a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
@@ -0,0 +1,129 @@
+/**
+ * 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.timeline.security;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.FilterContainer;
+import org.apache.hadoop.http.FilterInitializer;
+import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
+import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
+import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler;
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticationHandler;
+import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticationHandler;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Initializes {@link TimelineAuthenticationFilter} which provides support for
+ * Kerberos HTTP SPNEGO authentication.
+ * <p>
+ * It enables Kerberos HTTP SPNEGO plus delegation token authentication for the
+ * timeline server.
+ * <p>
+ * Refer to the {@code core-default.xml} file, after the comment 'HTTP
+ * Authentication' for details on the configuration options. All related
+ * configuration properties have {@code hadoop.http.authentication.} as prefix.
+ */
+public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
+
+  /**
+   * The configuration prefix of timeline HTTP authentication
+   */
+  public static final String PREFIX = "yarn.timeline-service.http-authentication.";
+
+  @VisibleForTesting
+  Map<String, String> filterConfig;
+
+  /**
+   * Initializes {@link TimelineAuthenticationFilter}
+   * <p>
+   * Propagates to {@link TimelineAuthenticationFilter} configuration all YARN
+   * configuration properties prefixed with {@value #PREFIX}
+   *
+   * @param container
+   *          The filter container
+   * @param conf
+   *          Configuration for run-time parameters
+   */
+  @Override
+  public void initFilter(FilterContainer container, Configuration conf) {
+    filterConfig = new HashMap<String, String>();
+
+    // setting the cookie path to root '/' so it is used for all resources.
+    filterConfig.put(TimelineAuthenticationFilter.COOKIE_PATH, "/");
+
+    for (Map.Entry<String, String> entry : conf) {
+      String name = entry.getKey();
+      if (name.startsWith(ProxyUsers.CONF_HADOOP_PROXYUSER)) {
+        String value = conf.get(name);
+        name = name.substring("hadoop.".length());
+        filterConfig.put(name, value);
+      }
+    }
+    for (Map.Entry<String, String> entry : conf) {
+      String name = entry.getKey();
+      if (name.startsWith(PREFIX)) {
+        // yarn.timeline-service.http-authentication.proxyuser will override
+        // hadoop.proxyuser
+        String value = conf.get(name);
+        name = name.substring(PREFIX.length());
+        filterConfig.put(name, value);
+      }
+    }
+
+    String authType = filterConfig.get(AuthenticationFilter.AUTH_TYPE);
+    if (authType.equals(PseudoAuthenticationHandler.TYPE)) {
+      filterConfig.put(AuthenticationFilter.AUTH_TYPE,
+          PseudoDelegationTokenAuthenticationHandler.class.getName());
+    } else if (authType.equals(KerberosAuthenticationHandler.TYPE)) {
+      filterConfig.put(AuthenticationFilter.AUTH_TYPE,
+          KerberosDelegationTokenAuthenticationHandler.class.getName());
+
+      // Resolve _HOST into bind address
+      String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
+      String principal =
+          filterConfig.get(KerberosAuthenticationHandler.PRINCIPAL);
+      if (principal != null) {
+        try {
+          principal = SecurityUtil.getServerPrincipal(principal, bindAddress);
+        } catch (IOException ex) {
+          throw new RuntimeException(
+              "Could not resolve Kerberos principal name: " + ex.toString(), ex);
+        }
+        filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL,
+            principal);
+      }
+    }
+
+    filterConfig.put(DelegationTokenAuthenticationHandler.TOKEN_KIND,
+        TimelineDelegationTokenIdentifier.KIND_NAME.toString());
+
+    container.addGlobalFilter("Timeline Authentication Filter",
+        TimelineAuthenticationFilter.class.getName(),
+        filterConfig);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelgationTokenSecretManagerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelgationTokenSecretManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelgationTokenSecretManagerService.java
new file mode 100644
index 0000000..2e95af2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelgationTokenSecretManagerService.java
@@ -0,0 +1,83 @@
+/**
+ * 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.timeline.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+
+/**
+ * Abstract implementation of delegation token manager service for different
+ * versions of timeline service.
+ */
+public abstract class TimelineDelgationTokenSecretManagerService extends
+    AbstractService {
+
+  public TimelineDelgationTokenSecretManagerService(String name) {
+    super(name);
+  }
+
+  private static long delegationTokenRemovalScanInterval = 3600000L;
+
+  private AbstractDelegationTokenSecretManager
+      <TimelineDelegationTokenIdentifier> secretManager = null;
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    long secretKeyInterval =
+        conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL,
+            YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL);
+    long tokenMaxLifetime =
+        conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME,
+            YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME);
+    long tokenRenewInterval =
+        conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL,
+            YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL);
+    secretManager = createTimelineDelegationTokenSecretManager(
+        secretKeyInterval, tokenMaxLifetime, tokenRenewInterval,
+        delegationTokenRemovalScanInterval);
+    super.init(conf);
+  }
+
+  protected abstract
+      AbstractDelegationTokenSecretManager<TimelineDelegationTokenIdentifier>
+          createTimelineDelegationTokenSecretManager(long secretKeyInterval,
+          long tokenMaxLifetime, long tokenRenewInterval,
+          long tokenRemovalScanInterval);
+
+  @Override
+  protected void serviceStart() throws Exception {
+    secretManager.startThreads();
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    secretManager.stopThreads();
+    super.stop();
+  }
+
+  public AbstractDelegationTokenSecretManager
+      <TimelineDelegationTokenIdentifier>
+          getTimelineDelegationTokenSecretManager() {
+    return secretManager;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/package-info.java
new file mode 100644
index 0000000..14a52e3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/package-info.java
@@ -0,0 +1,26 @@
+/**
+ * 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.server.timeline.security contains classes related
+ * to timeline authentication filters and abstract delegation token service for
+ * ATSv1 and ATSv2.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.server.timeline.security;
+import org.apache.hadoop.classification.InterfaceAudience;


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


[12/50] [abbrv] hadoop git commit: YARN-6253. FlowAcitivityColumnPrefix.store(byte[] rowKey, ...) drops timestamp. Contributed by Haibo Chen.

Posted by va...@apache.org.
YARN-6253. FlowAcitivityColumnPrefix.store(byte[] rowKey, ...) drops timestamp. Contributed by Haibo Chen.

(cherry picked from commit 34e7c30293b5a56f9f745769a29c5666bdb85d6c)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 177a324db7f07685ab593cf8ad33af34e7a47f4e
Parents: 70b788a
Author: Sangjin Lee <sj...@apache.org>
Authored: Tue Feb 28 16:10:25 2017 -0800
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:44 2017 +0530

----------------------------------------------------------------------
 .../timelineservice/storage/flow/FlowActivityColumnPrefix.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/177a324d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.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/flow/FlowActivityColumnPrefix.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/flow/FlowActivityColumnPrefix.java
index 439e0c8..5e7a5d6 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/flow/FlowActivityColumnPrefix.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/flow/FlowActivityColumnPrefix.java
@@ -271,7 +271,7 @@ public enum FlowActivityColumnPrefix
     byte[] columnQualifier = getColumnPrefixBytes(qualifier);
     Attribute[] combinedAttributes =
         HBaseTimelineStorageUtils.combineAttributes(attributes, this.aggOp);
-    column.store(rowKey, tableMutator, columnQualifier, null, inputValue,
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         combinedAttributes);
   }
 }
\ No newline at end of file


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


[10/50] [abbrv] hadoop git commit: YARN-6027. Support fromid(offset) filter for /flows API (Rohith Sharma K S via Varun Saxena)

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1862a030/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.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/TimelineReaderServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
index b45fd36..cf0fa50 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
@@ -175,7 +175,7 @@ public class TimelineReaderServer extends CompositeService {
   }
 
   @VisibleForTesting
-  int getWebServerPort() {
+  public int getWebServerPort() {
     return readerWebServer.getConnectorAddress(0).getPort();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1862a030/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.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/TimelineReaderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java
index c93c631..8f92433 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java
@@ -24,14 +24,30 @@ import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Set of utility methods to be used across timeline reader.
  */
-final class TimelineReaderUtils {
+public final class TimelineReaderUtils {
   private TimelineReaderUtils() {
   }
 
   /**
+   * Default delimiter for joining strings.
+   */
+  @VisibleForTesting
+  public static final char DEFAULT_DELIMITER_CHAR = '!';
+
+  /**
+   * Default escape character used for joining strings.
+   */
+  @VisibleForTesting
+  public static final char DEFAULT_ESCAPE_CHAR = '*';
+
+  public static final String FROMID_KEY = "FROM_ID";
+
+  /**
    * Split the passed string along the passed delimiter character while looking
    * for escape char to interpret the splitted parts correctly. For delimiter or
    * escape character to be interpreted as part of the string, they have to be
@@ -168,4 +184,14 @@ final class TimelineReaderUtils {
     // Join the strings after they have been escaped.
     return StringUtils.join(strs, delimiterChar);
   }
+
+  public static List<String> split(final String str)
+      throws IllegalArgumentException {
+    return split(str, DEFAULT_DELIMITER_CHAR, DEFAULT_ESCAPE_CHAR);
+  }
+
+  public static String joinAndEscapeStrings(final String[] strs) {
+    return joinAndEscapeStrings(strs, DEFAULT_DELIMITER_CHAR,
+        DEFAULT_ESCAPE_CHAR);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1862a030/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 df3ccab..1f82d91 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
@@ -1333,6 +1333,10 @@ public class TimelineReaderWebServices {
    *     2 dates.
    *     "daterange=20150711-" returns flows active on and after 20150711.
    *     "daterange=-20150711" returns flows active on and before 20150711.
+   * @param fromId If specified, retrieve the next set of flows from the given
+   *     fromId. The set of flows retrieved is inclusive of specified fromId.
+   *     fromId should be taken from the value associated with FROM_ID info key
+   *     in flow entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     set of <cite>FlowActivityEntity</cite> instances are returned.<br>
@@ -1349,8 +1353,9 @@ public class TimelineReaderWebServices {
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @QueryParam("limit") String limit,
-      @QueryParam("daterange") String dateRange) {
-    return getFlows(req, res, null, limit, dateRange);
+      @QueryParam("daterange") String dateRange,
+      @QueryParam("fromid") String fromId) {
+    return getFlows(req, res, null, limit, dateRange, fromId);
   }
 
   /**
@@ -1379,6 +1384,10 @@ public class TimelineReaderWebServices {
    *     2 dates.
    *     "daterange=20150711-" returns flows active on and after 20150711.
    *     "daterange=-20150711" returns flows active on and before 20150711.
+   * @param fromId If specified, retrieve the next set of flows from the given
+   *     fromId. The set of flows retrieved is inclusive of specified fromId.
+   *     fromId should be taken from the value associated with FROM_ID info key
+   *     in flow entity response which was sent earlier.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     set of <cite>FlowActivityEntity</cite> instances are returned.<br>
@@ -1396,7 +1405,8 @@ public class TimelineReaderWebServices {
       @Context HttpServletResponse res,
       @PathParam("clusterid") String clusterId,
       @QueryParam("limit") String limit,
-      @QueryParam("daterange") String dateRange) {
+      @QueryParam("daterange") String dateRange,
+      @QueryParam("fromid") String fromId) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -1413,7 +1423,7 @@ public class TimelineReaderWebServices {
       TimelineEntityFilters entityFilters =
           TimelineReaderWebServicesUtils.createTimelineEntityFilters(
               limit, null, null, null, null, null, null, null, null, null,
-              null);
+              fromId);
       entityFilters.setCreatedTimeBegin(range.dateStart);
       entityFilters.setCreatedTimeEnd(range.dateEnd);
       entities = timelineReaderManager.getEntities(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1862a030/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 780cfd0..52e24e1 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
@@ -195,39 +195,29 @@ enum TimelineUIDConverter {
   };
 
   /**
-   * Delimiter used for UID.
-   */
-  public static final char UID_DELIMITER_CHAR = '!';
-
-  /**
-   * Escape Character used if delimiter or escape character itself is part of
-   * different components of UID.
-   */
-  public static final char UID_ESCAPE_CHAR = '*';
-
-  /**
-   * Split UID using {@link #UID_DELIMITER_CHAR} and {@link #UID_ESCAPE_CHAR}.
+   * Split UID using {@link TimelineReaderUtils#DEFAULT_DELIMITER_CHAR} and
+   * {@link TimelineReaderUtils#DEFAULT_ESCAPE_CHAR}.
    * @param uid UID to be splitted.
    * @return a list of different parts of UID split across delimiter.
    * @throws IllegalArgumentException if UID is not properly escaped.
    */
   private static List<String> splitUID(String uid)
       throws IllegalArgumentException {
-    return TimelineReaderUtils.split(uid, UID_DELIMITER_CHAR, UID_ESCAPE_CHAR);
+    return TimelineReaderUtils.split(uid);
   }
 
   /**
-   * Join different parts of UID delimited by {@link #UID_DELIMITER_CHAR} with
-   * delimiter and escape character escaped using {@link #UID_ESCAPE_CHAR} if
-   * UID parts contain them.
+   * Join different parts of UID delimited by
+   * {@link TimelineReaderUtils#DEFAULT_DELIMITER_CHAR} with delimiter and
+   * escape character escaped using
+   * {@link TimelineReaderUtils#DEFAULT_ESCAPE_CHAR} if UID parts contain them.
    * @param parts an array of UID parts to be joined.
    * @return a string joined using the delimiter with escape and delimiter
-   *     characters escaped if they are part of the string parts to be joined.
-   *     Returns null if one of the parts is null.
+   *         characters escaped if they are part of the string parts to be
+   *         joined. Returns null if one of the parts is null.
    */
   private static String joinAndEscapeUIDParts(String[] parts) {
-    return TimelineReaderUtils.joinAndEscapeStrings(parts, UID_DELIMITER_CHAR,
-        UID_ESCAPE_CHAR);
+    return TimelineReaderUtils.joinAndEscapeStrings(parts);
   }
 
   /**


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


[47/50] [abbrv] hadoop git commit: YARN-6638. [ATSv2 Security] Timeline reader side changes for loading auth filters and principals. Contributed by Varun Saxena

Posted by va...@apache.org.
YARN-6638. [ATSv2 Security] Timeline reader side changes for loading auth filters and principals. Contributed by Varun Saxena

(cherry picked from commit 7f06c46065936b88e9864c526bfcda3a89f3e257)

Conflicts:
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.java
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java


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

Branch: refs/heads/YARN-5355_branch2
Commit: 19c7dc09d185e3284c77fe8ae245e954c098d74b
Parents: 9496748
Author: Jian He <ji...@apache.org>
Authored: Fri Jun 9 13:42:38 2017 -0700
Committer: Varun Saxena <va...@apache.org>
Committed: Tue Aug 1 04:19:24 2017 +0530

----------------------------------------------------------------------
 ...TimelineAuthenticationFilterInitializer.java | 69 +++++++++++---------
 .../reader/TimelineReaderServer.java            | 61 ++++++++++-------
 ...neReaderAuthenticationFilterInitializer.java | 53 +++++++++++++++
 .../reader/security/package-info.java           | 25 +++++++
 4 files changed, 154 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/19c7dc09/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
index 4e7c29a..06f9868e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineAuthenticationFilterInitializer.java
@@ -51,30 +51,18 @@ import java.util.Map;
 public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
 
   /**
-   * The configuration prefix of timeline HTTP authentication
+   * The configuration prefix of timeline HTTP authentication.
    */
   public static final String PREFIX = "yarn.timeline-service.http-authentication.";
 
   @VisibleForTesting
   Map<String, String> filterConfig;
 
-  /**
-   * Initializes {@link TimelineAuthenticationFilter}
-   * <p>
-   * Propagates to {@link TimelineAuthenticationFilter} configuration all YARN
-   * configuration properties prefixed with {@value #PREFIX}
-   *
-   * @param container
-   *          The filter container
-   * @param conf
-   *          Configuration for run-time parameters
-   */
-  @Override
-  public void initFilter(FilterContainer container, Configuration conf) {
+  protected void setAuthFilterConfig(Configuration conf) {
     filterConfig = new HashMap<String, String>();
 
     // setting the cookie path to root '/' so it is used for all resources.
-    filterConfig.put(TimelineAuthenticationFilter.COOKIE_PATH, "/");
+    filterConfig.put(AuthenticationFilter.COOKIE_PATH, "/");
 
     for (Map.Entry<String, String> entry : conf) {
       String name = entry.getKey();
@@ -95,6 +83,41 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
       }
     }
 
+    // Resolve _HOST into bind address
+    String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
+    String principal =
+        filterConfig.get(KerberosAuthenticationHandler.PRINCIPAL);
+    if (principal != null) {
+      try {
+        principal = SecurityUtil.getServerPrincipal(principal, bindAddress);
+      } catch (IOException ex) {
+        throw new RuntimeException("Could not resolve Kerberos principal " +
+            "name: " + ex.toString(), ex);
+      }
+      filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL,
+          principal);
+    }
+  }
+
+  protected Map<String, String> getFilterConfig() {
+    return filterConfig;
+  }
+
+  /**
+   * Initializes {@link TimelineAuthenticationFilter}
+   * <p>
+   * Propagates to {@link TimelineAuthenticationFilter} configuration all YARN
+   * configuration properties prefixed with {@value #PREFIX}
+   *
+   * @param container
+   *          The filter container
+   * @param conf
+   *          Configuration for run-time parameters
+   */
+  @Override
+  public void initFilter(FilterContainer container, Configuration conf) {
+    setAuthFilterConfig(conf);
+
     String authType = filterConfig.get(AuthenticationFilter.AUTH_TYPE);
     if (authType.equals(PseudoAuthenticationHandler.TYPE)) {
       filterConfig.put(AuthenticationFilter.AUTH_TYPE,
@@ -102,23 +125,7 @@ public class TimelineAuthenticationFilterInitializer extends FilterInitializer {
     } else if (authType.equals(KerberosAuthenticationHandler.TYPE)) {
       filterConfig.put(AuthenticationFilter.AUTH_TYPE,
           KerberosDelegationTokenAuthenticationHandler.class.getName());
-
-      // Resolve _HOST into bind address
-      String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
-      String principal =
-          filterConfig.get(KerberosAuthenticationHandler.PRINCIPAL);
-      if (principal != null) {
-        try {
-          principal = SecurityUtil.getServerPrincipal(principal, bindAddress);
-        } catch (IOException ex) {
-          throw new RuntimeException(
-              "Could not resolve Kerberos principal name: " + ex.toString(), ex);
-        }
-        filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL,
-            principal);
-      }
     }
-
     filterConfig.put(DelegationTokenAuthenticationHandler.TOKEN_KIND,
         TimelineDelegationTokenIdentifier.KIND_NAME.toString());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19c7dc09/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.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/TimelineReaderServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
index cf0fa50..1d5d6e2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
@@ -18,12 +18,11 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.reader;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
-import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER;
-
+import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.net.URI;
-import java.util.HashMap;
-import java.util.Map;
+import java.util.LinkedHashSet;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -31,8 +30,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
-import org.apache.hadoop.http.lib.StaticUserWebFilter;
 import org.apache.hadoop.security.HttpCrossOriginFilterInitializer;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -42,7 +41,9 @@ import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.timelineservice.reader.security.TimelineReaderAuthenticationFilterInitializer;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
+import org.apache.hadoop.yarn.server.util.timeline.TimelineServerUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
@@ -70,6 +71,17 @@ public class TimelineReaderServer extends CompositeService {
     if (!YarnConfiguration.timelineServiceV2Enabled(conf)) {
       throw new YarnException("timeline service v.2 is not enabled");
     }
+    InetSocketAddress bindAddr = conf.getSocketAddr(
+        YarnConfiguration.TIMELINE_SERVICE_ADDRESS,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ADDRESS,
+                YarnConfiguration.DEFAULT_TIMELINE_SERVICE_PORT);
+    // Login from keytab if security is enabled.
+    try {
+      SecurityUtil.login(conf, YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
+          YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL, bindAddr.getHostName());
+    } catch(IOException e) {
+      throw new YarnRuntimeException("Failed to login from keytab", e);
+    }
 
     TimelineReader timelineReaderStore = createTimelineReaderStore(conf);
     timelineReaderStore.init(conf);
@@ -129,36 +141,39 @@ public class TimelineReaderServer extends CompositeService {
     super.serviceStop();
   }
 
-  private void startTimelineReaderWebApp() {
-    Configuration conf = getConfig();
-    String bindAddress = WebAppUtils.getWebAppBindURL(conf,
-        YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
-        WebAppUtils.getTimelineReaderWebAppURL(conf));
-    LOG.info("Instantiating TimelineReaderWebApp at " + bindAddress);
+  protected void addFilters(Configuration conf) {
     boolean enableCorsFilter = conf.getBoolean(
         YarnConfiguration.TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED,
         YarnConfiguration.TIMELINE_SERVICE_HTTP_CROSS_ORIGIN_ENABLED_DEFAULT);
-    // setup CORS
+    // Setup CORS
     if (enableCorsFilter) {
       conf.setBoolean(HttpCrossOriginFilterInitializer.PREFIX
           + HttpCrossOriginFilterInitializer.ENABLED_SUFFIX, true);
     }
+    String initializers = conf.get("hadoop.http.filter.initializers", "");
+    Set<String> defaultInitializers = new LinkedHashSet<String>();
+    if (!initializers.contains(
+        TimelineReaderAuthenticationFilterInitializer.class.getName())) {
+      defaultInitializers.add(
+          TimelineReaderAuthenticationFilterInitializer.class.getName());
+    }
+    TimelineServerUtils.setTimelineFilters(
+        conf, initializers, defaultInitializers);
+  }
+
+  private void startTimelineReaderWebApp() {
+    Configuration conf = getConfig();
+    addFilters(conf);
+    String bindAddress = WebAppUtils.getWebAppBindURL(conf,
+        YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
+        WebAppUtils.getTimelineReaderWebAppURL(conf));
+    LOG.info("Instantiating TimelineReaderWebApp at " + bindAddress);
     try {
       HttpServer2.Builder builder = new HttpServer2.Builder()
             .setName("timeline")
             .setConf(conf)
             .addEndpoint(URI.create("http://" + bindAddress));
       readerWebServer = builder.build();
-
-      Map<String, String> options = new HashMap<>();
-      String username = conf.get(HADOOP_HTTP_STATIC_USER,
-          DEFAULT_HADOOP_HTTP_STATIC_USER);
-      options.put(HADOOP_HTTP_STATIC_USER, username);
-      HttpServer2.defineFilter(readerWebServer.getWebAppContext(),
-          "static_user_filter_timeline",
-          StaticUserWebFilter.StaticUserFilter.class.getName(),
-          options, new String[] {"/*"});
-
       readerWebServer.addJerseyResourcePackage(
           TimelineReaderWebServices.class.getPackage().getName() + ";"
               + GenericExceptionHandler.class.getPackage().getName() + ";"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19c7dc09/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderAuthenticationFilterInitializer.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/security/TimelineReaderAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderAuthenticationFilterInitializer.java
new file mode 100644
index 0000000..e0e1f4d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/TimelineReaderAuthenticationFilterInitializer.java
@@ -0,0 +1,53 @@
+/**
+ * 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.reader.security;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.FilterContainer;
+import org.apache.hadoop.security.AuthenticationWithProxyUserFilter;
+import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer;
+
+/**
+ * Filter initializer to initialize {@link AuthenticationWithProxyUserFilter}
+ * for ATSv2 timeline reader server with timeline service specific
+ * configurations.
+ */
+public class TimelineReaderAuthenticationFilterInitializer extends
+    TimelineAuthenticationFilterInitializer{
+
+  /**
+   * Initializes {@link AuthenticationWithProxyUserFilter}
+   * <p>
+   * Propagates to {@link AuthenticationWithProxyUserFilter} configuration all
+   * YARN configuration properties prefixed with
+   * {@value TimelineAuthenticationFilterInitializer#PREFIX}.
+   *
+   * @param container
+   *          The filter container
+   * @param conf
+   *          Configuration for run-time parameters
+   */
+  @Override
+  public void initFilter(FilterContainer container, Configuration conf) {
+    setAuthFilterConfig(conf);
+    container.addGlobalFilter("Timeline Reader Authentication Filter",
+        AuthenticationWithProxyUserFilter.class.getName(),
+        getFilterConfig());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19c7dc09/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/package-info.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/security/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/package-info.java
new file mode 100644
index 0000000..5888c98
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/security/package-info.java
@@ -0,0 +1,25 @@
+/**
+ * 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.server.timelineservice.reader.security contains
+ * classes to be used to support SPNEGO authentication for timeline reader.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.server.timelineservice.reader.security;
+import org.apache.hadoop.classification.InterfaceAudience;


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


[15/50] [abbrv] hadoop git commit: YARN-6342. Make TimelineV2Client's drain timeout after stop configurable (Haibo Chen via Varun Saxena)

Posted by va...@apache.org.
YARN-6342. Make TimelineV2Client's drain timeout after stop configurable (Haibo Chen via Varun Saxena)

(cherry picked from commit 7c2bc444b3d6750aafeed9b530c8e5b1bf95c1f4)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 739bf97ac3e1c7bdfd8363dc1ca9579962d42c92
Parents: 53c89d9
Author: Varun Saxena <va...@apache.org>
Authored: Fri Mar 31 02:02:57 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:45 2017 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/yarn/conf/YarnConfiguration.java    | 10 ++++++++++
 .../hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java |  9 ++++++---
 .../src/main/resources/yarn-default.xml                   |  9 +++++++++
 3 files changed, 25 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/739bf97a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index ab5bbb2..347c8fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -2110,6 +2110,16 @@ public class YarnConfiguration extends Configuration {
 
   public static final int DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE = 10;
 
+
+  /**
+   * The time period for which timeline v2 client will wait for draining
+   * leftover entities after stop.
+   */
+  public static final String TIMELINE_V2_CLIENT_DRAIN_TIME_MILLIS =
+      TIMELINE_SERVICE_CLIENT_PREFIX + "drain-entities.timeout.ms";
+  public static final long DEFAULT_TIMELINE_V2_CLIENT_DRAIN_TIME_MILLIS
+      = 2000L;
+
   // mark app-history related configs @Private as application history is going
   // to be integrated into the timeline service
   @Private

http://git-wip-us.apache.org/repos/asf/hadoop/blob/739bf97a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
index 848e238..e0e4f00 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
@@ -289,7 +289,7 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
      * Time period for which the timelineclient will wait for draining after
      * stop.
      */
-    private static final long DRAIN_TIME_PERIOD = 2000L;
+    private final long drainTimeoutPeriod;
 
     private int numberOfAsyncsToMerge;
     private final BlockingQueue<EntitiesHolder> timelineEntityQueue;
@@ -300,6 +300,9 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
       numberOfAsyncsToMerge =
           conf.getInt(YarnConfiguration.NUMBER_OF_ASYNC_ENTITIES_TO_MERGE,
               YarnConfiguration.DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE);
+      drainTimeoutPeriod = conf.getLong(
+          YarnConfiguration.TIMELINE_V2_CLIENT_DRAIN_TIME_MILLIS,
+          YarnConfiguration.DEFAULT_TIMELINE_V2_CLIENT_DRAIN_TIME_MILLIS);
     }
 
     Runnable createRunnable() {
@@ -330,7 +333,7 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
             // Try to drain the remaining entities to be published @ the max for
             // 2 seconds
             long timeTillweDrain =
-                System.currentTimeMillis() + DRAIN_TIME_PERIOD;
+                System.currentTimeMillis() + drainTimeoutPeriod;
             while (!timelineEntityQueue.isEmpty()) {
               publishWithoutBlockingOnQueue(timelineEntityQueue.poll());
               if (System.currentTimeMillis() > timeTillweDrain) {
@@ -449,7 +452,7 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
       LOG.info("Stopping TimelineClient.");
       executor.shutdownNow();
       try {
-        executor.awaitTermination(DRAIN_TIME_PERIOD, TimeUnit.MILLISECONDS);
+        executor.awaitTermination(drainTimeoutPeriod, TimeUnit.MILLISECONDS);
       } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/739bf97a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 4fd10ee..72606f0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -2091,6 +2091,15 @@
   </property>
 
   <property>
+    <description>
+    The time period for which timeline v2 client will wait for draining
+    leftover entities after stop.
+    </description>
+    <name>yarn.timeline-service.client.drain-entities.timeout.ms</name>
+    <value>2000</value>
+  </property>
+
+  <property>
     <description>Enable timeline server to recover state after starting. If
     true, then yarn.timeline-service.state-store-class must be specified.
     </description>


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


[22/50] [abbrv] hadoop git commit: YARN-6237. Move UID constant to TimelineReaderUtils (Rohith Sharma K S via Varun Saxena)

Posted by va...@apache.org.
YARN-6237. Move UID constant to TimelineReaderUtils (Rohith Sharma K S via Varun Saxena)

(cherry picked from commit ec32c4265bbf436ae61b6efc71ca33b16b70a23d)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 4846b0cc579de797f98b23e0dfe353b93f6bbd17
Parents: a391f54
Author: Varun Saxena <va...@apache.org>
Authored: Thu Mar 9 01:06:54 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:45 2017 +0530

----------------------------------------------------------------------
 .../TestTimelineReaderWebServicesHBaseStorage.java      | 12 ++++++------
 .../timelineservice/reader/TimelineReaderManager.java   | 12 ++++--------
 .../timelineservice/reader/TimelineReaderUtils.java     |  3 +++
 .../reader/TestTimelineReaderWebServices.java           |  4 ++--
 4 files changed, 15 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4846b0cc/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 6e416c8..6836cc1 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
@@ -648,7 +648,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
       List<String> listFlowUIDs = new ArrayList<String>();
       for (FlowActivityEntity entity : flowEntities) {
         String flowUID =
-            (String)entity.getInfo().get(TimelineReaderManager.UID_KEY);
+            (String) entity.getInfo().get(TimelineReaderUtils.UID_KEY);
         listFlowUIDs.add(flowUID);
         assertEquals(TimelineUIDConverter.FLOW_UID.encodeUID(
             new TimelineReaderContext(entity.getCluster(), entity.getUser(),
@@ -672,7 +672,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
         assertNotNull(frEntities);
         for (FlowRunEntity entity : frEntities) {
           String flowRunUID =
-              (String)entity.getInfo().get(TimelineReaderManager.UID_KEY);
+              (String) entity.getInfo().get(TimelineReaderUtils.UID_KEY);
           listFlowRunUIDs.add(flowRunUID);
           assertEquals(TimelineUIDConverter.FLOWRUN_UID.encodeUID(
               new TimelineReaderContext("cluster1", entity.getUser(),
@@ -704,7 +704,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
         assertNotNull(appEntities);
         for (TimelineEntity entity : appEntities) {
           String appUID =
-              (String)entity.getInfo().get(TimelineReaderManager.UID_KEY);
+              (String) entity.getInfo().get(TimelineReaderUtils.UID_KEY);
           listAppUIDs.add(appUID);
           assertEquals(TimelineUIDConverter.APPLICATION_UID.encodeUID(
               new TimelineReaderContext(context.getClusterId(),
@@ -737,7 +737,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
         assertNotNull(entities);
         for (TimelineEntity entity : entities) {
           String entityUID =
-              (String)entity.getInfo().get(TimelineReaderManager.UID_KEY);
+              (String) entity.getInfo().get(TimelineReaderUtils.UID_KEY);
           listEntityUIDs.add(entityUID);
           assertEquals(TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(
               new TimelineReaderContext(context.getClusterId(),
@@ -818,7 +818,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
         assertNotNull(entity.getInfo());
         assertEquals(2, entity.getInfo().size());
         String uid =
-            (String) entity.getInfo().get(TimelineReaderManager.UID_KEY);
+            (String) entity.getInfo().get(TimelineReaderUtils.UID_KEY);
         assertNotNull(uid);
         assertTrue(uid.equals(appUIDWithFlowInfo + "!type1!0!entity1")
             || uid.equals(appUIDWithFlowInfo + "!type1!0!entity2"));
@@ -846,7 +846,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
         assertNotNull(entity.getInfo());
         assertEquals(2, entity.getInfo().size());
         String uid =
-            (String) entity.getInfo().get(TimelineReaderManager.UID_KEY);
+            (String) entity.getInfo().get(TimelineReaderUtils.UID_KEY);
         assertNotNull(uid);
         assertTrue(uid.equals(appUIDWithoutFlowInfo + "!type1!0!entity1")
             || uid.equals(appUIDWithoutFlowInfo + "!type1!0!entity2"));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4846b0cc/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 66e4cbf..ee827da 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
@@ -32,8 +32,6 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * This class wraps over the timeline reader store implementation. It does some
  * non trivial manipulation of the timeline data before or after getting
@@ -43,8 +41,6 @@ import com.google.common.annotations.VisibleForTesting;
 @Unstable
 public class TimelineReaderManager extends AbstractService {
 
-  @VisibleForTesting
-  public static final String UID_KEY = "UID";
   private TimelineReader reader;
 
   public TimelineReaderManager(TimelineReader timelineReader) {
@@ -94,18 +90,18 @@ public class TimelineReaderManager extends AbstractService {
         FlowActivityEntity activityEntity = (FlowActivityEntity)entity;
         context.setUserId(activityEntity.getUser());
         context.setFlowName(activityEntity.getFlowName());
-        entity.setUID(UID_KEY,
+        entity.setUID(TimelineReaderUtils.UID_KEY,
             TimelineUIDConverter.FLOW_UID.encodeUID(context));
         return;
       case YARN_FLOW_RUN:
         FlowRunEntity runEntity = (FlowRunEntity)entity;
         context.setFlowRunId(runEntity.getRunId());
-        entity.setUID(UID_KEY,
+        entity.setUID(TimelineReaderUtils.UID_KEY,
             TimelineUIDConverter.FLOWRUN_UID.encodeUID(context));
         return;
       case YARN_APPLICATION:
         context.setAppId(entity.getId());
-        entity.setUID(UID_KEY,
+        entity.setUID(TimelineReaderUtils.UID_KEY,
             TimelineUIDConverter.APPLICATION_UID.encodeUID(context));
         return;
       default:
@@ -115,7 +111,7 @@ public class TimelineReaderManager extends AbstractService {
     context.setEntityType(entity.getType());
     context.setEntityIdPrefix(entity.getIdPrefix());
     context.setEntityId(entity.getId());
-    entity.setUID(UID_KEY,
+    entity.setUID(TimelineReaderUtils.UID_KEY,
         TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(context));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4846b0cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.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/TimelineReaderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java
index 8f92433..4fd8468 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderUtils.java
@@ -47,6 +47,9 @@ public final class TimelineReaderUtils {
 
   public static final String FROMID_KEY = "FROM_ID";
 
+  @VisibleForTesting
+  public static final String UID_KEY = "UID";
+
   /**
    * Split the passed string along the passed delimiter character while looking
    * for escape char to interpret the splitted parts correctly. For delimiter or

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4846b0cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.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/TestTimelineReaderWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
index 60bf6dc..915d2f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
@@ -233,7 +233,7 @@ public class TestTimelineReaderWebServices {
       assertEquals(3, entity.getConfigs().size());
       assertEquals(3, entity.getMetrics().size());
       assertTrue("UID should be present",
-          entity.getInfo().containsKey(TimelineReaderManager.UID_KEY));
+          entity.getInfo().containsKey(TimelineReaderUtils.UID_KEY));
       // Includes UID.
       assertEquals(3, entity.getInfo().size());
       // No events will be returned as events are not part of fields.
@@ -259,7 +259,7 @@ public class TestTimelineReaderWebServices {
       assertEquals(3, entity.getConfigs().size());
       assertEquals(3, entity.getMetrics().size());
       assertTrue("UID should be present",
-          entity.getInfo().containsKey(TimelineReaderManager.UID_KEY));
+          entity.getInfo().containsKey(TimelineReaderUtils.UID_KEY));
       // Includes UID.
       assertEquals(3, entity.getInfo().size());
       assertEquals(2, entity.getEvents().size());


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


[16/50] [abbrv] hadoop git commit: YARN-6357. Implement putEntitiesAsync API in TimelineCollector (Haibo Chen via Varun Saxena)

Posted by va...@apache.org.
YARN-6357. Implement putEntitiesAsync API in TimelineCollector (Haibo Chen via Varun Saxena)

(cherry picked from commit 063b513b1c10987461caab3d26c8543c6e657bf7)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 53c89d9760627e7a607c16407f06bf1e2e924c45
Parents: e59486a
Author: Varun Saxena <va...@apache.org>
Authored: Wed Mar 29 03:48:03 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:45 2017 +0530

----------------------------------------------------------------------
 .../collector/TimelineCollector.java            | 31 ++++++++--
 .../collector/TimelineCollectorWebService.java  | 12 ++--
 .../collector/TestTimelineCollector.java        | 63 ++++++++++++++++++++
 3 files changed, 96 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c89d97/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.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/collector/TimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
index 2fc3033..353066b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
@@ -133,19 +133,35 @@ public abstract class TimelineCollector extends CompositeService {
   public TimelineWriteResponse putEntities(TimelineEntities entities,
       UserGroupInformation callerUgi) throws IOException {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("SUCCESS - TIMELINE V2 PROTOTYPE");
       LOG.debug("putEntities(entities=" + entities + ", callerUgi="
           + callerUgi + ")");
     }
-    TimelineCollectorContext context = getTimelineEntityContext();
 
+    TimelineWriteResponse response = writeTimelineEntities(entities);
+    flushBufferedTimelineEntities();
+
+    return response;
+  }
+
+  private TimelineWriteResponse writeTimelineEntities(
+      TimelineEntities entities) throws IOException {
     // Update application metrics for aggregation
     updateAggregateStatus(entities, aggregationGroups,
         getEntityTypesSkipAggregation());
 
+    final TimelineCollectorContext context = getTimelineEntityContext();
     return writer.write(context.getClusterId(), context.getUserId(),
-        context.getFlowName(), context.getFlowVersion(), context.getFlowRunId(),
-        context.getAppId(), entities);
+        context.getFlowName(), context.getFlowVersion(),
+        context.getFlowRunId(), context.getAppId(), entities);
+  }
+
+  /**
+   * Flush buffered timeline entities, if any.
+   * @throws IOException if there is any exception encountered while
+   *      flushing buffered entities.
+   */
+  private void flushBufferedTimelineEntities() throws IOException {
+    writer.flush();
   }
 
   /**
@@ -158,14 +174,17 @@ public abstract class TimelineCollector extends CompositeService {
    *
    * @param entities entities to post
    * @param callerUgi the caller UGI
+   * @throws IOException if there is any exception encounted while putting
+   *     entities.
    */
   public void putEntitiesAsync(TimelineEntities entities,
-      UserGroupInformation callerUgi) {
-    // TODO implement
+      UserGroupInformation callerUgi) throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("putEntitiesAsync(entities=" + entities + ", callerUgi=" +
           callerUgi + ")");
     }
+
+    writeTimelineEntities(entities);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c89d97/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.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/collector/TimelineCollectorWebService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
index 29ef1f8..806a85b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorWebService.java
@@ -150,9 +150,6 @@ public class TimelineCollectorWebService {
       throw new ForbiddenException(msg);
     }
 
-    // TODO how to express async posts and handle them
-    boolean isAsync = async != null && async.trim().equalsIgnoreCase("true");
-
     try {
       ApplicationId appID = parseApplicationId(appId);
       if (appID == null) {
@@ -167,7 +164,14 @@ public class TimelineCollectorWebService {
         throw new NotFoundException(); // different exception?
       }
 
-      collector.putEntities(processTimelineEntities(entities), callerUgi);
+      boolean isAsync = async != null && async.trim().equalsIgnoreCase("true");
+      if (isAsync) {
+        collector.putEntitiesAsync(
+            processTimelineEntities(entities), callerUgi);
+      } else {
+        collector.putEntities(processTimelineEntities(entities), callerUgi);
+      }
+
       return Response.ok().build();
     } catch (Exception e) {
       LOG.error("Error putting entities", e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53c89d97/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.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/collector/TestTimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
index 5b4dc50..a55f227 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
@@ -18,17 +18,27 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.collector;
 
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 import org.junit.Test;
 
+import java.io.IOException;
 import java.util.HashSet;
 import java.util.Set;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 
 public class TestTimelineCollector {
 
@@ -124,4 +134,57 @@ public class TestTimelineCollector {
     }
 
   }
+
+  /**
+   * Test TimelineCollector's interaction with TimelineWriter upon
+   * putEntity() calls.
+   */
+  @Test
+  public void testPutEntity() throws IOException {
+    TimelineWriter writer = mock(TimelineWriter.class);
+    TimelineCollector collector = new TimelineCollectorForTest(writer);
+
+    TimelineEntities entities = generateTestEntities(1, 1);
+    collector.putEntities(
+        entities, UserGroupInformation.createRemoteUser("test-user"));
+
+    verify(writer, times(1)).write(
+        anyString(), anyString(), anyString(), anyString(), anyLong(),
+        anyString(), any(TimelineEntities.class));
+    verify(writer, times(1)).flush();
+  }
+
+  /**
+   * Test TimelineCollector's interaction with TimelineWriter upon
+   * putEntityAsync() calls.
+   */
+  @Test
+  public void testPutEntityAsync() throws IOException {
+    TimelineWriter writer = mock(TimelineWriter.class);
+    TimelineCollector collector = new TimelineCollectorForTest(writer);
+
+    TimelineEntities entities = generateTestEntities(1, 1);
+    collector.putEntitiesAsync(
+        entities, UserGroupInformation.createRemoteUser("test-user"));
+
+    verify(writer, times(1)).write(
+        anyString(), anyString(), anyString(), anyString(), anyLong(),
+        anyString(), any(TimelineEntities.class));
+    verify(writer, never()).flush();
+  }
+
+  private static class TimelineCollectorForTest extends TimelineCollector {
+    private final TimelineCollectorContext context =
+        new TimelineCollectorContext();
+
+    TimelineCollectorForTest(TimelineWriter writer) {
+      super("TimelineCollectorForTest");
+      setWriter(writer);
+    }
+
+    @Override
+    public TimelineCollectorContext getTimelineEntityContext() {
+      return context;
+    }
+  }
 }


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


[39/50] [abbrv] hadoop git commit: YARN-6733. Add table for storing sub-application entities. Contributed by Vrushali C.

Posted by va...@apache.org.
YARN-6733. Add table for storing sub-application entities. Contributed by Vrushali C.

(cherry picked from commit 416312e3a7f0f83cc1c8549640c3ba75aaa3d887)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 1ee2a98b0f10ef29d7d19bb00c2048426ff5ce99
Parents: 50f681e
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Tue Jul 25 15:25:21 2017 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Tue Jul 25 15:28:56 2017 +0530

----------------------------------------------------------------------
 .../storage/TimelineSchemaCreator.java          |  44 +++
 .../storage/application/ApplicationTable.java   |   2 +-
 .../storage/entity/EntityRowKey.java            |   6 +-
 .../subapplication/SubApplicationColumn.java    | 108 +++++++
 .../SubApplicationColumnFamily.java             |  68 +++++
 .../SubApplicationColumnPrefix.java             | 250 ++++++++++++++++
 .../subapplication/SubApplicationRowKey.java    | 290 +++++++++++++++++++
 .../SubApplicationRowKeyPrefix.java             |  89 ++++++
 .../subapplication/SubApplicationTable.java     | 174 +++++++++++
 .../storage/subapplication/package-info.java    |  28 ++
 .../storage/common/TestKeyConverters.java       |   4 +
 .../storage/common/TestRowKeys.java             |  26 ++
 .../storage/common/TestRowKeysAsString.java     |  29 ++
 13 files changed, 1114 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.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/TimelineSchemaCreator.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/TimelineSchemaCreator.java
index f93c977..f50309e 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/TimelineSchemaCreator.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/TimelineSchemaCreator.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelin
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationTable;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -62,7 +63,9 @@ public final class TimelineSchemaCreator {
   private static final Log LOG = LogFactory.getLog(TimelineSchemaCreator.class);
   private static final String SKIP_EXISTING_TABLE_OPTION_SHORT = "s";
   private static final String APP_METRICS_TTL_OPTION_SHORT = "ma";
+  private static final String SUB_APP_METRICS_TTL_OPTION_SHORT = "msa";
   private static final String APP_TABLE_NAME_SHORT = "a";
+  private static final String SUB_APP_TABLE_NAME_SHORT = "sa";
   private static final String APP_TO_FLOW_TABLE_NAME_SHORT = "a2f";
   private static final String ENTITY_METRICS_TTL_OPTION_SHORT = "me";
   private static final String ENTITY_TABLE_NAME_SHORT = "e";
@@ -120,6 +123,21 @@ public final class TimelineSchemaCreator {
         new ApplicationTable().setMetricsTTL(appMetricsTTL, hbaseConf);
       }
 
+      // Grab the subApplicationTableName argument
+      String subApplicationTableName = commandLine.getOptionValue(
+          SUB_APP_TABLE_NAME_SHORT);
+      if (StringUtils.isNotBlank(subApplicationTableName)) {
+        hbaseConf.set(SubApplicationTable.TABLE_NAME_CONF_NAME,
+            subApplicationTableName);
+      }
+      // Grab the subApplication metrics TTL
+      String subApplicationTableMetricsTTL = commandLine
+          .getOptionValue(SUB_APP_METRICS_TTL_OPTION_SHORT);
+      if (StringUtils.isNotBlank(subApplicationTableMetricsTTL)) {
+        int subAppMetricsTTL = Integer.parseInt(subApplicationTableMetricsTTL);
+        new SubApplicationTable().setMetricsTTL(subAppMetricsTTL, hbaseConf);
+      }
+
       // create all table schemas in hbase
       final boolean skipExisting = commandLine.hasOption(
           SKIP_EXISTING_TABLE_OPTION_SHORT);
@@ -181,6 +199,18 @@ public final class TimelineSchemaCreator {
     o.setRequired(false);
     options.addOption(o);
 
+    o = new Option(SUB_APP_TABLE_NAME_SHORT, "subApplicationTableName", true,
+        "subApplication table name");
+    o.setArgName("subApplicationTableName");
+    o.setRequired(false);
+    options.addOption(o);
+
+    o = new Option(SUB_APP_METRICS_TTL_OPTION_SHORT, "subApplicationMetricsTTL",
+        true, "TTL for metrics column family");
+    o.setArgName("subApplicationMetricsTTL");
+    o.setRequired(false);
+    options.addOption(o);
+
     // Options without an argument
     // No need to set arg name since we do not need an argument here
     o = new Option(SKIP_EXISTING_TABLE_OPTION_SHORT, "skipExistingTable",
@@ -219,6 +249,11 @@ public final class TimelineSchemaCreator {
         " The name of the Application table\n");
     usage.append("[-applicationMetricsTTL <Application Table Metrics TTL>]" +
         " TTL for metrics in the Application table\n");
+    usage.append("[-subApplicationTableName <SubApplication Table Name>]" +
+        " The name of the SubApplication table\n");
+    usage.append("[-subApplicationMetricsTTL " +
+        " <SubApplication Table Metrics TTL>]" +
+        " TTL for metrics in the SubApplication table\n");
     usage.append("[-skipExistingTable] Whether to skip existing" +
         " hbase tables\n");
     System.out.println(usage.toString());
@@ -311,6 +346,15 @@ public final class TimelineSchemaCreator {
           throw e;
         }
       }
+      try {
+        new SubApplicationTable().createTable(admin, hbaseConf);
+      } catch (IOException e) {
+        if (skipExisting) {
+          LOG.warn("Skip and continue on: " + e.getMessage());
+        } else {
+          throw e;
+        }
+      }
     } finally {
       if (conn != null) {
         conn.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.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/application/ApplicationTable.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/application/ApplicationTable.java
index a02f768..3c21eab 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/application/ApplicationTable.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/application/ApplicationTable.java
@@ -70,7 +70,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBas
 public class ApplicationTable extends BaseTable<ApplicationTable> {
   /** application prefix. */
   private static final String PREFIX =
-      YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".application";
+      YarnConfiguration.TIMELINE_SERVICE_PREFIX + "application";
 
   /** config param name that specifies the application table name. */
   public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.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/entity/EntityRowKey.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/entity/EntityRowKey.java
index 7bf02f2..b85a9b0 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/entity/EntityRowKey.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/entity/EntityRowKey.java
@@ -209,18 +209,18 @@ public class EntityRowKey {
             Separator.EMPTY_BYTES);
       }
 
-      byte[] enitityIdPrefix = Bytes.toBytes(rowKey.getEntityIdPrefix());
+      byte[] entityIdPrefix = Bytes.toBytes(rowKey.getEntityIdPrefix());
 
       if (rowKey.getEntityId() == null) {
         return Separator.QUALIFIERS.join(first, second, third, entityType,
-            enitityIdPrefix, Separator.EMPTY_BYTES);
+            entityIdPrefix, Separator.EMPTY_BYTES);
       }
 
       byte[] entityId = Separator.encode(rowKey.getEntityId(), Separator.SPACE,
           Separator.TAB, Separator.QUALIFIERS);
 
       byte[] fourth =
-          Separator.QUALIFIERS.join(entityType, enitityIdPrefix, entityId);
+          Separator.QUALIFIERS.join(entityType, entityIdPrefix, entityId);
 
       return Separator.QUALIFIERS.join(first, second, third, fourth);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumn.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/subapplication/SubApplicationColumn.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/subapplication/SubApplicationColumn.java
new file mode 100644
index 0000000..46b0cc9
--- /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/subapplication/SubApplicationColumn.java
@@ -0,0 +1,108 @@
+/**
+ * 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.subapplication;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+/**
+ * Identifies fully qualified columns for the {@link SubApplicationTable}.
+ */
+public enum SubApplicationColumn implements Column<SubApplicationTable> {
+
+  /**
+   * Identifier for the sub application.
+   */
+  ID(SubApplicationColumnFamily.INFO, "id"),
+
+  /**
+   * The type of sub application.
+   */
+  TYPE(SubApplicationColumnFamily.INFO, "type"),
+
+  /**
+   * When the sub application was created.
+   */
+  CREATED_TIME(SubApplicationColumnFamily.INFO, "created_time",
+      new LongConverter()),
+
+  /**
+   * The version of the flow that this sub application belongs to.
+   */
+  FLOW_VERSION(SubApplicationColumnFamily.INFO, "flow_version");
+
+  private final ColumnHelper<SubApplicationTable> column;
+  private final ColumnFamily<SubApplicationTable> columnFamily;
+  private final String columnQualifier;
+  private final byte[] columnQualifierBytes;
+
+  SubApplicationColumn(ColumnFamily<SubApplicationTable> columnFamily,
+      String columnQualifier) {
+    this(columnFamily, columnQualifier, GenericConverter.getInstance());
+  }
+
+  SubApplicationColumn(ColumnFamily<SubApplicationTable> columnFamily,
+      String columnQualifier, ValueConverter converter) {
+    this.columnFamily = columnFamily;
+    this.columnQualifier = columnQualifier;
+    // Future-proof by ensuring the right column prefix hygiene.
+    this.columnQualifierBytes =
+        Bytes.toBytes(Separator.SPACE.encode(columnQualifier));
+    this.column = new ColumnHelper<SubApplicationTable>(columnFamily,
+        converter);
+  }
+
+
+  public void store(byte[] rowKey,
+      TypedBufferedMutator<SubApplicationTable> tableMutator, Long timestamp,
+      Object inputValue, Attribute... attributes) throws IOException {
+    column.store(rowKey, tableMutator, columnQualifierBytes, timestamp,
+        inputValue, attributes);
+  }
+
+  public Object readResult(Result result) throws IOException {
+    return column.readResult(result, columnQualifierBytes);
+  }
+
+  @Override
+  public byte[] getColumnQualifierBytes() {
+    return columnQualifierBytes.clone();
+  }
+
+  @Override
+  public byte[] getColumnFamilyBytes() {
+    return columnFamily.getBytes();
+  }
+
+  @Override
+  public ValueConverter getValueConverter() {
+    return column.getValueConverter();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnFamily.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/subapplication/SubApplicationColumnFamily.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/subapplication/SubApplicationColumnFamily.java
new file mode 100644
index 0000000..1d7f8fd
--- /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/subapplication/SubApplicationColumnFamily.java
@@ -0,0 +1,68 @@
+/**
+ * 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.subapplication;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+
+/**
+ * Represents the sub application table column families.
+ */
+public enum SubApplicationColumnFamily
+    implements ColumnFamily<SubApplicationTable> {
+
+  /**
+   * Info column family houses known columns, specifically ones included in
+   * columnfamily filters.
+   */
+  INFO("i"),
+
+  /**
+   * Configurations are in a separate column family for two reasons:
+   * a) the size of the config values can be very large and
+   * b) we expect that config values
+   * are often separately accessed from other metrics and info columns.
+   */
+  CONFIGS("c"),
+
+  /**
+   * Metrics have a separate column family, because they have a separate TTL.
+   */
+  METRICS("m");
+
+  /**
+   * Byte representation of this column family.
+   */
+  private final byte[] bytes;
+
+  /**
+   * @param value
+   *          create a column family with this name. Must be lower case and
+   *          without spaces.
+   */
+  SubApplicationColumnFamily(String value) {
+    // column families should be lower case and not contain any spaces.
+    this.bytes = Bytes.toBytes(Separator.SPACE.encode(value));
+  }
+
+  public byte[] getBytes() {
+    return Bytes.copy(bytes);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationColumnPrefix.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/subapplication/SubApplicationColumnPrefix.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/subapplication/SubApplicationColumnPrefix.java
new file mode 100644
index 0000000..06ecced
--- /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/subapplication/SubApplicationColumnPrefix.java
@@ -0,0 +1,250 @@
+/**
+ * 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.subapplication;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.NavigableMap;
+
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+/**
+ * Identifies partially qualified columns for the sub app table.
+ */
+public enum SubApplicationColumnPrefix
+    implements ColumnPrefix<SubApplicationTable> {
+
+  /**
+   * To store TimelineEntity getIsRelatedToEntities values.
+   */
+  IS_RELATED_TO(SubApplicationColumnFamily.INFO, "s"),
+
+  /**
+   * To store TimelineEntity getRelatesToEntities values.
+   */
+  RELATES_TO(SubApplicationColumnFamily.INFO, "r"),
+
+  /**
+   * To store TimelineEntity info values.
+   */
+  INFO(SubApplicationColumnFamily.INFO, "i"),
+
+  /**
+   * Lifecycle events for an entity.
+   */
+  EVENT(SubApplicationColumnFamily.INFO, "e", true),
+
+  /**
+   * Config column stores configuration with config key as the column name.
+   */
+  CONFIG(SubApplicationColumnFamily.CONFIGS, null),
+
+  /**
+   * Metrics are stored with the metric name as the column name.
+   */
+  METRIC(SubApplicationColumnFamily.METRICS, null, new LongConverter());
+
+  private final ColumnHelper<SubApplicationTable> column;
+  private final ColumnFamily<SubApplicationTable> columnFamily;
+
+  /**
+   * Can be null for those cases where the provided column qualifier is the
+   * entire column name.
+   */
+  private final String columnPrefix;
+  private final byte[] columnPrefixBytes;
+
+  /**
+   * Private constructor, meant to be used by the enum definition.
+   *
+   * @param columnFamily that this column is stored in.
+   * @param columnPrefix for this column.
+   */
+  SubApplicationColumnPrefix(ColumnFamily<SubApplicationTable> columnFamily,
+      String columnPrefix) {
+    this(columnFamily, columnPrefix, false, GenericConverter.getInstance());
+  }
+
+  SubApplicationColumnPrefix(ColumnFamily<SubApplicationTable> columnFamily,
+      String columnPrefix, boolean compondColQual) {
+    this(columnFamily, columnPrefix, compondColQual,
+        GenericConverter.getInstance());
+  }
+
+  SubApplicationColumnPrefix(ColumnFamily<SubApplicationTable> columnFamily,
+      String columnPrefix, ValueConverter converter) {
+    this(columnFamily, columnPrefix, false, converter);
+  }
+
+  /**
+   * Private constructor, meant to be used by the enum definition.
+   *
+   * @param columnFamily that this column is stored in.
+   * @param columnPrefix for this column.
+   * @param converter used to encode/decode values to be stored in HBase for
+   * this column prefix.
+   */
+  SubApplicationColumnPrefix(ColumnFamily<SubApplicationTable> columnFamily,
+      String columnPrefix, boolean compondColQual, ValueConverter converter) {
+    column = new ColumnHelper<SubApplicationTable>(columnFamily, converter);
+    this.columnFamily = columnFamily;
+    this.columnPrefix = columnPrefix;
+    if (columnPrefix == null) {
+      this.columnPrefixBytes = null;
+    } else {
+      // Future-proof by ensuring the right column prefix hygiene.
+      this.columnPrefixBytes =
+          Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
+    }
+  }
+
+  /**
+   * @return the column name value
+   */
+  public String getColumnPrefix() {
+    return columnPrefix;
+  }
+
+  @Override
+  public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
+    return ColumnHelper.getColumnQualifier(
+        this.columnPrefixBytes, qualifierPrefix);
+  }
+
+  @Override
+  public byte[] getColumnPrefixBytes(String qualifierPrefix) {
+    return ColumnHelper.getColumnQualifier(
+        this.columnPrefixBytes, qualifierPrefix);
+  }
+
+  @Override
+  public byte[] getColumnFamilyBytes() {
+    return columnFamily.getBytes();
+  }
+
+  @Override
+  public ValueConverter getValueConverter() {
+    return column.getValueConverter();
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
+   * #store(byte[],
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.
+   * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[])
+   */
+  public void store(byte[] rowKey,
+      TypedBufferedMutator<SubApplicationTable> tableMutator, String qualifier,
+      Long timestamp, Object inputValue, Attribute... attributes)
+      throws IOException {
+
+    // Null check
+    if (qualifier == null) {
+      throw new IOException("Cannot store column with null qualifier in "
+          + tableMutator.getName().getNameAsString());
+    }
+
+    byte[] columnQualifier = getColumnPrefixBytes(qualifier);
+
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
+        attributes);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
+   * #store(byte[],
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.
+   * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object)
+   */
+  public void store(byte[] rowKey,
+      TypedBufferedMutator<SubApplicationTable> tableMutator, byte[] qualifier,
+      Long timestamp, Object inputValue, Attribute... attributes)
+      throws IOException {
+
+    // Null check
+    if (qualifier == null) {
+      throw new IOException("Cannot store column with null qualifier in "
+          + tableMutator.getName().getNameAsString());
+    }
+
+    byte[] columnQualifier = getColumnPrefixBytes(qualifier);
+
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
+        attributes);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
+   * #readResult(org.apache.hadoop.hbase.client.Result, java.lang.String)
+   */
+  public Object readResult(Result result, String qualifier) throws IOException {
+    byte[] columnQualifier =
+        ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
+    return column.readResult(result, columnQualifier);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
+   * #readResults(org.apache.hadoop.hbase.client.Result,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
+   */
+  public <K> Map<K, Object> readResults(Result result,
+      KeyConverter<K> keyConverter) throws IOException {
+    return column.readResults(result, columnPrefixBytes, keyConverter);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
+   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
+   */
+  public <K, V> NavigableMap<K, NavigableMap<Long, V>>
+      readResultsWithTimestamps(Result result, KeyConverter<K> keyConverter)
+      throws IOException {
+    return column.readResultsWithTimestamps(result, columnPrefixBytes,
+        keyConverter);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKey.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/subapplication/SubApplicationRowKey.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/subapplication/SubApplicationRowKey.java
new file mode 100644
index 0000000..fb1f774
--- /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/subapplication/SubApplicationRowKey.java
@@ -0,0 +1,290 @@
+/**
+ * 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.subapplication;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverterToString;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+
+/**
+ * Represents a rowkey for the sub app table.
+ */
+public class SubApplicationRowKey {
+  private final String subAppUserId;
+  private final String clusterId;
+  private final String entityType;
+  private final Long entityIdPrefix;
+  private final String entityId;
+  private final String userId;
+  private final SubApplicationRowKeyConverter subAppRowKeyConverter =
+      new SubApplicationRowKeyConverter();
+
+  public SubApplicationRowKey(String subAppUserId, String clusterId,
+      String entityType, Long entityIdPrefix, String entityId, String userId) {
+    this.subAppUserId = subAppUserId;
+    this.clusterId = clusterId;
+    this.entityType = entityType;
+    this.entityIdPrefix = entityIdPrefix;
+    this.entityId = entityId;
+    this.userId = userId;
+  }
+
+  public String getClusterId() {
+    return clusterId;
+  }
+
+  public String getSubAppUserId() {
+    return subAppUserId;
+  }
+
+  public String getEntityType() {
+    return entityType;
+  }
+
+  public String getEntityId() {
+    return entityId;
+  }
+
+  public Long getEntityIdPrefix() {
+    return entityIdPrefix;
+  }
+
+  public String getUserId() {
+    return userId;
+  }
+
+  /**
+   * Constructs a row key for the sub app table as follows:
+   * {@code subAppUserId!clusterId!entityType
+   * !entityPrefix!entityId!userId}.
+   * Typically used while querying a specific sub app.
+   *
+   * subAppUserId is usually the doAsUser.
+   * userId is the yarn user that the AM runs as.
+   *
+   * @return byte array with the row key.
+   */
+  public byte[] getRowKey() {
+    return subAppRowKeyConverter.encode(this);
+  }
+
+  /**
+   * Given the raw row key as bytes, returns the row key as an object.
+   *
+   * @param rowKey byte representation of row key.
+   * @return An <cite>SubApplicationRowKey</cite> object.
+   */
+  public static SubApplicationRowKey parseRowKey(byte[] rowKey) {
+    return new SubApplicationRowKeyConverter().decode(rowKey);
+  }
+
+  /**
+   * Constructs a row key for the sub app table as follows:
+   * <p>
+   * {@code subAppUserId!clusterId!
+   * entityType!entityIdPrefix!entityId!userId}.
+   *
+   * subAppUserId is usually the doAsUser.
+   * userId is the yarn user that that the AM runs as.
+   *
+   * </p>
+   *
+   * @return String representation of row key.
+   */
+  public String getRowKeyAsString() {
+    return subAppRowKeyConverter.encodeAsString(this);
+  }
+
+  /**
+   * Given the encoded row key as string, returns the row key as an object.
+   *
+   * @param encodedRowKey String representation of row key.
+   * @return A <cite>SubApplicationRowKey</cite> object.
+   */
+  public static SubApplicationRowKey parseRowKeyFromString(
+      String encodedRowKey) {
+    return new SubApplicationRowKeyConverter().decodeFromString(encodedRowKey);
+  }
+
+  /**
+   * Encodes and decodes row key for sub app table.
+   * The row key is of the form :
+   * subAppUserId!clusterId!flowRunId!appId!entityType!entityId!userId
+   *
+   * subAppUserId is usually the doAsUser.
+   * userId is the yarn user that the AM runs as.
+   *
+   * <p>
+   */
+  final private static class SubApplicationRowKeyConverter
+      implements KeyConverter<SubApplicationRowKey>,
+      KeyConverterToString<SubApplicationRowKey> {
+
+    private SubApplicationRowKeyConverter() {
+    }
+
+    /**
+     * sub app row key is of the form
+     * subAppUserId!clusterId!entityType!entityPrefix!entityId!userId
+     * w. each segment separated by !.
+     *
+     * subAppUserId is usually the doAsUser.
+     * userId is the yarn user that the AM runs as.
+     *
+     * The sizes below indicate sizes of each one of these
+     * segments in sequence. clusterId, subAppUserId, entityType,
+     * entityId and userId are strings.
+     * entity prefix is a long hence 8 bytes in size. Strings are
+     * variable in size (i.e. end whenever separator is encountered).
+     * This is used while decoding and helps in determining where to split.
+     */
+    private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE,
+        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE};
+
+    /*
+     * (non-Javadoc)
+     *
+     * Encodes SubApplicationRowKey object into a byte array with each
+     * component/field in SubApplicationRowKey separated by
+     * Separator#QUALIFIERS.
+     * This leads to an sub app table row key of the form
+     * subAppUserId!clusterId!entityType!entityPrefix!entityId!userId
+     *
+     * subAppUserId is usually the doAsUser.
+     * userId is the yarn user that the AM runs as.
+     *
+     * If entityType in passed SubApplicationRowKey object is null (and the
+     * fields preceding it are not null i.e. clusterId, subAppUserId), this
+     * returns a row key prefix of the form subAppUserId!clusterId!
+     * If entityId in SubApplicationRowKey is null
+     * (other components are not null), this returns a row key prefix
+     * of the form subAppUserId!clusterId!entityType!
+     *
+     * @see org.apache.hadoop.yarn.server.timelineservice.storage.common
+     * .KeyConverter#encode(java.lang.Object)
+     */
+    @Override
+    public byte[] encode(SubApplicationRowKey rowKey) {
+      byte[] subAppUser = Separator.encode(rowKey.getSubAppUserId(),
+          Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
+      byte[] cluster = Separator.encode(rowKey.getClusterId(), Separator.SPACE,
+          Separator.TAB, Separator.QUALIFIERS);
+      byte[] first = Separator.QUALIFIERS.join(subAppUser, cluster);
+      if (rowKey.getEntityType() == null) {
+        return first;
+      }
+      byte[] entityType = Separator.encode(rowKey.getEntityType(),
+          Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
+
+      if (rowKey.getEntityIdPrefix() == null) {
+        return Separator.QUALIFIERS.join(first, entityType,
+            Separator.EMPTY_BYTES);
+      }
+
+      byte[] entityIdPrefix = Bytes.toBytes(rowKey.getEntityIdPrefix());
+
+      if (rowKey.getEntityId() == null) {
+        return Separator.QUALIFIERS.join(first, entityType, entityIdPrefix,
+            Separator.EMPTY_BYTES);
+      }
+
+      byte[] entityId = Separator.encode(rowKey.getEntityId(), Separator.SPACE,
+          Separator.TAB, Separator.QUALIFIERS);
+
+      byte[] userId = Separator.encode(rowKey.getUserId(),
+          Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
+
+      byte[] second = Separator.QUALIFIERS.join(entityType, entityIdPrefix,
+          entityId, userId);
+
+      return Separator.QUALIFIERS.join(first, second);
+    }
+
+    /*
+     * (non-Javadoc)
+     *
+     * Decodes a sub application row key of the form
+     * subAppUserId!clusterId!entityType!entityPrefix!entityId!userId
+     *
+     * subAppUserId is usually the doAsUser.
+     * userId is the yarn user that the AM runs as.
+     *
+     * represented in byte format
+     * and converts it into an SubApplicationRowKey object.
+     *
+     * @see org.apache.hadoop.yarn.server.timelineservice.storage.common
+     * .KeyConverter#decode(byte[])
+     */
+    @Override
+    public SubApplicationRowKey decode(byte[] rowKey) {
+      byte[][] rowKeyComponents =
+          Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
+      if (rowKeyComponents.length != 6) {
+        throw new IllegalArgumentException(
+            "the row key is not valid for " + "a sub app");
+      }
+      String subAppUserId =
+          Separator.decode(Bytes.toString(rowKeyComponents[0]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[1]),
+          Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      String entityType = Separator.decode(Bytes.toString(rowKeyComponents[2]),
+          Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+
+      Long entityPrefixId = Bytes.toLong(rowKeyComponents[3]);
+
+      String entityId = Separator.decode(Bytes.toString(rowKeyComponents[4]),
+          Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      String userId =
+          Separator.decode(Bytes.toString(rowKeyComponents[5]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+
+      return new SubApplicationRowKey(subAppUserId, clusterId, entityType,
+          entityPrefixId, entityId, userId);
+    }
+
+    @Override
+    public String encodeAsString(SubApplicationRowKey key) {
+      if (key.subAppUserId == null || key.clusterId == null
+          || key.entityType == null || key.entityIdPrefix == null
+          || key.entityId == null || key.userId == null) {
+        throw new IllegalArgumentException();
+      }
+      return TimelineReaderUtils.joinAndEscapeStrings(
+          new String[] {key.subAppUserId, key.clusterId, key.entityType,
+              key.entityIdPrefix.toString(), key.entityId, key.userId});
+    }
+
+    @Override
+    public SubApplicationRowKey decodeFromString(String encodedRowKey) {
+      List<String> split = TimelineReaderUtils.split(encodedRowKey);
+      if (split == null || split.size() != 6) {
+        throw new IllegalArgumentException(
+            "Invalid row key for sub app table.");
+      }
+      Long entityIdPrefix = Long.valueOf(split.get(3));
+      return new SubApplicationRowKey(split.get(0), split.get(1),
+          split.get(2), entityIdPrefix, split.get(4), split.get(5));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKeyPrefix.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/subapplication/SubApplicationRowKeyPrefix.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/subapplication/SubApplicationRowKeyPrefix.java
new file mode 100644
index 0000000..e42c6cd
--- /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/subapplication/SubApplicationRowKeyPrefix.java
@@ -0,0 +1,89 @@
+/**
+ * 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.subapplication;
+
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
+
+/**
+ * Represents a partial rowkey without the entityId or without entityType and
+ * entityId for the sub application table.
+ *
+ */
+public class SubApplicationRowKeyPrefix extends SubApplicationRowKey
+    implements RowKeyPrefix<SubApplicationRowKey> {
+
+  /**
+   * Creates a prefix which generates the following rowKeyPrefixes for the sub
+   * application table:
+   * {@code subAppUserId!clusterId!entityType!entityPrefix!userId}.
+   *
+   * @param subAppUserId
+   *          identifying the subApp User
+   * @param clusterId
+   *          identifying the cluster
+   * @param entityType
+   *          which entity type
+   * @param entityIdPrefix
+   *          for entityId
+   * @param entityId
+   *          for an entity
+   * @param userId
+   *          for the user who runs the AM
+   *
+   * subAppUserId is usually the doAsUser.
+   * userId is the yarn user that the AM runs as.
+   *
+   */
+  public SubApplicationRowKeyPrefix(String subAppUserId, String clusterId,
+      String entityType, Long entityIdPrefix, String entityId,
+      String userId) {
+    super(subAppUserId, clusterId, entityType, entityIdPrefix, entityId,
+        userId);
+  }
+
+  /**
+   * Creates a prefix which generates the following rowKeyPrefixes for the sub
+   * application table:
+   * {@code subAppUserId!clusterId!entityType!entityPrefix!entityId!userId}.
+   *
+   * subAppUserId is usually the doAsUser.
+   * userId is the yarn user that the AM runs as.
+   *
+   * @param clusterId
+   *          identifying the cluster
+   * @param subAppUserId
+   *          identifying the sub app user
+   * @param userId
+   *          identifying the user who runs the AM
+   */
+  public SubApplicationRowKeyPrefix(String clusterId, String subAppUserId,
+      String userId) {
+    this(subAppUserId, clusterId, null, null, null, userId);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.
+   * RowKeyPrefix#getRowKeyPrefix()
+   */
+  public byte[] getRowKeyPrefix() {
+    return super.getRowKey();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationTable.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/subapplication/SubApplicationTable.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/subapplication/SubApplicationTable.java
new file mode 100644
index 0000000..334bab6
--- /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/subapplication/SubApplicationTable.java
@@ -0,0 +1,174 @@
+/**
+ * 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.subapplication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
+
+/**
+ * The sub application table has column families:
+ * info, config and metrics.
+ * Info stores information about a timeline entity object
+ * config stores configuration data of a timeline entity object
+ * metrics stores the metrics of a timeline entity object
+ *
+ * Example sub application table record:
+ *
+ * <pre>
+ * |-------------------------------------------------------------------------|
+ * |  Row          | Column Family             | Column Family| Column Family|
+ * |  key          | info                      | metrics      | config       |
+ * |-------------------------------------------------------------------------|
+ * | subAppUserId! | id:entityId               | metricId1:   | configKey1:  |
+ * | clusterId!    | type:entityType           | metricValue1 | configValue1 |
+ * | entityType!   |                           | @timestamp1  |              |
+ * | idPrefix!|    |                           |              | configKey2:  |
+ * | entityId!     | created_time:             | metricId1:   | configValue2 |
+ * | userId        | 1392993084018             | metricValue2 |              |
+ * |               |                           | @timestamp2  |              |
+ * |               | i!infoKey:                |              |              |
+ * |               | infoValue                 | metricId1:   |              |
+ * |               |                           | metricValue1 |              |
+ * |               |                           | @timestamp2  |              |
+ * |               | e!eventId=timestamp=      |              |              |
+ * |               | infoKey:                  |              |              |
+ * |               | eventInfoValue            |              |              |
+ * |               |                           |              |              |
+ * |               | r!relatesToKey:           |              |              |
+ * |               | id3=id4=id5               |              |              |
+ * |               |                           |              |              |
+ * |               | s!isRelatedToKey          |              |              |
+ * |               | id7=id9=id6               |              |              |
+ * |               |                           |              |              |
+ * |               | flowVersion:              |              |              |
+ * |               | versionValue              |              |              |
+ * |-------------------------------------------------------------------------|
+ * </pre>
+ */
+public class SubApplicationTable extends BaseTable<SubApplicationTable> {
+  /** sub app prefix. */
+  private static final String PREFIX =
+      YarnConfiguration.TIMELINE_SERVICE_PREFIX + "subapplication";
+
+  /** config param name that specifies the subapplication table name. */
+  public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name";
+
+  /**
+   * config param name that specifies the TTL for metrics column family in
+   * subapplication table.
+   */
+  private static final String METRICS_TTL_CONF_NAME = PREFIX
+      + ".table.metrics.ttl";
+
+  /**
+   * config param name that specifies max-versions for
+   * metrics column family in subapplication table.
+   */
+  private static final String METRICS_MAX_VERSIONS =
+      PREFIX + ".table.metrics.max-versions";
+
+  /** default value for subapplication table name. */
+  public static final String DEFAULT_TABLE_NAME =
+      "timelineservice.subapplication";
+
+  /** default TTL is 30 days for metrics timeseries. */
+  private static final int DEFAULT_METRICS_TTL = 2592000;
+
+  /** default max number of versions. */
+  private static final int DEFAULT_METRICS_MAX_VERSIONS = 10000;
+
+  private static final Log LOG = LogFactory.getLog(
+      SubApplicationTable.class);
+
+  public SubApplicationTable() {
+    super(TABLE_NAME_CONF_NAME, DEFAULT_TABLE_NAME);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.BaseTable#createTable
+   * (org.apache.hadoop.hbase.client.Admin,
+   * org.apache.hadoop.conf.Configuration)
+   */
+  public void createTable(Admin admin, Configuration hbaseConf)
+      throws IOException {
+
+    TableName table = getTableName(hbaseConf);
+    if (admin.tableExists(table)) {
+      // do not disable / delete existing table
+      // similar to the approach taken by map-reduce jobs when
+      // output directory exists
+      throw new IOException("Table " + table.getNameAsString()
+          + " already exists.");
+    }
+
+    HTableDescriptor subAppTableDescp = new HTableDescriptor(table);
+    HColumnDescriptor infoCF =
+        new HColumnDescriptor(SubApplicationColumnFamily.INFO.getBytes());
+    infoCF.setBloomFilterType(BloomType.ROWCOL);
+    subAppTableDescp.addFamily(infoCF);
+
+    HColumnDescriptor configCF =
+        new HColumnDescriptor(SubApplicationColumnFamily.CONFIGS.getBytes());
+    configCF.setBloomFilterType(BloomType.ROWCOL);
+    configCF.setBlockCacheEnabled(true);
+    subAppTableDescp.addFamily(configCF);
+
+    HColumnDescriptor metricsCF =
+        new HColumnDescriptor(SubApplicationColumnFamily.METRICS.getBytes());
+    subAppTableDescp.addFamily(metricsCF);
+    metricsCF.setBlockCacheEnabled(true);
+    // always keep 1 version (the latest)
+    metricsCF.setMinVersions(1);
+    metricsCF.setMaxVersions(
+        hbaseConf.getInt(METRICS_MAX_VERSIONS, DEFAULT_METRICS_MAX_VERSIONS));
+    metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME,
+        DEFAULT_METRICS_TTL));
+    subAppTableDescp.setRegionSplitPolicyClassName(
+        "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy");
+    subAppTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length",
+        TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH);
+    admin.createTable(subAppTableDescp,
+        TimelineHBaseSchemaConstants.getUsernameSplits());
+    LOG.info("Status of table creation for " + table.getNameAsString() + "="
+        + admin.tableExists(table));
+  }
+
+  /**
+   * @param metricsTTL time to live parameter for the metricss in this table.
+   * @param hbaseConf configururation in which to set the metrics TTL config
+   *          variable.
+   */
+  public void setMetricsTTL(int metricsTTL, Configuration hbaseConf) {
+    hbaseConf.setInt(METRICS_TTL_CONF_NAME, metricsTTL);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/package-info.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/subapplication/package-info.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/subapplication/package-info.java
new file mode 100644
index 0000000..52cc399
--- /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/subapplication/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * 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.subapplication
+ * contains classes related to implementation for subapplication table.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.subapplication;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java
index 58df970..1bd363f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java
@@ -26,6 +26,10 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.junit.Test;
 
+/**
+ * Unit tests for key converters for various tables' row keys.
+ *
+ */
 public class TestKeyConverters {
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
index cbd2273..4770238 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
@@ -31,9 +31,14 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey;
 import org.junit.Test;
 
 
+/**
+ * Class to test the row key structures for various tables.
+ *
+ */
 public class TestRowKeys {
 
   private final static String QUALIFIER_SEP = Separator.QUALIFIERS.getValue();
@@ -41,6 +46,7 @@ public class TestRowKeys {
       .toBytes(QUALIFIER_SEP);
   private final static String CLUSTER = "cl" + QUALIFIER_SEP + "uster";
   private final static String USER = QUALIFIER_SEP + "user";
+  private final static String SUB_APP_USER = QUALIFIER_SEP + "subAppUser";
   private final static String FLOW_NAME = "dummy_" + QUALIFIER_SEP + "flow"
       + QUALIFIER_SEP;
   private final static Long FLOW_RUN_ID;
@@ -247,4 +253,24 @@ public class TestRowKeys {
     verifyRowPrefixBytes(byteRowKeyPrefix);
   }
 
+  @Test
+  public void testSubAppRowKey() {
+    TimelineEntity entity = new TimelineEntity();
+    entity.setId("entity1");
+    entity.setType("DAG");
+    entity.setIdPrefix(54321);
+
+    byte[] byteRowKey =
+        new SubApplicationRowKey(SUB_APP_USER, CLUSTER,
+            entity.getType(), entity.getIdPrefix(),
+            entity.getId(), USER).getRowKey();
+    SubApplicationRowKey rowKey = SubApplicationRowKey.parseRowKey(byteRowKey);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(SUB_APP_USER, rowKey.getSubAppUserId());
+    assertEquals(entity.getType(), rowKey.getEntityType());
+    assertEquals(entity.getIdPrefix(), rowKey.getEntityIdPrefix().longValue());
+    assertEquals(entity.getId(), rowKey.getEntityId());
+    assertEquals(USER, rowKey.getUserId());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1ee2a98b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java
index f0ef720..148cf56 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeysAsString.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationRowKey;
 import org.junit.Test;
 
 /**
@@ -38,6 +39,9 @@ public class TestRowKeysAsString {
           + TimelineReaderUtils.DEFAULT_ESCAPE_CHAR;
   private final static String USER =
       TimelineReaderUtils.DEFAULT_ESCAPE_CHAR + "user";
+  private final static String SUB_APP_USER =
+      TimelineReaderUtils.DEFAULT_ESCAPE_CHAR + "subAppUser";
+
   private final static String FLOW_NAME =
       "dummy_" + TimelineReaderUtils.DEFAULT_DELIMITER_CHAR
           + TimelineReaderUtils.DEFAULT_ESCAPE_CHAR + "flow"
@@ -112,4 +116,29 @@ public class TestRowKeysAsString {
     assertEquals(FLOW_NAME, rowKey.getFlowName());
     assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
   }
+
+  @Test(timeout = 10000)
+  public void testSubApplicationRowKey() {
+    char del = TimelineReaderUtils.DEFAULT_DELIMITER_CHAR;
+    char esc = TimelineReaderUtils.DEFAULT_ESCAPE_CHAR;
+    String id = del + esc + "ent" + esc + del + "ity" + esc + del + esc + "id"
+        + esc + del + esc;
+    String type = "entity" + esc + del + esc + "Type";
+    TimelineEntity entity = new TimelineEntity();
+    entity.setId(id);
+    entity.setType(type);
+    entity.setIdPrefix(54321);
+
+    String rowKeyAsString = new SubApplicationRowKey(SUB_APP_USER, CLUSTER,
+        entity.getType(), entity.getIdPrefix(), entity.getId(), USER)
+            .getRowKeyAsString();
+    SubApplicationRowKey rowKey = SubApplicationRowKey
+        .parseRowKeyFromString(rowKeyAsString);
+    assertEquals(SUB_APP_USER, rowKey.getSubAppUserId());
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(entity.getType(), rowKey.getEntityType());
+    assertEquals(entity.getIdPrefix(), rowKey.getEntityIdPrefix().longValue());
+    assertEquals(entity.getId(), rowKey.getEntityId());
+    assertEquals(USER, rowKey.getUserId());
+  }
 }


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


[26/50] [abbrv] hadoop git commit: YARN-6518. Fix warnings from Spotbugs in hadoop-yarn-server-timelineservice. Contributed by Weiwei Yang.

Posted by va...@apache.org.
YARN-6518. Fix warnings from Spotbugs in hadoop-yarn-server-timelineservice. Contributed by Weiwei Yang.

(cherry picked from commit 0f1af3178eb2a38aaaa1e1a27edd409cda19a198)


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

Branch: refs/heads/YARN-5355_branch2
Commit: a23a6200e0b519df226a2287f41c26956cb4c691
Parents: 1a771bd
Author: Naganarasimha <na...@apache.org>
Authored: Mon May 1 16:49:30 2017 +0530
Committer: Naganarasimha <na...@apache.org>
Committed: Mon May 1 16:54:11 2017 +0530

----------------------------------------------------------------------
 .../storage/FileSystemTimelineReaderImpl.java   | 129 ++++++++++---------
 1 file changed, 67 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a23a6200/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.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/storage/FileSystemTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java
index 6612192..6ef01dc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java
@@ -279,69 +279,74 @@ public class FileSystemTimelineReaderImpl extends AbstractService
             }
           }
         );
-    for (File entityFile : dir.listFiles()) {
-      if (!entityFile.getName().contains(TIMELINE_SERVICE_STORAGE_EXTENSION)) {
-        continue;
-      }
-      try (BufferedReader reader =
-               new BufferedReader(
-                   new InputStreamReader(
-                       new FileInputStream(
-                           entityFile), Charset.forName("UTF-8")))) {
-        TimelineEntity entity = readEntityFromFile(reader);
-        if (!entity.getType().equals(entityType)) {
-          continue;
-        }
-        if (!isTimeInRange(entity.getCreatedTime(),
-            filters.getCreatedTimeBegin(), filters.getCreatedTimeEnd())) {
-          continue;
-        }
-        if (filters.getRelatesTo() != null &&
-            !filters.getRelatesTo().getFilterList().isEmpty() &&
-            !TimelineStorageUtils.matchRelatesTo(entity,
-            filters.getRelatesTo())) {
-          continue;
-        }
-        if (filters.getIsRelatedTo()  != null &&
-            !filters.getIsRelatedTo().getFilterList().isEmpty() &&
-            !TimelineStorageUtils.matchIsRelatedTo(entity,
-            filters.getIsRelatedTo())) {
-          continue;
-        }
-        if (filters.getInfoFilters() != null &&
-            !filters.getInfoFilters().getFilterList().isEmpty() &&
-            !TimelineStorageUtils.matchInfoFilters(entity,
-            filters.getInfoFilters())) {
-          continue;
-        }
-        if (filters.getConfigFilters() != null &&
-            !filters.getConfigFilters().getFilterList().isEmpty() &&
-            !TimelineStorageUtils.matchConfigFilters(entity,
-            filters.getConfigFilters())) {
-          continue;
-        }
-        if (filters.getMetricFilters() != null &&
-            !filters.getMetricFilters().getFilterList().isEmpty() &&
-            !TimelineStorageUtils.matchMetricFilters(entity,
-            filters.getMetricFilters())) {
-          continue;
-        }
-        if (filters.getEventFilters() != null &&
-            !filters.getEventFilters().getFilterList().isEmpty() &&
-            !TimelineStorageUtils.matchEventFilters(entity,
-            filters.getEventFilters())) {
-          continue;
-        }
-        TimelineEntity entityToBeReturned = createEntityToBeReturned(
-            entity, dataToRetrieve.getFieldsToRetrieve());
-        Set<TimelineEntity> entitiesCreatedAtSameTime =
-            sortedEntities.get(entityToBeReturned.getCreatedTime());
-        if (entitiesCreatedAtSameTime == null) {
-          entitiesCreatedAtSameTime = new HashSet<TimelineEntity>();
+    if (dir != null) {
+      File[] files = dir.listFiles();
+      if (files != null) {
+        for (File entityFile : files) {
+          if (!entityFile.getName()
+              .contains(TIMELINE_SERVICE_STORAGE_EXTENSION)) {
+            continue;
+          }
+          try (BufferedReader reader = new BufferedReader(
+              new InputStreamReader(new FileInputStream(entityFile),
+                  Charset.forName("UTF-8")))) {
+            TimelineEntity entity = readEntityFromFile(reader);
+            if (!entity.getType().equals(entityType)) {
+              continue;
+            }
+            if (!isTimeInRange(entity.getCreatedTime(),
+                filters.getCreatedTimeBegin(),
+                filters.getCreatedTimeEnd())) {
+              continue;
+            }
+            if (filters.getRelatesTo() != null &&
+                !filters.getRelatesTo().getFilterList().isEmpty() &&
+                !TimelineStorageUtils.matchRelatesTo(entity,
+                    filters.getRelatesTo())) {
+              continue;
+            }
+            if (filters.getIsRelatedTo() != null &&
+                !filters.getIsRelatedTo().getFilterList().isEmpty() &&
+                !TimelineStorageUtils.matchIsRelatedTo(entity,
+                    filters.getIsRelatedTo())) {
+              continue;
+            }
+            if (filters.getInfoFilters() != null &&
+                !filters.getInfoFilters().getFilterList().isEmpty() &&
+                !TimelineStorageUtils.matchInfoFilters(entity,
+                    filters.getInfoFilters())) {
+              continue;
+            }
+            if (filters.getConfigFilters() != null &&
+                !filters.getConfigFilters().getFilterList().isEmpty() &&
+                !TimelineStorageUtils.matchConfigFilters(entity,
+                    filters.getConfigFilters())) {
+              continue;
+            }
+            if (filters.getMetricFilters() != null &&
+                !filters.getMetricFilters().getFilterList().isEmpty() &&
+                !TimelineStorageUtils.matchMetricFilters(entity,
+                    filters.getMetricFilters())) {
+              continue;
+            }
+            if (filters.getEventFilters() != null &&
+                !filters.getEventFilters().getFilterList().isEmpty() &&
+                !TimelineStorageUtils.matchEventFilters(entity,
+                    filters.getEventFilters())) {
+              continue;
+            }
+            TimelineEntity entityToBeReturned = createEntityToBeReturned(
+                entity, dataToRetrieve.getFieldsToRetrieve());
+            Set<TimelineEntity> entitiesCreatedAtSameTime =
+                sortedEntities.get(entityToBeReturned.getCreatedTime());
+            if (entitiesCreatedAtSameTime == null) {
+              entitiesCreatedAtSameTime = new HashSet<TimelineEntity>();
+            }
+            entitiesCreatedAtSameTime.add(entityToBeReturned);
+            sortedEntities.put(entityToBeReturned.getCreatedTime(),
+                entitiesCreatedAtSameTime);
+          }
         }
-        entitiesCreatedAtSameTime.add(entityToBeReturned);
-        sortedEntities.put(
-            entityToBeReturned.getCreatedTime(), entitiesCreatedAtSameTime);
       }
     }
 


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


[02/50] [abbrv] hadoop git commit: YARN-5378. Accommodate app-id->cluster mapping (Sangjin Lee via Varun Saxena)

Posted by va...@apache.org.
YARN-5378. Accommodate app-id->cluster mapping (Sangjin Lee via Varun Saxena)

(cherry picked from commit 6baea680ba6e5df6f254ced086d6defa64fb99f0)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 44216a4660fbdef7818e5de07d238510db5dec22
Parents: 7d883f4
Author: Varun Saxena <va...@apache.org>
Authored: Tue Jan 17 20:05:47 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Tue Apr 25 23:14:28 2017 +0530

----------------------------------------------------------------------
 .../storage/HBaseTimelineWriterImpl.java        |  20 +-
 .../storage/apptoflow/AppToFlowColumn.java      | 148 -------------
 .../apptoflow/AppToFlowColumnPrefix.java        | 206 +++++++++++++++++++
 .../storage/apptoflow/AppToFlowRowKey.java      | 101 +--------
 .../storage/apptoflow/AppToFlowTable.java       |  21 +-
 .../storage/common/ColumnHelper.java            |   5 +-
 .../reader/AbstractTimelineStorageReader.java   |  39 ++--
 .../storage/common/TestRowKeys.java             |   4 +-
 8 files changed, 271 insertions(+), 273 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/44216a46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.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/storage/HBaseTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
index c1c2a5e..dfd63bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
@@ -172,9 +172,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
         FlowRunRowKey flowRunRowKey =
             new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
         if (event != null) {
-          AppToFlowRowKey appToFlowRowKey =
-              new AppToFlowRowKey(clusterId, appId);
-          onApplicationCreated(flowRunRowKey, appToFlowRowKey, appId, userId,
+          onApplicationCreated(flowRunRowKey, clusterId, appId, userId,
               flowVersion, te, event.getTimestamp());
         }
         // if it's an application entity, store metrics
@@ -193,18 +191,22 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   }
 
   private void onApplicationCreated(FlowRunRowKey flowRunRowKey,
-      AppToFlowRowKey appToFlowRowKey, String appId, String userId,
-      String flowVersion, TimelineEntity te, long appCreatedTimeStamp)
+      String clusterId, String appId, String userId, String flowVersion,
+      TimelineEntity te, long appCreatedTimeStamp)
       throws IOException {
 
     String flowName = flowRunRowKey.getFlowName();
     Long flowRunId = flowRunRowKey.getFlowRunId();
 
     // store in App to flow table
+    AppToFlowRowKey appToFlowRowKey = new AppToFlowRowKey(appId);
     byte[] rowKey = appToFlowRowKey.getRowKey();
-    AppToFlowColumn.FLOW_ID.store(rowKey, appToFlowTable, null, flowName);
-    AppToFlowColumn.FLOW_RUN_ID.store(rowKey, appToFlowTable, null, flowRunId);
-    AppToFlowColumn.USER_ID.store(rowKey, appToFlowTable, null, userId);
+    AppToFlowColumnPrefix.FLOW_NAME.store(rowKey, appToFlowTable, clusterId,
+        null, flowName);
+    AppToFlowColumnPrefix.FLOW_RUN_ID.store(rowKey, appToFlowTable, clusterId,
+        null, flowRunId);
+    AppToFlowColumnPrefix.USER_ID.store(rowKey, appToFlowTable, clusterId, null,
+        userId);
 
     // store in flow run table
     storeAppCreatedInFlowRunTable(flowRunRowKey, appId, te);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44216a46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.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/storage/apptoflow/AppToFlowColumn.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.java
deleted file mode 100644
index ff61633..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/**
- * 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.apptoflow;
-
-
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-
-import java.io.IOException;
-
-/**
- * Identifies fully qualified columns for the {@link AppToFlowTable}.
- */
-public enum AppToFlowColumn implements Column<AppToFlowTable> {
-
-  /**
-   * The flow ID.
-   */
-  FLOW_ID(AppToFlowColumnFamily.MAPPING, "flow_id"),
-
-  /**
-   * The flow run ID.
-   */
-  FLOW_RUN_ID(AppToFlowColumnFamily.MAPPING, "flow_run_id"),
-
-  /**
-   * The user.
-   */
-  USER_ID(AppToFlowColumnFamily.MAPPING, "user_id");
-
-  private final ColumnHelper<AppToFlowTable> column;
-  private final ColumnFamily<AppToFlowTable> columnFamily;
-  private final String columnQualifier;
-  private final byte[] columnQualifierBytes;
-
-  AppToFlowColumn(ColumnFamily<AppToFlowTable> columnFamily,
-      String columnQualifier) {
-    this.columnFamily = columnFamily;
-    this.columnQualifier = columnQualifier;
-    // Future-proof by ensuring the right column prefix hygiene.
-    this.columnQualifierBytes =
-        Bytes.toBytes(Separator.SPACE.encode(columnQualifier));
-    this.column = new ColumnHelper<AppToFlowTable>(columnFamily);
-  }
-
-  /**
-   * @return the column name value
-   */
-  private String getColumnQualifier() {
-    return columnQualifier;
-  }
-
-  @Override
-  public byte[] getColumnQualifierBytes() {
-    return columnQualifierBytes.clone();
-  }
-
-  public void store(byte[] rowKey,
-      TypedBufferedMutator<AppToFlowTable> tableMutator, Long timestamp,
-      Object inputValue, Attribute... attributes) throws IOException {
-    column.store(rowKey, tableMutator, columnQualifierBytes, timestamp,
-        inputValue, attributes);
-  }
-
-  @Override
-  public byte[] getColumnFamilyBytes() {
-    return columnFamily.getBytes();
-  }
-
-  @Override
-  public ValueConverter getValueConverter() {
-    return column.getValueConverter();
-  }
-
-  public Object readResult(Result result) throws IOException {
-    return column.readResult(result, columnQualifierBytes);
-  }
-
-  /**
-   * Retrieve an {@link AppToFlowColumn} given a name, or null if there is no
-   * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
-   * and only if {@code x.equals(y)} or {@code (x == y == null)}
-   *
-   * @param columnQualifier Name of the column to retrieve
-   * @return the corresponding {@link AppToFlowColumn} or null
-   */
-  public static final AppToFlowColumn columnFor(String columnQualifier) {
-
-    // Match column based on value, assume column family matches.
-    for (AppToFlowColumn ec : AppToFlowColumn.values()) {
-      // Find a match based only on name.
-      if (ec.getColumnQualifier().equals(columnQualifier)) {
-        return ec;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-
-  /**
-   * Retrieve an {@link AppToFlowColumn} given a name, or null if there is no
-   * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
-   * if and only if {@code a.equals(b) & x.equals(y)} or
-   * {@code (x == y == null)}
-   *
-   * @param columnFamily The columnFamily for which to retrieve the column.
-   * @param name Name of the column to retrieve
-   * @return the corresponding {@link AppToFlowColumn} or null if both arguments
-   *         don't match.
-   */
-  public static final AppToFlowColumn columnFor(
-      AppToFlowColumnFamily columnFamily, String name) {
-
-    for (AppToFlowColumn ec : AppToFlowColumn.values()) {
-      // Find a match based column family and on name.
-      if (ec.columnFamily.equals(columnFamily)
-          && ec.getColumnQualifier().equals(name)) {
-        return ec;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44216a46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.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/storage/apptoflow/AppToFlowColumnPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java
new file mode 100644
index 0000000..f1e4495
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnPrefix.java
@@ -0,0 +1,206 @@
+/**
+ * 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.apptoflow;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.NavigableMap;
+
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+
+/**
+ * Identifies partially qualified columns for the app-to-flow table.
+ */
+public enum AppToFlowColumnPrefix implements ColumnPrefix<AppToFlowTable> {
+
+  /**
+   * The flow name.
+   */
+  FLOW_NAME(AppToFlowColumnFamily.MAPPING, "flow_name"),
+
+  /**
+   * The flow run ID.
+   */
+  FLOW_RUN_ID(AppToFlowColumnFamily.MAPPING, "flow_run_id"),
+
+  /**
+   * The user.
+   */
+  USER_ID(AppToFlowColumnFamily.MAPPING, "user_id");
+
+  private final ColumnHelper<AppToFlowTable> column;
+  private final ColumnFamily<AppToFlowTable> columnFamily;
+  private final String columnPrefix;
+  private final byte[] columnPrefixBytes;
+
+  private AppToFlowColumnPrefix(ColumnFamily<AppToFlowTable> columnFamily,
+      String columnPrefix) {
+    this.columnFamily = columnFamily;
+    this.columnPrefix = columnPrefix;
+    if (columnPrefix == null) {
+      this.columnPrefixBytes = null;
+    } else {
+      // Future-proof by ensuring the right column prefix hygiene.
+      this.columnPrefixBytes =
+          Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
+    }
+    this.column = new ColumnHelper<AppToFlowTable>(columnFamily);
+  }
+
+  @Override
+  public byte[] getColumnPrefixBytes(String qualifierPrefix) {
+    return ColumnHelper.getColumnQualifier(
+        columnPrefixBytes, qualifierPrefix);
+  }
+
+  @Override
+  public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
+    return ColumnHelper.getColumnQualifier(
+        columnPrefixBytes, qualifierPrefix);
+  }
+
+  @Override
+  public byte[] getColumnFamilyBytes() {
+    return columnFamily.getBytes();
+  }
+
+  @Override
+  public void store(byte[] rowKey,
+      TypedBufferedMutator<AppToFlowTable> tableMutator, byte[] qualifier,
+      Long timestamp, Object inputValue, Attribute... attributes)
+      throws IOException {
+
+    // Null check
+    if (qualifier == null) {
+      throw new IOException("Cannot store column with null qualifier in "
+          + tableMutator.getName().getNameAsString());
+    }
+
+    byte[] columnQualifier = getColumnPrefixBytes(qualifier);
+
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
+        attributes);
+  }
+
+  @Override
+  public void store(byte[] rowKey,
+      TypedBufferedMutator<AppToFlowTable> tableMutator, String qualifier,
+      Long timestamp, Object inputValue, Attribute... attributes)
+      throws IOException {
+
+    // Null check
+    if (qualifier == null) {
+      throw new IOException("Cannot store column with null qualifier in "
+          + tableMutator.getName().getNameAsString());
+    }
+
+    byte[] columnQualifier = getColumnPrefixBytes(qualifier);
+
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
+        attributes);
+  }
+
+  @Override
+  public ValueConverter getValueConverter() {
+    return column.getValueConverter();
+  }
+
+  @Override
+  public Object readResult(Result result, String qualifier) throws IOException {
+    byte[] columnQualifier =
+        ColumnHelper.getColumnQualifier(columnPrefixBytes, qualifier);
+    return column.readResult(result, columnQualifier);
+  }
+
+  @Override
+  public <K> Map<K, Object> readResults(Result result,
+      KeyConverter<K> keyConverter)
+      throws IOException {
+    return column.readResults(result, columnPrefixBytes, keyConverter);
+  }
+
+  @Override
+  public <K, V> NavigableMap<K, NavigableMap<Long, V>>
+      readResultsWithTimestamps(Result result,
+      KeyConverter<K> keyConverter) throws IOException {
+    return column.readResultsWithTimestamps(result, columnPrefixBytes,
+        keyConverter);
+  }
+
+  /**
+   * Retrieve an {@link AppToFlowColumnPrefix} given a name, or null if there
+   * is no match. The following holds true: {@code columnFor(x) == columnFor(y)}
+   * if and only if {@code x.equals(y)} or {@code (x == y == null)}
+   *
+   * @param columnPrefix Name of the column to retrieve
+   * @return the corresponding {@link AppToFlowColumnPrefix} or null
+   */
+  public static final AppToFlowColumnPrefix columnFor(String columnPrefix) {
+
+    // Match column based on value, assume column family matches.
+    for (AppToFlowColumnPrefix afcp : AppToFlowColumnPrefix.values()) {
+      // Find a match based only on name.
+      if (afcp.columnPrefix.equals(columnPrefix)) {
+        return afcp;
+      }
+    }
+
+    // Default to null
+    return null;
+  }
+
+  /**
+   * Retrieve an {@link AppToFlowColumnPrefix} given a name, or null if there
+   * is no match. The following holds true:
+   * {@code columnFor(a,x) == columnFor(b,y)} if and only if
+   * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)}
+   *
+   * @param columnFamily The columnFamily for which to retrieve the column.
+   * @param columnPrefix Name of the column to retrieve
+   * @return the corresponding {@link AppToFlowColumnPrefix} or null if both
+   *         arguments don't match.
+   */
+  public static final AppToFlowColumnPrefix columnFor(
+      AppToFlowColumnFamily columnFamily, String columnPrefix) {
+
+    // TODO: needs unit test to confirm and need to update javadoc to explain
+    // null prefix case.
+
+    for (AppToFlowColumnPrefix afcp : AppToFlowColumnPrefix.values()) {
+      // Find a match based column family and on name.
+      if (afcp.columnFamily.equals(columnFamily)
+          && (((columnPrefix == null) && (afcp.columnPrefix == null)) ||
+          (afcp.columnPrefix.equals(columnPrefix)))) {
+        return afcp;
+      }
+    }
+
+    // Default to null
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44216a46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.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/storage/apptoflow/AppToFlowRowKey.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java
index 8df4407..146c475 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java
@@ -17,41 +17,32 @@
  */
 package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
 
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 
 /**
- * Represents a rowkey for the app_flow table.
+ * Represents a row key for the app_flow table, which is the app id.
  */
 public class AppToFlowRowKey {
-  private final String clusterId;
   private final String appId;
-  private final KeyConverter<AppToFlowRowKey> appToFlowRowKeyConverter =
-      new AppToFlowRowKeyConverter();
+  private final KeyConverter<String> appIdKeyConverter =
+      new AppIdKeyConverter();
 
-  public AppToFlowRowKey(String clusterId, String appId) {
-    this.clusterId = clusterId;
+  public AppToFlowRowKey(String appId) {
     this.appId = appId;
   }
 
-  public String getClusterId() {
-    return clusterId;
-  }
-
   public String getAppId() {
     return appId;
   }
 
   /**
-   * Constructs a row key prefix for the app_flow table as follows:
-   * {@code clusterId!AppId}.
+   * Constructs a row key prefix for the app_flow table.
    *
    * @return byte array with the row key
    */
   public  byte[] getRowKey() {
-    return appToFlowRowKeyConverter.encode(this);
+    return appIdKeyConverter.encode(appId);
   }
 
   /**
@@ -61,83 +52,7 @@ public class AppToFlowRowKey {
    * @return an <cite>AppToFlowRowKey</cite> object.
    */
   public static AppToFlowRowKey parseRowKey(byte[] rowKey) {
-    return new AppToFlowRowKeyConverter().decode(rowKey);
-  }
-
-  /**
-   * Encodes and decodes row key for app_flow table. The row key is of the form
-   * clusterId!appId. clusterId is a string and appId is encoded/decoded using
-   * {@link AppIdKeyConverter}.
-   * <p>
-   */
-  final private static class AppToFlowRowKeyConverter implements
-      KeyConverter<AppToFlowRowKey> {
-
-    private final KeyConverter<String> appIDKeyConverter =
-        new AppIdKeyConverter();
-
-    /**
-     * Intended for use in AppToFlowRowKey only.
-     */
-    private AppToFlowRowKeyConverter() {
-    }
-
-
-    /**
-     * App to flow row key is of the form clusterId!appId with the 2 segments
-     * separated by !. The sizes below indicate sizes of both of these segments
-     * in sequence. clusterId is a string. appId is represented as 12 bytes w.
-     * cluster Timestamp part of appid taking 8 bytes(long) and seq id taking 4
-     * bytes(int). Strings are variable in size (i.e. end whenever separator is
-     * encountered). This is used while decoding and helps in determining where
-     * to split.
-     */
-    private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE,
-        Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT };
-
-    /*
-     * (non-Javadoc)
-     *
-     * Encodes AppToFlowRowKey object into a byte array with each
-     * component/field in AppToFlowRowKey separated by Separator#QUALIFIERS.
-     * This leads to an app to flow table row key of the form clusterId!appId
-     *
-     * @see
-     * org.apache.hadoop.yarn.server.timelineservice.storage.common
-     * .KeyConverter#encode(java.lang.Object)
-     */
-    @Override
-    public byte[] encode(AppToFlowRowKey rowKey) {
-      byte[] first =
-          Separator.encode(rowKey.getClusterId(), Separator.SPACE,
-              Separator.TAB, Separator.QUALIFIERS);
-      byte[] second = appIDKeyConverter.encode(rowKey.getAppId());
-      return Separator.QUALIFIERS.join(first, second);
-    }
-
-    /*
-     * (non-Javadoc)
-     *
-     * Decodes an app to flow row key of the form clusterId!appId represented
-     * in byte format and converts it into an AppToFlowRowKey object.
-     *
-     * @see
-     * org.apache.hadoop.yarn.server.timelineservice.storage.common
-     * .KeyConverter#decode(byte[])
-     */
-    @Override
-    public AppToFlowRowKey decode(byte[] rowKey) {
-      byte[][] rowKeyComponents =
-          Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
-      if (rowKeyComponents.length != 2) {
-        throw new IllegalArgumentException("the row key is not valid for "
-            + "the app-to-flow table");
-      }
-      String clusterId =
-          Separator.decode(Bytes.toString(rowKeyComponents[0]),
-              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-      String appId = appIDKeyConverter.decode(rowKeyComponents[1]);
-      return new AppToFlowRowKey(clusterId, appId);
-    }
+    String appId = new AppIdKeyConverter().decode(rowKey);
+    return new AppToFlowRowKey(appId);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44216a46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.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/storage/apptoflow/AppToFlowTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java
index 301cf99..583ee04 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTable.java
@@ -41,21 +41,32 @@ import java.io.IOException;
  * <pre>
  * |--------------------------------------|
  * |  Row       | Column Family           |
- * |  key       | info                    |
+ * |  key       | mapping                 |
  * |--------------------------------------|
- * | clusterId! | flowName:               |
- * | AppId      | foo@daily_hive_report   |
+ * | appId      | flow_name!cluster1:     |
+ * |            | foo@daily_hive_report   |
  * |            |                         |
- * |            | flowRunId:              |
+ * |            | flow_run_id!cluster1:   |
  * |            | 1452828720457           |
  * |            |                         |
- * |            | user_id:                |
+ * |            | user_id!cluster1:       |
  * |            | admin                   |
  * |            |                         |
+ * |            | flow_name!cluster2:     |
+ * |            | bar@ad_hoc_query        |
  * |            |                         |
+ * |            | flow_run_id!cluster2:   |
+ * |            | 1452828498752           |
+ * |            |                         |
+ * |            | user_id!cluster2:       |
+ * |            | joe                     |
  * |            |                         |
  * |--------------------------------------|
  * </pre>
+ *
+ * It is possible (although unlikely) in a multi-cluster environment that there
+ * may be more than one applications for a given app id. Different clusters are
+ * recorded as different sets of columns.
  */
 public class AppToFlowTable extends BaseTable<AppToFlowTable> {
   /** app_flow prefix. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44216a46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.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/storage/common/ColumnHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java
index be55db5..b9815eb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.java
@@ -316,8 +316,9 @@ public class ColumnHelper<T> {
   /**
    * @param columnPrefixBytes The byte representation for the column prefix.
    *          Should not contain {@link Separator#QUALIFIERS}.
-   * @param qualifier for the remainder of the column. Any
-   *          {@link Separator#QUALIFIERS} will be encoded in the qualifier.
+   * @param qualifier for the remainder of the column.
+   *          {@link Separator#QUALIFIERS} is permissible in the qualifier
+   *          as it is joined only with the column prefix bytes.
    * @return fully sanitized column qualifier that is a combination of prefix
    *         and qualifier. If prefix is null, the result is simply the encoded
    *         qualifier without any separator.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44216a46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.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/storage/reader/AbstractTimelineStorageReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.java
index fcd8320..5bacf66 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/AbstractTimelineStorageReader.java
@@ -17,18 +17,18 @@
  */
 package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
-import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 
-import java.io.IOException;
-
 /**
  * The base class for reading timeline data from the HBase storage. This class
  * provides basic support to validate and augment reader context.
@@ -53,26 +53,38 @@ public abstract class AbstractTimelineStorageReader {
    * Looks up flow context from AppToFlow table.
    *
    * @param appToFlowRowKey to identify Cluster and App Ids.
+   * @param clusterId the cluster id.
    * @param hbaseConf HBase configuration.
    * @param conn HBase Connection.
    * @return flow context information.
    * @throws IOException if any problem occurs while fetching flow information.
    */
   protected FlowContext lookupFlowContext(AppToFlowRowKey appToFlowRowKey,
-      Configuration hbaseConf, Connection conn) throws IOException {
+      String clusterId, Configuration hbaseConf, Connection conn)
+      throws IOException {
     byte[] rowKey = appToFlowRowKey.getRowKey();
     Get get = new Get(rowKey);
     Result result = appToFlowTable.getResult(hbaseConf, conn, get);
     if (result != null && !result.isEmpty()) {
-      return new FlowContext(AppToFlowColumn.USER_ID.readResult(result)
-          .toString(), AppToFlowColumn.FLOW_ID.readResult(result).toString(),
-          ((Number) AppToFlowColumn.FLOW_RUN_ID.readResult(result))
-          .longValue());
+      Object flowName =
+          AppToFlowColumnPrefix.FLOW_NAME.readResult(result, clusterId);
+      Object flowRunId =
+          AppToFlowColumnPrefix.FLOW_RUN_ID.readResult(result, clusterId);
+      Object userId =
+          AppToFlowColumnPrefix.USER_ID.readResult(result, clusterId);
+      if (flowName == null || userId == null || flowRunId == null) {
+        throw new NotFoundException(
+            "Unable to find the context flow name, and flow run id, "
+            + "and user id for clusterId=" + clusterId
+            + ", appId=" + appToFlowRowKey.getAppId());
+      }
+      return new FlowContext((String)userId, (String)flowName,
+          ((Number)flowRunId).longValue());
     } else {
       throw new NotFoundException(
-          "Unable to find the context flow ID and flow run ID for clusterId="
-              + appToFlowRowKey.getClusterId() + ", appId="
-              + appToFlowRowKey.getAppId());
+          "Unable to find the context flow name, and flow run id, "
+          + "and user id for clusterId=" + clusterId
+          + ", appId=" + appToFlowRowKey.getAppId());
     }
   }
 
@@ -102,9 +114,10 @@ public abstract class AbstractTimelineStorageReader {
         || context.getUserId() == null) {
       // Get flow context information from AppToFlow table.
       AppToFlowRowKey appToFlowRowKey =
-          new AppToFlowRowKey(context.getClusterId(), context.getAppId());
+          new AppToFlowRowKey(context.getAppId());
       FlowContext flowContext =
-          lookupFlowContext(appToFlowRowKey, hbaseConf, conn);
+          lookupFlowContext(appToFlowRowKey, context.getClusterId(), hbaseConf,
+          conn);
       context.setFlowName(flowContext.flowName);
       context.setFlowRunId(flowContext.flowRunId);
       context.setUserId(flowContext.userId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44216a46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.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/storage/common/TestRowKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
index 7560f33..cbd2273 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
@@ -127,10 +127,8 @@ public class TestRowKeys {
    */
   @Test
   public void testAppToFlowRowKey() {
-    byte[] byteRowKey = new AppToFlowRowKey(CLUSTER,
-        APPLICATION_ID).getRowKey();
+    byte[] byteRowKey = new AppToFlowRowKey(APPLICATION_ID).getRowKey();
     AppToFlowRowKey rowKey = AppToFlowRowKey.parseRowKey(byteRowKey);
-    assertEquals(CLUSTER, rowKey.getClusterId());
     assertEquals(APPLICATION_ID, rowKey.getAppId());
   }
 


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


[32/50] [abbrv] hadoop git commit: YARN-6316 Provide help information and documentation for TimelineSchemaCreator (Contributed by Haibo Chen via Vrushali C)

Posted by va...@apache.org.
YARN-6316 Provide help information and documentation for TimelineSchemaCreator (Contributed by Haibo Chen via Vrushali C)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 274c22fd1078548caa7219d5075384bbfbdd902b
Parents: ee6a4d0
Author: vrushali <vr...@apache.org>
Authored: Thu Jun 1 17:04:40 2017 -0700
Committer: vrushali <vr...@apache.org>
Committed: Thu Jun 1 17:04:40 2017 -0700

----------------------------------------------------------------------
 .../storage/TimelineSchemaCreator.java          | 144 +++++++++++++------
 .../src/site/markdown/TimelineServiceV2.md      |   3 +-
 2 files changed, 100 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/274c22fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.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/TimelineSchemaCreator.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/TimelineSchemaCreator.java
index a4c1bbb..a9c74d2 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/TimelineSchemaCreator.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/TimelineSchemaCreator.java
@@ -65,6 +65,8 @@ public final class TimelineSchemaCreator {
   private static final String APP_TO_FLOW_TABLE_NAME_SHORT = "a2f";
   private static final String TTL_OPTION_SHORT = "m";
   private static final String ENTITY_TABLE_NAME_SHORT = "e";
+  private static final String HELP_SHORT = "h";
+  private static final String CREATE_TABLES_SHORT = "c";
 
   public static void main(String[] args) throws Exception {
 
@@ -79,54 +81,44 @@ public final class TimelineSchemaCreator {
     // Grab the arguments we're looking for.
     CommandLine commandLine = parseArgs(otherArgs);
 
-    // Grab the entityTableName argument
-    String entityTableName
-        = commandLine.getOptionValue(ENTITY_TABLE_NAME_SHORT);
-    if (StringUtils.isNotBlank(entityTableName)) {
-      hbaseConf.set(EntityTable.TABLE_NAME_CONF_NAME, entityTableName);
-    }
-    String entityTableTTLMetrics = commandLine.getOptionValue(TTL_OPTION_SHORT);
-    if (StringUtils.isNotBlank(entityTableTTLMetrics)) {
-      int metricsTTL = Integer.parseInt(entityTableTTLMetrics);
-      new EntityTable().setMetricsTTL(metricsTTL, hbaseConf);
-    }
-    // Grab the appToflowTableName argument
-    String appToflowTableName = commandLine.getOptionValue(
-        APP_TO_FLOW_TABLE_NAME_SHORT);
-    if (StringUtils.isNotBlank(appToflowTableName)) {
-      hbaseConf.set(AppToFlowTable.TABLE_NAME_CONF_NAME, appToflowTableName);
-    }
-    // Grab the applicationTableName argument
-    String applicationTableName = commandLine.getOptionValue(
-        APP_TABLE_NAME_SHORT);
-    if (StringUtils.isNotBlank(applicationTableName)) {
-      hbaseConf.set(ApplicationTable.TABLE_NAME_CONF_NAME,
-          applicationTableName);
-    }
-
-    List<Exception> exceptions = new ArrayList<>();
-    try {
-      boolean skipExisting
-          = commandLine.hasOption(SKIP_EXISTING_TABLE_OPTION_SHORT);
-      if (skipExisting) {
-        LOG.info("Will skip existing tables and continue on htable creation "
-            + "exceptions!");
+    if (commandLine.hasOption(HELP_SHORT)) {
+      // -help option has the highest precedence
+      printUsage();
+    } else if (commandLine.hasOption(CREATE_TABLES_SHORT)) {
+      // Grab the entityTableName argument
+      String entityTableName = commandLine.getOptionValue(
+          ENTITY_TABLE_NAME_SHORT);
+      if (StringUtils.isNotBlank(entityTableName)) {
+        hbaseConf.set(EntityTable.TABLE_NAME_CONF_NAME, entityTableName);
       }
-      createAllTables(hbaseConf, skipExisting);
-      LOG.info("Successfully created HBase schema. ");
-    } catch (IOException e) {
-      LOG.error("Error in creating hbase tables: " + e.getMessage());
-      exceptions.add(e);
-    }
-
-    if (exceptions.size() > 0) {
-      LOG.warn("Schema creation finished with the following exceptions");
-      for (Exception e : exceptions) {
-        LOG.warn(e.getMessage());
+      // Grab the TTL argument
+      String entityTableTTLMetrics =commandLine.getOptionValue(
+          TTL_OPTION_SHORT);
+      if (StringUtils.isNotBlank(entityTableTTLMetrics)) {
+        int metricsTTL = Integer.parseInt(entityTableTTLMetrics);
+        new EntityTable().setMetricsTTL(metricsTTL, hbaseConf);
       }
-      System.exit(-1);
+      // Grab the appToflowTableName argument
+      String appToflowTableName = commandLine.getOptionValue(
+          APP_TO_FLOW_TABLE_NAME_SHORT);
+      if (StringUtils.isNotBlank(appToflowTableName)) {
+        hbaseConf.set(AppToFlowTable.TABLE_NAME_CONF_NAME, appToflowTableName);
+      }
+      // Grab the applicationTableName argument
+      String applicationTableName = commandLine.getOptionValue(
+          APP_TABLE_NAME_SHORT);
+      if (StringUtils.isNotBlank(applicationTableName)) {
+        hbaseConf.set(ApplicationTable.TABLE_NAME_CONF_NAME,
+            applicationTableName);
+      }
+
+      // create all table schemas in hbase
+      final boolean skipExisting = commandLine.hasOption(
+          SKIP_EXISTING_TABLE_OPTION_SHORT);
+      createAllSchemas(hbaseConf, skipExisting);
     } else {
-      LOG.info("Schema creation finished successfully");
+      // print usage information if -create is not specified
+      printUsage();
     }
   }
 
@@ -142,7 +134,16 @@ public final class TimelineSchemaCreator {
     Options options = new Options();
 
     // Input
-    Option o = new Option(ENTITY_TABLE_NAME_SHORT, "entityTableName", true,
+    Option o = new Option(HELP_SHORT, "help", false, "print help information");
+    o.setRequired(false);
+    options.addOption(o);
+
+    o = new Option(CREATE_TABLES_SHORT, "create", false,
+        "a mandatory option to create hbase tables");
+    o.setRequired(false);
+    options.addOption(o);
+
+    o = new Option(ENTITY_TABLE_NAME_SHORT, "entityTableName", true,
         "entity table name");
     o.setArgName("entityTableName");
     o.setRequired(false);
@@ -187,6 +188,57 @@ public final class TimelineSchemaCreator {
     return commandLine;
   }
 
+  private static void printUsage() {
+    StringBuilder usage = new StringBuilder("Command Usage: \n");
+    usage.append("TimelineSchemaCreator [-help] Display help info" +
+        " for all commands. Or\n");
+    usage.append("TimelineSchemaCreator -create [OPTIONAL_OPTIONS]" +
+        " Create hbase tables.\n\n");
+    usage.append("The Optional options for creating tables include: \n");
+    usage.append("[-entityTableName <Entity Table Name>] " +
+        "The name of the Entity table\n");
+    usage.append("[-metricsTTL <Entity Table Metrics TTL>]" +
+        " TTL for metrics in the Entity table\n");
+    usage.append("[-appToflowTableName <AppToflow Table Name>]" +
+        " The name of the AppToFlow table\n");
+    usage.append("[-applicationTableName <Application Table Name>]" +
+        " The name of the Application table\n");
+    usage.append("[-skipExistingTable] Whether to skip existing" +
+        " hbase tables\n");
+    System.out.println(usage.toString());
+  }
+
+  /**
+   * Create all table schemas and log success or exception if failed.
+   * @param hbaseConf the hbase configuration to create tables with
+   * @param skipExisting whether to skip existing hbase tables
+   */
+  private static void createAllSchemas(Configuration hbaseConf,
+      boolean skipExisting) {
+    List<Exception> exceptions = new ArrayList<>();
+    try {
+      if (skipExisting) {
+        LOG.info("Will skip existing tables and continue on htable creation "
+            + "exceptions!");
+      }
+      createAllTables(hbaseConf, skipExisting);
+      LOG.info("Successfully created HBase schema. ");
+    } catch (IOException e) {
+      LOG.error("Error in creating hbase tables: " + e.getMessage());
+      exceptions.add(e);
+    }
+
+    if (exceptions.size() > 0) {
+      LOG.warn("Schema creation finished with the following exceptions");
+      for (Exception e : exceptions) {
+        LOG.warn(e.getMessage());
+      }
+      System.exit(-1);
+    } else {
+      LOG.info("Schema creation finished successfully");
+    }
+  }
+
   @VisibleForTesting
   public static void createAllTables(Configuration hbaseConf,
       boolean skipExisting) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/274c22fd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
index 8b18474..b3df844 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
@@ -254,7 +254,8 @@ Finally, run the schema creator tool to create the necessary tables:
 
 The `TimelineSchemaCreator` tool supports a few options that may come handy especially when you
 are testing. For example, you can use `-skipExistingTable` (`-s` for short) to skip existing tables
-and continue to create other tables rather than failing the schema creation. By default, the tables
+and continue to create other tables rather than failing the schema creation. When no option or '-help'
+('-h' for short) is provided, the command usage is printed. By default, the tables
 will have a schema prefix of "prod."
 
 #### Enabling Timeline Service v.2


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


[45/50] [abbrv] hadoop git commit: YARN-5647. [ATSv2 Security] Collector side changes for loading auth filters and principals. Contributed by Varun Saxena

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/TimelineServerUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/TimelineServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/TimelineServerUtils.java
new file mode 100644
index 0000000..78bf20f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/TimelineServerUtils.java
@@ -0,0 +1,92 @@
+/**
+ * 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.util.timeline;
+
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.AuthenticationFilterInitializer;
+import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilter;
+import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer;
+import org.apache.hadoop.yarn.server.timeline.security.TimelineDelgationTokenSecretManagerService;
+
+/**
+ * Set of utility methods to be used across timeline reader and collector.
+ */
+public final class TimelineServerUtils {
+  private static final Log LOG = LogFactory.getLog(TimelineServerUtils.class);
+
+  private TimelineServerUtils() {
+  }
+
+  /**
+   * Sets filter initializers configuration based on existing configuration and
+   * default filters added by timeline service(such as timeline auth filter and
+   * CORS filter).
+   * @param conf Configuration object.
+   * @param configuredInitializers Comma separated list of filter initializers.
+   * @param defaultInitializers Set of initializers added by default by timeline
+   *     service.
+   */
+  public static void setTimelineFilters(Configuration conf,
+      String configuredInitializers, Set<String> defaultInitializers) {
+    String[] parts = configuredInitializers.split(",");
+    Set<String> target = new LinkedHashSet<String>();
+    for (String filterInitializer : parts) {
+      filterInitializer = filterInitializer.trim();
+      if (filterInitializer.equals(
+          AuthenticationFilterInitializer.class.getName()) ||
+          filterInitializer.isEmpty()) {
+        continue;
+      }
+      target.add(filterInitializer);
+    }
+    target.addAll(defaultInitializers);
+    String actualInitializers =
+        org.apache.commons.lang.StringUtils.join(target, ",");
+    LOG.info("Filter initializers set for timeline service: " +
+        actualInitializers);
+    conf.set("hadoop.http.filter.initializers", actualInitializers);
+  }
+
+  /**
+   * Adds timeline authentication filter to the set of default filter
+   * initializers and assigns the delegation token manager service to it.
+   * @param initializers Comma separated list of filter initializers.
+   * @param defaultInitializers Set of initializers added by default by timeline
+   *     service.
+   * @param delegationTokenMgrService Delegation token manager service.
+   *     This will be used by timeline authentication filter to assign
+   *     delegation tokens.
+   */
+  public static void addTimelineAuthFilter(String initializers,
+      Set<String> defaultInitializers,
+      TimelineDelgationTokenSecretManagerService delegationTokenMgrService) {
+    TimelineAuthenticationFilter.setTimelineDelegationTokenSecretManager(
+        delegationTokenMgrService.getTimelineDelegationTokenSecretManager());
+    if (!initializers.contains(
+        TimelineAuthenticationFilterInitializer.class.getName())) {
+      defaultInitializers.add(
+          TimelineAuthenticationFilterInitializer.class.getName());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/package-info.java
new file mode 100644
index 0000000..75c6973
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/util/timeline/package-info.java
@@ -0,0 +1,25 @@
+/**
+ * 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.server.util.timeline contains utility classes used
+ * by ATSv1 and ATSv2 on the server side.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.server.util.timeline;
+import org.apache.hadoop.classification.InterfaceAudience;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java
new file mode 100644
index 0000000..430911e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/timeline/security/TestTimelineAuthenticationFilterInitializer.java
@@ -0,0 +1,76 @@
+/**
+ * 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.timeline.security;
+
+import org.junit.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.http.FilterContainer;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import static org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer.PREFIX;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+
+public class TestTimelineAuthenticationFilterInitializer {
+
+  @Test
+  public void testProxyUserConfiguration() {
+    FilterContainer container = Mockito.mock(FilterContainer.class);
+    for (int i = 0; i < 3; ++i) {
+      Configuration conf = new YarnConfiguration();
+      switch (i) {
+      case 0:
+        // hadoop.proxyuser prefix
+        conf.set("hadoop.proxyuser.foo.hosts", "*");
+        conf.set("hadoop.proxyuser.foo.users", "*");
+        conf.set("hadoop.proxyuser.foo.groups", "*");
+        break;
+      case 1:
+        // yarn.timeline-service.http-authentication.proxyuser prefix
+        conf.set(PREFIX + "proxyuser.foo.hosts", "*");
+        conf.set(PREFIX + "proxyuser.foo.users", "*");
+        conf.set(PREFIX + "proxyuser.foo.groups", "*");
+        break;
+      case 2:
+        // hadoop.proxyuser prefix has been overwritten by
+        // yarn.timeline-service.http-authentication.proxyuser prefix
+        conf.set("hadoop.proxyuser.foo.hosts", "bar");
+        conf.set("hadoop.proxyuser.foo.users", "bar");
+        conf.set("hadoop.proxyuser.foo.groups", "bar");
+        conf.set(PREFIX + "proxyuser.foo.hosts", "*");
+        conf.set(PREFIX + "proxyuser.foo.users", "*");
+        conf.set(PREFIX + "proxyuser.foo.groups", "*");
+        break;
+      default:
+        break;
+      }
+
+      TimelineAuthenticationFilterInitializer initializer =
+          new TimelineAuthenticationFilterInitializer();
+      initializer.initFilter(container, conf);
+      Assert.assertEquals(
+          "*", initializer.filterConfig.get("proxyuser.foo.hosts"));
+      Assert.assertEquals(
+          "*", initializer.filterConfig.get("proxyuser.foo.users"));
+      Assert.assertEquals(
+          "*", initializer.filterConfig.get("proxyuser.foo.groups"));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.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/collector/NodeTimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
index 0323d7b..50ebb0f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/NodeTimelineCollectorManager.java
@@ -18,13 +18,11 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.collector;
 
-import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
-import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER;
-
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
-import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -32,7 +30,9 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer2;
-import org.apache.hadoop.http.lib.StaticUserWebFilter;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -42,6 +42,8 @@ import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
 import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReportNewCollectorInfoRequest;
+import org.apache.hadoop.yarn.server.timelineservice.security.TimelineV2DelegationTokenSecretManagerService;
+import org.apache.hadoop.yarn.server.util.timeline.TimelineServerUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
@@ -65,17 +67,51 @@ public class NodeTimelineCollectorManager extends TimelineCollectorManager {
 
   private volatile CollectorNodemanagerProtocol nmCollectorService;
 
+  private TimelineV2DelegationTokenSecretManagerService tokenMgrService;
+
+  private final boolean runningAsAuxService;
+
   static final String COLLECTOR_MANAGER_ATTR_KEY = "collector.manager";
 
   @VisibleForTesting
   protected NodeTimelineCollectorManager() {
+    this(true);
+  }
+
+  protected NodeTimelineCollectorManager(boolean asAuxService) {
     super(NodeTimelineCollectorManager.class.getName());
+    this.runningAsAuxService = asAuxService;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    tokenMgrService = new TimelineV2DelegationTokenSecretManagerService();
+    addService(tokenMgrService);
+    super.serviceInit(conf);
   }
 
   @Override
   protected void serviceStart() throws Exception {
-    startWebApp();
+    if (UserGroupInformation.isSecurityEnabled() && !runningAsAuxService) {
+      // Do security login for cases where collector is running outside NM.
+      try {
+        doSecureLogin();
+      } catch(IOException ie) {
+        throw new YarnRuntimeException("Failed to login", ie);
+      }
+    }
     super.serviceStart();
+    startWebApp();
+  }
+
+  private void doSecureLogin() throws IOException {
+    Configuration conf = getConfig();
+    InetSocketAddress addr = NetUtils.createSocketAddr(conf.getTrimmed(
+        YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST), 0,
+                YarnConfiguration.TIMELINE_SERVICE_BIND_HOST);
+    SecurityUtil.login(conf, YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
+        YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL, addr.getHostName());
   }
 
   @Override
@@ -105,6 +141,12 @@ public class NodeTimelineCollectorManager extends TimelineCollectorManager {
    */
   private void startWebApp() {
     Configuration conf = getConfig();
+    String initializers = conf.get("hadoop.http.filter.initializers", "");
+    Set<String> defaultInitializers = new LinkedHashSet<String>();
+    TimelineServerUtils.addTimelineAuthFilter(
+        initializers, defaultInitializers, tokenMgrService);
+    TimelineServerUtils.setTimelineFilters(
+        conf, initializers, defaultInitializers);
     String bindAddress = conf.get(YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
         YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST) + ":0";
     try {
@@ -114,16 +156,10 @@ public class NodeTimelineCollectorManager extends TimelineCollectorManager {
           .addEndpoint(URI.create(
               (YarnConfiguration.useHttps(conf) ? "https://" : "http://") +
                   bindAddress));
+      if (YarnConfiguration.useHttps(conf)) {
+        builder = WebAppUtils.loadSslConfiguration(builder, conf);
+      }
       timelineRestServer = builder.build();
-      // TODO: replace this by an authentication filter in future.
-      HashMap<String, String> options = new HashMap<>();
-      String username = conf.get(HADOOP_HTTP_STATIC_USER,
-          DEFAULT_HADOOP_HTTP_STATIC_USER);
-      options.put(HADOOP_HTTP_STATIC_USER, username);
-      HttpServer2.defineFilter(timelineRestServer.getWebAppContext(),
-          "static_user_filter_timeline",
-          StaticUserWebFilter.StaticUserFilter.class.getName(),
-          options, new String[] {"/*"});
 
       timelineRestServer.addJerseyResourcePackage(
           TimelineCollectorWebService.class.getPackage().getName() + ";"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.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/collector/PerNodeTimelineCollectorsAuxService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
index 93e5666..669e752 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/PerNodeTimelineCollectorsAuxService.java
@@ -61,7 +61,7 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
   private ScheduledExecutorService scheduler;
 
   public PerNodeTimelineCollectorsAuxService() {
-    this(new NodeTimelineCollectorManager());
+    this(new NodeTimelineCollectorManager(true));
   }
 
   @VisibleForTesting PerNodeTimelineCollectorsAuxService(
@@ -202,7 +202,8 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
     PerNodeTimelineCollectorsAuxService auxService = null;
     try {
       auxService = collectorManager == null ?
-          new PerNodeTimelineCollectorsAuxService() :
+          new PerNodeTimelineCollectorsAuxService(
+              new NodeTimelineCollectorManager(false)) :
           new PerNodeTimelineCollectorsAuxService(collectorManager);
       ShutdownHookManager.get().addShutdownHook(new ShutdownHook(auxService),
           SHUTDOWN_HOOK_PRIORITY);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.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/collector/TimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
index 07cbb2b..cc23cc9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
@@ -31,7 +31,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -47,7 +47,7 @@ import com.google.common.annotations.VisibleForTesting;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class TimelineCollectorManager extends AbstractService {
+public class TimelineCollectorManager extends CompositeService {
   private static final Log LOG =
       LogFactory.getLog(TimelineCollectorManager.class);
 
@@ -57,7 +57,7 @@ public class TimelineCollectorManager extends AbstractService {
   private boolean writerFlusherRunning;
 
   @Override
-  public void serviceInit(Configuration conf) throws Exception {
+  protected void serviceInit(Configuration conf) throws Exception {
     writer = createTimelineWriter(conf);
     writer.init(conf);
     // create a single dedicated thread for flushing the writer on a periodic

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/TimelineV2DelegationTokenSecretManagerService.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/security/TimelineV2DelegationTokenSecretManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/TimelineV2DelegationTokenSecretManagerService.java
new file mode 100644
index 0000000..eef8436
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/TimelineV2DelegationTokenSecretManagerService.java
@@ -0,0 +1,78 @@
+/**
+ * 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.security;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.timeline.security.TimelineDelgationTokenSecretManagerService;
+
+/**
+ * The service wrapper of {@link TimelineV2DelegationTokenSecretManager}.
+ */
+public class TimelineV2DelegationTokenSecretManagerService extends
+    TimelineDelgationTokenSecretManagerService {
+  public TimelineV2DelegationTokenSecretManagerService() {
+    super(TimelineV2DelegationTokenSecretManagerService.class.getName());
+  }
+
+  @Override
+  protected AbstractDelegationTokenSecretManager
+      <TimelineDelegationTokenIdentifier>
+      createTimelineDelegationTokenSecretManager(long secretKeyInterval,
+          long tokenMaxLifetime, long tokenRenewInterval,
+          long tokenRemovalScanInterval) {
+    return new TimelineV2DelegationTokenSecretManager(secretKeyInterval,
+        tokenMaxLifetime, tokenRenewInterval, tokenRemovalScanInterval);
+  }
+
+  /**
+   * Delegation token secret manager for ATSv2.
+   */
+  @Private
+  @Unstable
+  public static class TimelineV2DelegationTokenSecretManager extends
+      AbstractDelegationTokenSecretManager<TimelineDelegationTokenIdentifier> {
+
+    /**
+     * Create a timeline v2 secret manager.
+     * @param delegationKeyUpdateInterval the number of milliseconds for rolling
+     *        new secret keys.
+     * @param delegationTokenMaxLifetime the maximum lifetime of the delegation
+     *        tokens in milliseconds
+     * @param delegationTokenRenewInterval how often the tokens must be renewed
+     *        in milliseconds
+     * @param delegationTokenRemoverScanInterval how often the tokens are
+     *        scanned for expired tokens in milliseconds
+     */
+    public TimelineV2DelegationTokenSecretManager(
+        long delegationKeyUpdateInterval, long delegationTokenMaxLifetime,
+        long delegationTokenRenewInterval,
+        long delegationTokenRemoverScanInterval) {
+      super(delegationKeyUpdateInterval, delegationTokenMaxLifetime,
+          delegationTokenRenewInterval, delegationTokenRemoverScanInterval);
+    }
+
+    @Override
+    public TimelineDelegationTokenIdentifier createIdentifier() {
+      return new TimelineDelegationTokenIdentifier();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9496748b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/package-info.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/security/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/package-info.java
new file mode 100644
index 0000000..8250092
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/security/package-info.java
@@ -0,0 +1,25 @@
+/**
+ * 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.server.timelineservice.security contains classes
+ * to be used to generate delegation tokens for ATSv2.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.yarn.server.timelineservice.security;
+import org.apache.hadoop.classification.InterfaceAudience;


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


[35/50] [abbrv] hadoop git commit: YARN-6801. NPE in RM while setting collectors map in NodeHeartbeatResponse. Contributed by Vrushali C.

Posted by va...@apache.org.
YARN-6801. NPE in RM while setting collectors map in NodeHeartbeatResponse. Contributed by Vrushali C.

(cherry picked from commit 5791ced4bd864ff6c08cb2c3ee10ab62008f0168)


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

Branch: refs/heads/YARN-5355_branch2
Commit: bb4d7c64b24dfcf4ceda08a5f246db3867fe7db9
Parents: 7147450
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Tue Jul 11 17:59:47 2017 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Tue Jul 11 18:07:36 2017 +0530

----------------------------------------------------------------------
 .../resourcemanager/ResourceTrackerService.java   | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bb4d7c64/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index dc91dd3..4888dda 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -620,13 +620,17 @@ public class ResourceTrackerService extends AbstractService implements
     Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
     // Set collectors for all running apps on this node.
     for (ApplicationId appId : runningApps) {
-      AppCollectorData appCollectorData = rmApps.get(appId).getCollectorData();
-      if (appCollectorData != null) {
-        liveAppCollectorsMap.put(appId, appCollectorData);
-      } else {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Collector for applicaton: " + appId +
-              " hasn't registered yet!");
+      RMApp app = rmApps.get(appId);
+      if (app != null) {
+        AppCollectorData appCollectorData = rmApps.get(appId)
+            .getCollectorData();
+        if (appCollectorData != null) {
+          liveAppCollectorsMap.put(appId, appCollectorData);
+        } else {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Collector for applicaton: " + appId +
+                " hasn't registered yet!");
+          }
         }
       }
     }


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


[34/50] [abbrv] hadoop git commit: YARN-6707. [ATSv2] Update HBase version to 1.2.6 (Vrushali C via Varun Saxena)

Posted by va...@apache.org.
YARN-6707. [ATSv2] Update HBase version to 1.2.6 (Vrushali C via Varun Saxena)

(cherry picked from commit e86eef9812d055f8689646b0f521f80cfc0c6399)

Conflicts:
	hadoop-project/pom.xml


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

Branch: refs/heads/YARN-5355_branch2
Commit: 71474506edd86a741ea5532de907c95281d6310a
Parents: fe6884d
Author: Varun Saxena <va...@apache.org>
Authored: Sun Jun 11 01:37:50 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Sun Jun 11 01:43:48 2017 +0530

----------------------------------------------------------------------
 hadoop-project/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/71474506/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 1124189..56083bf 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -50,7 +50,7 @@
 
     <kafka.version>0.8.2.1</kafka.version>
 
-    <hbase.version>1.2.4</hbase.version>
+    <hbase.version>1.2.6</hbase.version>
     <hbase-compatible-hadoop.version>2.5.1</hbase-compatible-hadoop.version>
 
     <hadoop.assemblies.version>${project.version}</hadoop.assemblies.version>


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


[07/50] [abbrv] hadoop git commit: YARN-6170. TimelineReaderServer should wait to join with HttpServer2 (Sangjin Lee via Varun Saxena)

Posted by va...@apache.org.
YARN-6170. TimelineReaderServer should wait to join with HttpServer2 (Sangjin Lee via Varun Saxena)

(cherry picked from commit 649deb72fbb62568b4ea0d67444df6faaaed169d)


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

Branch: refs/heads/YARN-5355_branch2
Commit: d573ce4489fe2733c37ca45691120bb820522c2f
Parents: e7eed95
Author: Varun Saxena <va...@apache.org>
Authored: Sat Feb 11 19:21:45 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:44 2017 +0530

----------------------------------------------------------------------
 .../timelineservice/reader/TimelineReaderServer.java   | 13 ++++++++++++-
 1 file changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d573ce44/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.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/TimelineReaderServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
index 116cc2a..8c5e72d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderServer.java
@@ -110,6 +110,16 @@ public class TimelineReaderServer extends CompositeService {
     startTimelineReaderWebApp();
   }
 
+  private void join() {
+    // keep the main thread that started the server up until it receives a stop
+    // signal
+    if (readerWebServer != null) {
+      try {
+        readerWebServer.join();
+      } catch (InterruptedException ignore) {}
+    }
+  }
+
   @Override
   protected void serviceStop() throws Exception {
     if (readerWebServer != null) {
@@ -185,6 +195,7 @@ public class TimelineReaderServer extends CompositeService {
     Configuration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-    startTimelineReaderServer(args, conf);
+    TimelineReaderServer server = startTimelineReaderServer(args, conf);
+    server.join();
   }
 }
\ No newline at end of file


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


[43/50] [abbrv] hadoop git commit: Addendum for YARN-6102.

Posted by va...@apache.org.
Addendum for YARN-6102.


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

Branch: refs/heads/YARN-5355_branch2
Commit: aeecf6985fee9fc59eeb2d7dbc87a81042b40957
Parents: 278ca2f
Author: Varun Saxena <va...@apache.org>
Authored: Fri Jul 28 22:56:05 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Fri Jul 28 22:56:05 2017 +0530

----------------------------------------------------------------------
 .../hadoop/yarn/server/resourcemanager/RMContextImpl.java      | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aeecf698/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index ab3672e..c27e1bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -24,7 +24,6 @@ import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.yarn.LocalConfigurationProvider;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -468,6 +467,11 @@ public class RMContextImpl implements RMContext {
   }
 
   @Override
+  public RMTimelineCollectorManager getRMTimelineCollectorManager() {
+    return activeServiceContext.getRMTimelineCollectorManager();
+  }
+
+  @Override
   public long getEpoch() {
     return activeServiceContext.getEpoch();
   }


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


[21/50] [abbrv] hadoop git commit: YARN-6318. timeline service schema creator fails if executed from a remote machine (Sangjin Lee via Varun Saxena)

Posted by va...@apache.org.
YARN-6318. timeline service schema creator fails if executed from a remote machine (Sangjin Lee via Varun Saxena)

(cherry picked from commit 68ec0d9b471356d3adef15d78111126f57ee50f8)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 7e6fdeb36e45355fccc0acfb41fd9b4628a4cb3d
Parents: 4846b0c
Author: Varun Saxena <va...@apache.org>
Authored: Tue Mar 14 02:05:01 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:45 2017 +0530

----------------------------------------------------------------------
 .../storage/TimelineSchemaCreator.java          |  5 ++-
 .../common/HBaseTimelineStorageUtils.java       | 29 ++++++++++-------
 .../common/TestHBaseTimelineStorageUtils.java   | 33 ++++++++++++++++++++
 3 files changed, 54 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e6fdeb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.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/TimelineSchemaCreator.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/TimelineSchemaCreator.java
index dd87169..a4c1bbb 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/TimelineSchemaCreator.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/TimelineSchemaCreator.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
@@ -67,8 +68,10 @@ public final class TimelineSchemaCreator {
 
   public static void main(String[] args) throws Exception {
 
+    LOG.info("Starting the schema creation");
     Configuration hbaseConf =
-        HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(null);
+        HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(
+            new YarnConfiguration());
     // Grab input args and allow for -Dxyz style arguments
     String[] otherArgs = new GenericOptionsParser(hbaseConf, args)
         .getRemainingArgs();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e6fdeb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.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/common/HBaseTimelineStorageUtils.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/common/HBaseTimelineStorageUtils.java
index afe4d6a..865a70d 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/common/HBaseTimelineStorageUtils.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/common/HBaseTimelineStorageUtils.java
@@ -17,6 +17,15 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.storage.common;
 
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -30,17 +39,12 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCom
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-
 /**
  * A bunch of utility functions used in HBase TimelineService backend.
  */
 public final class HBaseTimelineStorageUtils {
+  private static final Log LOG =
+      LogFactory.getLog(HBaseTimelineStorageUtils.class);
 
   /** milliseconds in one day. */
   public static final long MILLIS_ONE_DAY = 86400000L;
@@ -225,8 +229,8 @@ public final class HBaseTimelineStorageUtils {
 
   /**
    * @param conf Yarn configuration. Used to see if there is an explicit config
-   *          pointing to the HBase config file to read. If null then a new
-   *          HBase configuration will be returned.
+   *          pointing to the HBase config file to read. It should not be null
+   *          or a NullPointerException will be thrown.
    * @return a configuration with the HBase configuration from the classpath,
    *         optionally overwritten by the timeline service configuration URL if
    *         specified.
@@ -235,16 +239,17 @@ public final class HBaseTimelineStorageUtils {
    */
   public static Configuration getTimelineServiceHBaseConf(Configuration conf)
       throws MalformedURLException {
-    Configuration hbaseConf;
-
     if (conf == null) {
-      return HBaseConfiguration.create();
+      throw new NullPointerException();
     }
 
+    Configuration hbaseConf;
     String timelineServiceHBaseConfFileURL =
         conf.get(YarnConfiguration.TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE);
     if (timelineServiceHBaseConfFileURL != null
         && timelineServiceHBaseConfFileURL.length() > 0) {
+      LOG.info("Using hbase configuration at " +
+          timelineServiceHBaseConfFileURL);
       // create a clone so that we don't mess with out input one
       hbaseConf = new Configuration(conf);
       Configuration plainHBaseConf = new Configuration(false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e6fdeb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestHBaseTimelineStorageUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestHBaseTimelineStorageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestHBaseTimelineStorageUtils.java
new file mode 100644
index 0000000..402a89b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestHBaseTimelineStorageUtils.java
@@ -0,0 +1,33 @@
+/**
+ * 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.common;
+
+import org.junit.Test;
+
+/**
+ * Unit tests for HBaseTimelineStorageUtils static methos.
+ */
+public class TestHBaseTimelineStorageUtils {
+
+  @Test(expected=NullPointerException.class)
+  public void testGetTimelineServiceHBaseConfNullArgument() throws Exception {
+    HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(null);
+  }
+
+}


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


[38/50] [abbrv] hadoop git commit: YARN-6850 Ensure that supplemented timestamp is stored only for flow run metrics (Contributed by Varun Saxena via Vrushali C)

Posted by va...@apache.org.
YARN-6850 Ensure that supplemented timestamp is stored only for flow run metrics (Contributed by Varun Saxena via Vrushali C)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 50f681eb460d92a9c062cd9d800f790db110efb6
Parents: a6aafd2
Author: Vrushali C <vr...@apache.org>
Authored: Mon Jul 24 16:00:21 2017 -0700
Committer: Vrushali C <vr...@apache.org>
Committed: Mon Jul 24 16:00:21 2017 -0700

----------------------------------------------------------------------
 .../storage/common/ColumnHelper.java            | 44 +++++++++++++++-----
 .../common/HBaseTimelineStorageUtils.java       | 10 +----
 .../storage/flow/FlowRunColumnPrefix.java       |  2 +-
 .../storage/reader/ApplicationEntityReader.java |  8 ++--
 .../storage/reader/GenericEntityReader.java     |  8 ++--
 5 files changed, 44 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/50f681eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.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/common/ColumnHelper.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/common/ColumnHelper.java
index 496b0ae..7a81aef 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/common/ColumnHelper.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/common/ColumnHelper.java
@@ -50,11 +50,28 @@ public class ColumnHelper<T> {
 
   private final ValueConverter converter;
 
+  private final boolean supplementTs;
+
   public ColumnHelper(ColumnFamily<T> columnFamily) {
     this(columnFamily, GenericConverter.getInstance());
   }
 
   public ColumnHelper(ColumnFamily<T> columnFamily, ValueConverter converter) {
+    this(columnFamily, converter, false);
+  }
+
+  /**
+   * @param columnFamily column family implementation.
+   * @param converter converter use to encode/decode values stored in the column
+   *     or column prefix.
+   * @param needSupplementTs flag to indicate if cell timestamp needs to be
+   *     modified for this column by calling
+   *     {@link TimestampGenerator#getSupplementedTimestamp(long, String)}. This
+   *     would be required for columns(such as metrics in flow run table) where
+   *     potential collisions can occur due to same timestamp.
+   */
+  public ColumnHelper(ColumnFamily<T> columnFamily, ValueConverter converter,
+      boolean needSupplementTs) {
     this.columnFamily = columnFamily;
     columnFamilyBytes = columnFamily.getBytes();
     if (converter == null) {
@@ -62,6 +79,7 @@ public class ColumnHelper<T> {
     } else {
       this.converter = converter;
     }
+    this.supplementTs = needSupplementTs;
   }
 
   /**
@@ -104,18 +122,24 @@ public class ColumnHelper<T> {
   }
 
   /*
-   * Figures out the cell timestamp used in the Put For storing into flow run
-   * table. We would like to left shift the timestamp and supplement it with the
-   * AppId id so that there are no collisions in the flow run table's cells
+   * Figures out the cell timestamp used in the Put For storing.
+   * Will supplement the timestamp if required. Typically done for flow run
+   * table.If we supplement the timestamp, we left shift the timestamp and
+   * supplement it with the AppId id so that there are no collisions in the flow
+   * run table's cells.
    */
   private long getPutTimestamp(Long timestamp, Attribute[] attributes) {
     if (timestamp == null) {
       timestamp = System.currentTimeMillis();
     }
-    String appId = getAppIdFromAttributes(attributes);
-    long supplementedTS = TimestampGenerator.getSupplementedTimestamp(
-        timestamp, appId);
-    return supplementedTS;
+    if (!this.supplementTs) {
+      return timestamp;
+    } else {
+      String appId = getAppIdFromAttributes(attributes);
+      long supplementedTS = TimestampGenerator.getSupplementedTimestamp(
+          timestamp, appId);
+      return supplementedTS;
+    }
   }
 
   private String getAppIdFromAttributes(Attribute[] attributes) {
@@ -232,9 +256,9 @@ public class ColumnHelper<T> {
               for (Entry<Long, byte[]> cell : cells.entrySet()) {
                 V value =
                     (V) converter.decodeValue(cell.getValue());
-                cellResults.put(
-                    TimestampGenerator.getTruncatedTimestamp(cell.getKey()),
-                    value);
+                Long ts = supplementTs ? TimestampGenerator.
+                    getTruncatedTimestamp(cell.getKey()) : cell.getKey();
+                cellResults.put(ts, value);
               }
             }
             results.put(converterColumnKey, cellResults);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50f681eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.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/common/HBaseTimelineStorageUtils.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/common/HBaseTimelineStorageUtils.java
index d3ef847..7e4713f 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/common/HBaseTimelineStorageUtils.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/common/HBaseTimelineStorageUtils.java
@@ -313,16 +313,8 @@ public final class HBaseTimelineStorageUtils {
   public static void setMetricsTimeRange(Query query, byte[] metricsCf,
       long tsBegin, long tsEnd) {
     if (tsBegin != 0 || tsEnd != Long.MAX_VALUE) {
-      long supplementedTsBegin = tsBegin == 0 ? 0 :
-          TimestampGenerator.getSupplementedTimestamp(tsBegin, null);
-      long supplementedTsEnd =
-          (tsEnd == Long.MAX_VALUE) ? Long.MAX_VALUE :
-          TimestampGenerator.getSupplementedTimestamp(tsEnd + 1, null);
-      // Handle overflow by resetting time begin to 0 and time end to
-      // Long#MAX_VALUE, if required.
       query.setColumnFamilyTimeRange(metricsCf,
-          ((supplementedTsBegin < 0) ? 0 : supplementedTsBegin),
-          ((supplementedTsEnd < 0) ? Long.MAX_VALUE : supplementedTsEnd));
+          tsBegin, ((tsEnd == Long.MAX_VALUE) ? Long.MAX_VALUE : (tsEnd + 1)));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50f681eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.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/flow/FlowRunColumnPrefix.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/flow/FlowRunColumnPrefix.java
index 103674e..f521cd7 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/flow/FlowRunColumnPrefix.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/flow/FlowRunColumnPrefix.java
@@ -69,7 +69,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
   private FlowRunColumnPrefix(ColumnFamily<FlowRunTable> columnFamily,
       String columnPrefix, AggregationOperation fra, ValueConverter converter,
       boolean compoundColQual) {
-    column = new ColumnHelper<FlowRunTable>(columnFamily, converter);
+    column = new ColumnHelper<FlowRunTable>(columnFamily, converter, true);
     this.columnFamily = columnFamily;
     this.columnPrefix = columnPrefix;
     if (columnPrefix == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50f681eb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.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/ApplicationEntityReader.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/ApplicationEntityReader.java
index cda4510..0edd6a5 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/ApplicationEntityReader.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/ApplicationEntityReader.java
@@ -362,10 +362,10 @@ class ApplicationEntityReader extends GenericEntityReader {
 
   private void setMetricsTimeRange(Query query) {
     // Set time range for metric values.
-    HBaseTimelineStorageUtils.
-        setMetricsTimeRange(query, ApplicationColumnFamily.METRICS.getBytes(),
-            getDataToRetrieve().getMetricsTimeBegin(),
-            getDataToRetrieve().getMetricsTimeEnd());
+    HBaseTimelineStorageUtils.setMetricsTimeRange(
+        query, ApplicationColumnFamily.METRICS.getBytes(),
+        getDataToRetrieve().getMetricsTimeBegin(),
+        getDataToRetrieve().getMetricsTimeEnd());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50f681eb/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 6b740e2..d7aca74 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
@@ -471,10 +471,10 @@ class GenericEntityReader extends TimelineEntityReader {
 
   private void setMetricsTimeRange(Query query) {
     // Set time range for metric values.
-    HBaseTimelineStorageUtils.
-        setMetricsTimeRange(query, EntityColumnFamily.METRICS.getBytes(),
-            getDataToRetrieve().getMetricsTimeBegin(),
-            getDataToRetrieve().getMetricsTimeEnd());
+    HBaseTimelineStorageUtils.setMetricsTimeRange(
+        query, EntityColumnFamily.METRICS.getBytes(),
+        getDataToRetrieve().getMetricsTimeBegin(),
+        getDataToRetrieve().getMetricsTimeEnd());
   }
 
   @Override


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


[30/50] [abbrv] hadoop git commit: YARN-6555. Store application flow context in NM state store for work-preserving restart. (Rohith Sharma K S via Haibo Chen)

Posted by va...@apache.org.
YARN-6555. Store application flow context in NM state store for work-preserving restart. (Rohith Sharma K S via Haibo Chen)

(cherry picked from commit 47474fffac085e0e5ea46336bf80ccd0677017a3)
(cherry picked from commit 8817cb5c8424359b880c6d700e53092f0269c1bb)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 303d7e0a284544b13d5ea04ef699823d31b7933e
Parents: f1f7d65
Author: Haibo Chen <ha...@apache.org>
Authored: Thu May 25 21:15:27 2017 -0700
Committer: Haibo Chen <ha...@apache.org>
Committed: Thu May 25 21:38:58 2017 -0700

----------------------------------------------------------------------
 .../containermanager/ContainerManagerImpl.java  | 71 +++++++++++++-------
 .../application/ApplicationImpl.java            | 27 ++++++--
 .../yarn_server_nodemanager_recovery.proto      |  7 ++
 .../TestContainerManagerRecovery.java           | 40 +++++++++--
 4 files changed, 111 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/303d7e0a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 1d822fe..a9d5f47 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -85,6 +85,7 @@ import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationACLMapProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto;
+import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.FlowContextProto;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ContainerType;
@@ -384,10 +385,20 @@ public class ContainerManagerImpl extends CompositeService implements
           new LogAggregationContextPBImpl(p.getLogAggregationContext());
     }
 
+    FlowContext fc = null;
+    if (p.getFlowContext() != null) {
+      FlowContextProto fcp = p.getFlowContext();
+      fc = new FlowContext(fcp.getFlowName(), fcp.getFlowVersion(),
+          fcp.getFlowRunId());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(
+            "Recovering Flow context: " + fc + " for an application " + appId);
+      }
+    }
+
     LOG.info("Recovering application " + appId);
-    //TODO: Recover flow and flow run ID
-    ApplicationImpl app = new ApplicationImpl(dispatcher, p.getUser(), appId,
-        creds, context, p.getAppLogAggregationInitedTime());
+    ApplicationImpl app = new ApplicationImpl(dispatcher, p.getUser(), fc,
+        appId, creds, context, p.getAppLogAggregationInitedTime());
     context.getApplications().put(appId, app);
     app.handle(new ApplicationInitEvent(appId, acls, logAggregationContext));
   }
@@ -941,7 +952,7 @@ public class ContainerManagerImpl extends CompositeService implements
   private ContainerManagerApplicationProto buildAppProto(ApplicationId appId,
       String user, Credentials credentials,
       Map<ApplicationAccessType, String> appAcls,
-      LogAggregationContext logAggregationContext) {
+      LogAggregationContext logAggregationContext, FlowContext flowContext) {
 
     ContainerManagerApplicationProto.Builder builder =
         ContainerManagerApplicationProto.newBuilder();
@@ -976,6 +987,16 @@ public class ContainerManagerImpl extends CompositeService implements
       }
     }
 
+    builder.clearFlowContext();
+    if (flowContext != null && flowContext.getFlowName() != null
+        && flowContext.getFlowVersion() != null) {
+      FlowContextProto fcp =
+          FlowContextProto.newBuilder().setFlowName(flowContext.getFlowName())
+              .setFlowVersion(flowContext.getFlowVersion())
+              .setFlowRunId(flowContext.getFlowRunId()).build();
+      builder.setFlowContext(fcp);
+    }
+
     return builder.build();
   }
 
@@ -1022,25 +1043,29 @@ public class ContainerManagerImpl extends CompositeService implements
     this.readLock.lock();
     try {
       if (!isServiceStopped()) {
-        // Create the application
-        // populate the flow context from the launch context if the timeline
-        // service v.2 is enabled
-        FlowContext flowContext = null;
-        if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
-          String flowName = launchContext.getEnvironment().get(
-              TimelineUtils.FLOW_NAME_TAG_PREFIX);
-          String flowVersion = launchContext.getEnvironment().get(
-              TimelineUtils.FLOW_VERSION_TAG_PREFIX);
-          String flowRunIdStr = launchContext.getEnvironment().get(
-              TimelineUtils.FLOW_RUN_ID_TAG_PREFIX);
-          long flowRunId = 0L;
-          if (flowRunIdStr != null && !flowRunIdStr.isEmpty()) {
-            flowRunId = Long.parseLong(flowRunIdStr);
-          }
-          flowContext =
-              new FlowContext(flowName, flowVersion, flowRunId);
-        }
         if (!context.getApplications().containsKey(applicationID)) {
+          // Create the application
+          // populate the flow context from the launch context if the timeline
+          // service v.2 is enabled
+          FlowContext flowContext = null;
+          if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
+            String flowName = launchContext.getEnvironment()
+                .get(TimelineUtils.FLOW_NAME_TAG_PREFIX);
+            String flowVersion = launchContext.getEnvironment()
+                .get(TimelineUtils.FLOW_VERSION_TAG_PREFIX);
+            String flowRunIdStr = launchContext.getEnvironment()
+                .get(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX);
+            long flowRunId = 0L;
+            if (flowRunIdStr != null && !flowRunIdStr.isEmpty()) {
+              flowRunId = Long.parseLong(flowRunIdStr);
+            }
+            flowContext = new FlowContext(flowName, flowVersion, flowRunId);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Flow context: " + flowContext
+                  + " created for an application " + applicationID);
+            }
+          }
+
           Application application =
               new ApplicationImpl(dispatcher, user, flowContext,
                   applicationID, credentials, context);
@@ -1054,7 +1079,7 @@ public class ContainerManagerImpl extends CompositeService implements
                 container.getLaunchContext().getApplicationACLs();
             context.getNMStateStore().storeApplication(applicationID,
                 buildAppProto(applicationID, user, credentials, appAcls,
-                    logAggregationContext));
+                    logAggregationContext, flowContext));
             dispatcher.getEventHandler().handle(new ApplicationInitEvent(
                 applicationID, appAcls, logAggregationContext));
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/303d7e0a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
index 4e14eb0..aafb8d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationImpl.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.proto.YarnProtos;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.ContainerManagerApplicationProto;
 import org.apache.hadoop.yarn.server.api.records.AppCollectorData;
+import org.apache.hadoop.yarn.proto.YarnServerNodemanagerRecoveryProtos.FlowContextProto;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType;
@@ -107,13 +108,6 @@ public class ApplicationImpl implements Application {
   }
 
   public ApplicationImpl(Dispatcher dispatcher, String user,
-      ApplicationId appId, Credentials credentials, Context context,
-      long recoveredLogInitedTime) {
-    this(dispatcher, user, null, appId, credentials, context,
-      recoveredLogInitedTime);
-  }
-
-  public ApplicationImpl(Dispatcher dispatcher, String user,
       FlowContext flowContext, ApplicationId appId, Credentials credentials,
       Context context, long recoveredLogInitedTime) {
     this.dispatcher = dispatcher;
@@ -172,6 +166,15 @@ public class ApplicationImpl implements Application {
     public long getFlowRunId() {
       return flowRunId;
     }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("{");
+      sb.append("Flow Name=").append(getFlowName());
+      sb.append(" Flow Versioin=").append(getFlowVersion());
+      sb.append(" Flow Run Id=").append(getFlowRunId()).append(" }");
+      return sb.toString();
+    }
   }
 
   @Override
@@ -391,6 +394,16 @@ public class ApplicationImpl implements Application {
 
     builder.setAppLogAggregationInitedTime(app.applicationLogInitedTimestamp);
 
+    builder.clearFlowContext();
+    if (app.flowContext != null && app.flowContext.getFlowName() != null
+        && app.flowContext.getFlowVersion() != null) {
+      FlowContextProto fcp = FlowContextProto.newBuilder()
+          .setFlowName(app.flowContext.getFlowName())
+          .setFlowVersion(app.flowContext.getFlowVersion())
+          .setFlowRunId(app.flowContext.getFlowRunId()).build();
+      builder.setFlowContext(fcp);
+    }
+
     return builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/303d7e0a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto
index 0dfa20e..7831711 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/proto/yarn_server_nodemanager_recovery.proto
@@ -31,6 +31,7 @@ message ContainerManagerApplicationProto {
   repeated ApplicationACLMapProto acls = 4;
   optional LogAggregationContextProto log_aggregation_context = 5;
   optional int64 appLogAggregationInitedTime = 6 [ default = -1 ];
+  optional FlowContextProto flowContext = 7;
 }
 
 message DeletionServiceDeleteTaskProto {
@@ -52,3 +53,9 @@ message LogDeleterProto {
   optional string user = 1;
   optional int64 deletionTime = 2;
 }
+
+message FlowContextProto {
+  optional string flowName = 1;
+  optional string flowVersion = 2;
+  optional int64 flowRunId = 3;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/303d7e0a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
index 36db50d..85d9639 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
@@ -107,6 +107,7 @@ import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecret
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
 import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -136,6 +137,11 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
     conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
     conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath());
     conf.setLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 1);
+
+    // enable atsv2 by default in test
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+
     // Default delSrvc
     delSrvc = createDeletionService();
     delSrvc.init(conf);
@@ -144,6 +150,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
     nodeHealthChecker = new NodeHealthCheckerService(
         NodeManager.getNodeHealthScriptRunner(conf), dirsHandler);
     nodeHealthChecker.init(conf);
+
   }
 
   @Test
@@ -161,6 +168,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
     cm.start();
 
     // add an application by starting a container
+    String appName = "app_name1";
     String appUser = "app_user1";
     String modUser = "modify_user1";
     String viewUser = "view_user1";
@@ -170,7 +178,8 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
         ApplicationAttemptId.newInstance(appId, 1);
     ContainerId cid = ContainerId.newContainerId(attemptId, 1);
     Map<String, LocalResource> localResources = Collections.emptyMap();
-    Map<String, String> containerEnv = Collections.emptyMap();
+    Map<String, String> containerEnv = new HashMap<>();
+    setFlowContext(containerEnv, appName, appId);
     List<String> containerCmds = Collections.emptyList();
     Map<String, ByteBuffer> serviceData = Collections.emptyMap();
     Credentials containerCreds = new Credentials();
@@ -318,7 +327,8 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
         ApplicationAttemptId.newInstance(appId, 1);
     ContainerId cid = ContainerId.newContainerId(attemptId, 1);
     Map<String, LocalResource> localResources = Collections.emptyMap();
-    Map<String, String> containerEnv = Collections.emptyMap();
+    Map<String, String> containerEnv = new HashMap<>();
+    setFlowContext(containerEnv, "app_name1", appId);
     List<String> containerCmds = Collections.emptyList();
     Map<String, ByteBuffer> serviceData = Collections.emptyMap();
 
@@ -399,7 +409,8 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
     ApplicationAttemptId attemptId =
         ApplicationAttemptId.newInstance(appId, 1);
     ContainerId cid = ContainerId.newContainerId(attemptId, 1);
-    Map<String, String> containerEnv = Collections.emptyMap();
+    Map<String, String> containerEnv = new HashMap<>();
+    setFlowContext(containerEnv, "app_name1", appId);
     Map<String, ByteBuffer> serviceData = Collections.emptyMap();
     Credentials containerCreds = new Credentials();
     DataOutputBuffer dob = new DataOutputBuffer();
@@ -475,7 +486,8 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
         ApplicationAttemptId.newInstance(appId, 1);
     ContainerId cid = ContainerId.newContainerId(attemptId, 1);
     Map<String, LocalResource> localResources = Collections.emptyMap();
-    Map<String, String> containerEnv = Collections.emptyMap();
+    Map<String, String> containerEnv = new HashMap<>();
+    setFlowContext(containerEnv, "app_name1", appId);
     List<String> containerCmds = Collections.emptyList();
     Map<String, ByteBuffer> serviceData = Collections.emptyMap();
     Credentials containerCreds = new Credentials();
@@ -768,4 +780,24 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
           }
     };
   }
+
+  private void setFlowContext(Map<String, String> containerEnv, String appName,
+      ApplicationId appId) {
+    if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      setFlowTags(containerEnv, TimelineUtils.FLOW_NAME_TAG_PREFIX,
+          TimelineUtils.generateDefaultFlowName(appName, appId));
+      setFlowTags(containerEnv, TimelineUtils.FLOW_VERSION_TAG_PREFIX,
+          TimelineUtils.DEFAULT_FLOW_VERSION);
+      setFlowTags(containerEnv, TimelineUtils.FLOW_RUN_ID_TAG_PREFIX,
+          String.valueOf(System.currentTimeMillis()));
+    }
+  }
+
+  private static void setFlowTags(Map<String, String> environment,
+      String tagPrefix, String value) {
+    if (!value.isEmpty()) {
+      environment.put(tagPrefix, value);
+    }
+  }
+
 }


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


[27/50] [abbrv] hadoop git commit: YARN-6375 App level aggregation should not consider metric values reported in the previous aggregation cycle (Varun Saxena via Vrushali C)

Posted by va...@apache.org.
YARN-6375 App level aggregation should not consider metric values reported in the previous aggregation cycle (Varun Saxena via Vrushali C)


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

Branch: refs/heads/YARN-5355_branch2
Commit: ab20af135453ce404fcfb1a0a138edd5050ad01b
Parents: a23a620
Author: Vrushali Channapattan <vr...@apache.org>
Authored: Thu May 4 15:30:29 2017 -0700
Committer: Vrushali Channapattan <vr...@apache.org>
Committed: Thu May 4 15:30:29 2017 -0700

----------------------------------------------------------------------
 .../collector/TimelineCollector.java            | 23 +++--
 .../collector/TestTimelineCollector.java        | 95 +++++++++++++++++++-
 2 files changed, 108 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab20af13/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.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/collector/TimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
index c94c505..5416b26 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
@@ -310,13 +310,15 @@ public abstract class TimelineCollector extends CompositeService {
         // Update aggregateTable
         Map<String, TimelineMetric> aggrRow = aggregateTable.get(m);
         if (aggrRow == null) {
-          Map<String, TimelineMetric> tempRow = new ConcurrentHashMap<>();
+          Map<String, TimelineMetric> tempRow = new HashMap<>();
           aggrRow = aggregateTable.putIfAbsent(m, tempRow);
           if (aggrRow == null) {
             aggrRow = tempRow;
           }
         }
-        aggrRow.put(entityId, m);
+        synchronized (aggrRow) {
+          aggrRow.put(entityId, m);
+        }
       }
     }
 
@@ -335,14 +337,17 @@ public abstract class TimelineCollector extends CompositeService {
         }
         aggrMetric.setRealtimeAggregationOp(TimelineMetricOperation.NOP);
         Map<Object, Object> status = new HashMap<>();
-        for (TimelineMetric m : aggrRow.values()) {
-          TimelineMetric.aggregateTo(m, aggrMetric, status);
-          // getRealtimeAggregationOp returns an enum so we can directly
-          // compare with "!=".
-          if (m.getRealtimeAggregationOp()
-              != aggrMetric.getRealtimeAggregationOp()) {
-            aggrMetric.setRealtimeAggregationOp(m.getRealtimeAggregationOp());
+        synchronized (aggrRow) {
+          for (TimelineMetric m : aggrRow.values()) {
+            TimelineMetric.aggregateTo(m, aggrMetric, status);
+            // getRealtimeAggregationOp returns an enum so we can directly
+            // compare with "!=".
+            if (m.getRealtimeAggregationOp()
+                != aggrMetric.getRealtimeAggregationOp()) {
+              aggrMetric.setRealtimeAggregationOp(m.getRealtimeAggregationOp());
+            }
           }
+          aggrRow.clear();
         }
         Set<TimelineMetric> metrics = e.getMetrics();
         metrics.remove(aggrMetric);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ab20af13/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.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/collector/TestTimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
index a55f227..0f17553 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
@@ -18,19 +18,27 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.collector;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector.AggregationStatusTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 import org.junit.Test;
 
+import com.google.common.collect.Sets;
+
 import java.io.IOException;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
@@ -187,4 +195,89 @@ public class TestTimelineCollector {
       return context;
     }
   }
-}
+
+  private static TimelineEntity createEntity(String id, String type) {
+    TimelineEntity entity = new TimelineEntity();
+    entity.setId(id);
+    entity.setType(type);
+    return entity;
+  }
+
+  private static TimelineMetric createDummyMetric(long ts, Long value) {
+    TimelineMetric metric = new TimelineMetric();
+    metric.setId("dummy_metric");
+    metric.addValue(ts, value);
+    metric.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
+    return metric;
+  }
+
+  @Test
+  public void testClearPreviousEntitiesOnAggregation() throws Exception {
+    final long ts = System.currentTimeMillis();
+    TimelineCollector collector = new TimelineCollector("") {
+        @Override
+        public TimelineCollectorContext getTimelineEntityContext() {
+          return new TimelineCollectorContext("cluster", "user", "flow", "1",
+              1L, ApplicationId.newInstance(ts, 1).toString());
+        }
+    };
+    collector.init(new Configuration());
+    collector.setWriter(mock(TimelineWriter.class));
+
+    // Put 5 entities with different metric values.
+    TimelineEntities entities = new TimelineEntities();
+    for (int i = 1; i <=5; i++) {
+      TimelineEntity entity = createEntity("e" + i, "type");
+      entity.addMetric(createDummyMetric(ts + i, Long.valueOf(i * 50)));
+      entities.addEntity(entity);
+    }
+    collector.putEntities(entities, UserGroupInformation.getCurrentUser());
+
+    TimelineCollectorContext currContext = collector.getTimelineEntityContext();
+    // Aggregate the entities.
+    Map<String, AggregationStatusTable> aggregationGroups
+        = collector.getAggregationGroups();
+    assertEquals(Sets.newHashSet("type"), aggregationGroups.keySet());
+    TimelineEntity aggregatedEntity = TimelineCollector.
+        aggregateWithoutGroupId(aggregationGroups, currContext.getAppId(),
+            TimelineEntityType.YARN_APPLICATION.toString());
+    TimelineMetric aggregatedMetric =
+        aggregatedEntity.getMetrics().iterator().next();
+    assertEquals(750L, aggregatedMetric.getValues().values().iterator().next());
+    assertEquals(TimelineMetricOperation.SUM,
+        aggregatedMetric.getRealtimeAggregationOp());
+
+    // Aggregate entities.
+    aggregatedEntity = TimelineCollector.
+        aggregateWithoutGroupId(aggregationGroups, currContext.getAppId(),
+            TimelineEntityType.YARN_APPLICATION.toString());
+    aggregatedMetric = aggregatedEntity.getMetrics().iterator().next();
+    // No values aggregated as no metrics put for an entity between this
+    // aggregation and the previous one.
+    assertTrue(aggregatedMetric.getValues().isEmpty());
+    assertEquals(TimelineMetricOperation.NOP,
+        aggregatedMetric.getRealtimeAggregationOp());
+
+    // Put 3 entities.
+    entities = new TimelineEntities();
+    for (int i = 1; i <=3; i++) {
+      TimelineEntity entity = createEntity("e" + i, "type");
+      entity.addMetric(createDummyMetric(System.currentTimeMillis() + i, 50L));
+      entities.addEntity(entity);
+    }
+    aggregationGroups = collector.getAggregationGroups();
+    collector.putEntities(entities, UserGroupInformation.getCurrentUser());
+
+    // Aggregate entities.
+    aggregatedEntity = TimelineCollector.
+        aggregateWithoutGroupId(aggregationGroups, currContext.getAppId(),
+            TimelineEntityType.YARN_APPLICATION.toString());
+    // Last 3 entities picked up for aggregation.
+    aggregatedMetric = aggregatedEntity.getMetrics().iterator().next();
+    assertEquals(150L, aggregatedMetric.getValues().values().iterator().next());
+    assertEquals(TimelineMetricOperation.SUM,
+        aggregatedMetric.getRealtimeAggregationOp());
+
+    collector.close();
+  }
+}
\ No newline at end of file


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


[48/50] [abbrv] hadoop git commit: YARN-5648. [ATSv2 Security] Client side changes for authentication. Contributed by Varun Saxena

Posted by va...@apache.org.
YARN-5648. [ATSv2 Security] Client side changes for authentication. Contributed by Varun Saxena

(cherry picked from commit 3829100cf75e733b2b3a4e8c6806a3696f4ae658)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 235006bdb3a89abedfcbcf2a2fb9f46d3b029025
Parents: 19c7dc0
Author: Jian He <ji...@apache.org>
Authored: Fri Jun 23 10:44:12 2017 -0700
Committer: Varun Saxena <va...@apache.org>
Committed: Tue Aug 1 04:21:58 2017 +0530

----------------------------------------------------------------------
 .../client/api/impl/TimelineV2ClientImpl.java   |  38 ++-
 .../hadoop-yarn-server-tests/pom.xml            |  11 +
 .../hadoop/yarn/server/TestRMNMSecretKeys.java  |  34 +-
 .../security/TestTimelineAuthFilterForV2.java   | 309 +++++++++++++++++++
 .../src/test/resources/krb5.conf                |  28 --
 5 files changed, 381 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/235006bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
index e0e4f00..cd30d98 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
@@ -19,7 +19,10 @@
 package org.apache.hadoop.yarn.client.api.impl;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.lang.reflect.UndeclaredThrowableException;
 import java.net.URI;
+import java.security.PrivilegedExceptionAction;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -69,6 +72,8 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
 
   private ApplicationId contextAppId;
 
+  private UserGroupInformation authUgi;
+
   public TimelineV2ClientImpl(ApplicationId appId) {
     super(TimelineV2ClientImpl.class.getName());
     this.contextAppId = appId;
@@ -88,7 +93,6 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     UserGroupInformation realUgi = ugi.getRealUser();
     String doAsUser = null;
-    UserGroupInformation authUgi = null;
     if (realUgi != null) {
       authUgi = realUgi;
       doAsUser = ugi.getShortUserName();
@@ -192,19 +196,33 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
     }
   }
 
+  private ClientResponse doPutObjects(URI base, String path,
+      MultivaluedMap<String, String> params, Object obj) {
+    return connector.getClient().resource(base).path(path).queryParams(params)
+        .accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_JSON)
+        .put(ClientResponse.class, obj);
+  }
+
   protected void putObjects(URI base, String path,
       MultivaluedMap<String, String> params, Object obj)
       throws IOException, YarnException {
-    ClientResponse resp;
+    ClientResponse resp = null;
     try {
-      resp = connector.getClient().resource(base).path(path).queryParams(params)
-          .accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_JSON)
-          .put(ClientResponse.class, obj);
-    } catch (RuntimeException re) {
-      // runtime exception is expected if the client cannot connect the server
-      String msg = "Failed to get the response from the timeline server.";
-      LOG.error(msg, re);
-      throw new IOException(re);
+      resp = authUgi.doAs(new PrivilegedExceptionAction<ClientResponse>() {
+        @Override
+        public ClientResponse run() throws Exception {
+          return doPutObjects(base, path, params, obj);
+        }
+      });
+    } catch (UndeclaredThrowableException ue) {
+      Throwable cause = ue.getCause();
+      if (cause instanceof IOException) {
+        throw (IOException)cause;
+      } else {
+        throw new IOException(cause);
+      }
+    } catch (InterruptedException ie) {
+      throw (IOException) new InterruptedIOException().initCause(ie);
     }
     if (resp == null ||
         resp.getClientResponseStatus() != ClientResponse.Status.OK) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/235006bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml
index eb3b600..ca7b5fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/pom.xml
@@ -126,6 +126,17 @@
       <groupId>commons-logging</groupId>
       <artifactId>commons-logging</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.bouncycastle</groupId>
+      <artifactId>bcprov-jdk16</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-auth</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/235006bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java
index 8cc2dee..ba14491 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java
@@ -18,10 +18,13 @@
 
 package org.apache.hadoop.yarn.server;
 
+import java.io.File;
 import java.io.IOException;
+import java.util.UUID;
 
+import org.junit.AfterClass;
 import org.junit.Assert;
-
+import org.junit.BeforeClass;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -32,9 +35,38 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResp
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.kerby.util.IOUtil;
 import org.junit.Test;
 
 public class TestRMNMSecretKeys {
+  private static final String KRB5_CONF = "java.security.krb5.conf";
+  private static final File KRB5_CONF_ROOT_DIR = new File(
+      System.getProperty("test.build.dir", "target/test-dir"),
+          UUID.randomUUID().toString());
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    KRB5_CONF_ROOT_DIR.mkdir();
+    File krb5ConfFile = new File(KRB5_CONF_ROOT_DIR, "krb5.conf");
+    krb5ConfFile.createNewFile();
+    String content = "[libdefaults]\n" +
+        "    default_realm = APACHE.ORG\n" +
+        "    udp_preference_limit = 1\n"+
+        "    extra_addresses = 127.0.0.1\n" +
+        "[realms]\n" +
+        "    APACHE.ORG = {\n" +
+        "        admin_server = localhost:88\n" +
+        "        kdc = localhost:88\n}\n" +
+        "[domain_realm]\n" +
+        "    localhost = APACHE.ORG";
+    IOUtil.writeFile(content, krb5ConfFile);
+    System.setProperty(KRB5_CONF, krb5ConfFile.getAbsolutePath());
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    KRB5_CONF_ROOT_DIR.delete();
+  }
 
   @Test(timeout = 1000000)
   public void testNMUpdation() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/235006bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
new file mode 100644
index 0000000..608ef67
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
@@ -0,0 +1,309 @@
+/**
+ * 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.security;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.KerberosTestUtils;
+import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse;
+import org.apache.hadoop.yarn.server.timeline.security.TimelineAuthenticationFilterInitializer;
+import org.apache.hadoop.yarn.server.timelineservice.collector.NodeTimelineCollectorManager;
+import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
+import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Tests timeline authentication filter based security for timeline service v2.
+ */
+@RunWith(Parameterized.class)
+public class TestTimelineAuthFilterForV2 {
+
+  private static final String FOO_USER = "foo";
+  private static final String HTTP_USER = "HTTP";
+
+  private static final File TEST_ROOT_DIR = new File(
+      System.getProperty("test.build.dir", "target" + File.separator +
+          "test-dir"), UUID.randomUUID().toString());
+  private static final String BASEDIR =
+      System.getProperty("test.build.dir", "target/test-dir") + "/"
+          + TestTimelineAuthFilterForV2.class.getSimpleName();
+  private static File httpSpnegoKeytabFile = new File(KerberosTestUtils.
+      getKeytabFile());
+  private static String httpSpnegoPrincipal = KerberosTestUtils.
+      getServerPrincipal();
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> withSsl() {
+    return Arrays.asList(new Object[][] {{false}, {true}});
+  }
+
+  private static MiniKdc testMiniKDC;
+  private static String keystoresDir;
+  private static String sslConfDir;
+  private static Configuration conf;
+  private boolean withSsl;
+  private NodeTimelineCollectorManager collectorManager;
+  private PerNodeTimelineCollectorsAuxService auxService;
+
+  public TestTimelineAuthFilterForV2(boolean withSsl) {
+    this.withSsl = withSsl;
+  }
+
+  @BeforeClass
+  public static void setup() {
+    try {
+      testMiniKDC = new MiniKdc(MiniKdc.createConf(), TEST_ROOT_DIR);
+      testMiniKDC.start();
+      testMiniKDC.createPrincipal(
+          httpSpnegoKeytabFile, HTTP_USER + "/localhost");
+    } catch (Exception e) {
+      fail("Couldn't setup MiniKDC.");
+    }
+
+    // Setup timeline service v2.
+    try {
+      conf = new Configuration(false);
+      conf.setStrings(TimelineAuthenticationFilterInitializer.PREFIX + "type",
+          "kerberos");
+      conf.set(TimelineAuthenticationFilterInitializer.PREFIX +
+          KerberosAuthenticationHandler.PRINCIPAL, httpSpnegoPrincipal);
+      conf.set(TimelineAuthenticationFilterInitializer.PREFIX +
+          KerberosAuthenticationHandler.KEYTAB,
+          httpSpnegoKeytabFile.getAbsolutePath());
+      conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+          "kerberos");
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL,
+          httpSpnegoPrincipal);
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
+          httpSpnegoKeytabFile.getAbsolutePath());
+      // Enable timeline service v2
+      conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+      conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+      conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
+          FileSystemTimelineWriterImpl.class, TimelineWriter.class);
+      conf.set(YarnConfiguration.TIMELINE_SERVICE_BIND_HOST, "localhost");
+      conf.set(FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT,
+          TEST_ROOT_DIR.getAbsolutePath());
+      conf.set("hadoop.proxyuser.HTTP.hosts", "*");
+      conf.set("hadoop.proxyuser.HTTP.users", FOO_USER);
+      UserGroupInformation.setConfiguration(conf);
+      SecurityUtil.login(conf, YarnConfiguration.TIMELINE_SERVICE_KEYTAB,
+          YarnConfiguration.TIMELINE_SERVICE_PRINCIPAL, "localhost");
+    } catch (Exception e) {
+      fail("Couldn't setup TimelineServer V2.");
+    }
+  }
+
+  @Before
+  public void initialize() throws Exception {
+    if (withSsl) {
+      conf.set(YarnConfiguration.YARN_HTTP_POLICY_KEY,
+          HttpConfig.Policy.HTTPS_ONLY.name());
+      File base = new File(BASEDIR);
+      FileUtil.fullyDelete(base);
+      base.mkdirs();
+      keystoresDir = new File(BASEDIR).getAbsolutePath();
+      sslConfDir =
+          KeyStoreTestUtil.getClasspathDir(TestTimelineAuthFilterForV2.class);
+      KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
+    } else  {
+      conf.set(YarnConfiguration.YARN_HTTP_POLICY_KEY,
+          HttpConfig.Policy.HTTP_ONLY.name());
+    }
+    collectorManager = new DummyNodeTimelineCollectorManager();
+    auxService = PerNodeTimelineCollectorsAuxService.launchServer(new String[0],
+        collectorManager, conf);
+  }
+
+  private TimelineV2Client createTimelineClientForUGI(ApplicationId appId) {
+    TimelineV2Client client =
+        TimelineV2Client.createTimelineClient(ApplicationId.newInstance(0, 1));
+    // set the timeline service address.
+    String restBindAddr = collectorManager.getRestServerBindAddress();
+    String addr =
+        "localhost" + restBindAddr.substring(restBindAddr.indexOf(":"));
+    client.setTimelineServiceAddress(addr);
+    client.init(conf);
+    client.start();
+    return client;
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    if (testMiniKDC != null) {
+      testMiniKDC.stop();
+    }
+    FileUtil.fullyDelete(TEST_ROOT_DIR);
+  }
+
+  @After
+  public void destroy() throws Exception {
+    if (auxService != null) {
+      auxService.stop();
+    }
+    if (withSsl) {
+      KeyStoreTestUtil.cleanupSSLConfig(keystoresDir, sslConfDir);
+      File base = new File(BASEDIR);
+      FileUtil.fullyDelete(base);
+    }
+  }
+
+  private static TimelineEntity createEntity(String id, String type) {
+    TimelineEntity entityToStore = new TimelineEntity();
+    entityToStore.setId(id);
+    entityToStore.setType(type);
+    entityToStore.setCreatedTime(0L);
+    return entityToStore;
+  }
+
+  private static void verifyEntity(File entityTypeDir, String id, String type)
+      throws IOException {
+    File entityFile = new File(entityTypeDir, id +
+        FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION);
+    assertTrue(entityFile.exists());
+    TimelineEntity entity = readEntityFile(entityFile);
+    assertNotNull(entity);
+    assertEquals(id, entity.getId());
+    assertEquals(type, entity.getType());
+  }
+
+  private static TimelineEntity readEntityFile(File entityFile)
+      throws IOException {
+    BufferedReader reader = null;
+    String strLine;
+    try {
+      reader = new BufferedReader(new FileReader(entityFile));
+      while ((strLine = reader.readLine()) != null) {
+        if (strLine.trim().length() > 0) {
+          return FileSystemTimelineReaderImpl.
+              getTimelineRecordFromJSON(strLine.trim(), TimelineEntity.class);
+        }
+      }
+      return null;
+    } finally {
+      reader.close();
+    }
+  }
+
+  @Test
+  public void testPutTimelineEntities() throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(0, 1);
+    auxService.addApplication(appId);
+    final String entityType = "dummy_type";
+    File entityTypeDir = new File(TEST_ROOT_DIR.getAbsolutePath() +
+        File.separator + "entities" + File.separator +
+        YarnConfiguration.DEFAULT_RM_CLUSTER_ID + File.separator + "test_user" +
+        File.separator + "test_flow_name" + File.separator +
+        "test_flow_version" + File.separator + "1" + File.separator +
+        appId.toString() + File.separator + entityType);
+    try {
+      KerberosTestUtils.doAs(HTTP_USER + "/localhost", new Callable<Void>() {
+        @Override
+        public Void call() throws Exception {
+          TimelineV2Client client = createTimelineClientForUGI(appId);
+          try {
+            // Sync call. Results available immediately.
+            client.putEntities(createEntity("entity1", entityType));
+            assertEquals(1, entityTypeDir.listFiles().length);
+            verifyEntity(entityTypeDir, "entity1", entityType);
+            // Async call.
+            client.putEntitiesAsync(createEntity("entity2", entityType));
+            return null;
+          } finally {
+            client.stop();
+          }
+        }
+      });
+      // Wait for async entity to be published.
+      for (int i = 0; i < 50; i++) {
+        if (entityTypeDir.listFiles().length == 2) {
+          break;
+        }
+        Thread.sleep(50);
+      }
+      assertEquals(2, entityTypeDir.listFiles().length);
+      verifyEntity(entityTypeDir, "entity2", entityType);
+    } finally {
+      FileUtils.deleteQuietly(entityTypeDir);
+    }
+  }
+
+  private static class DummyNodeTimelineCollectorManager extends
+      NodeTimelineCollectorManager {
+    DummyNodeTimelineCollectorManager() {
+      super();
+    }
+
+    @Override
+    protected CollectorNodemanagerProtocol getNMCollectorService() {
+      CollectorNodemanagerProtocol protocol =
+          mock(CollectorNodemanagerProtocol.class);
+      try {
+        GetTimelineCollectorContextResponse response =
+            GetTimelineCollectorContextResponse.newInstance("test_user",
+                "test_flow_name", "test_flow_version", 1L);
+        when(protocol.getTimelineCollectorContext(any(
+            GetTimelineCollectorContextRequest.class))).thenReturn(response);
+      } catch (YarnException | IOException e) {
+        fail();
+      }
+      return protocol;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/235006bd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/resources/krb5.conf
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/resources/krb5.conf b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/resources/krb5.conf
deleted file mode 100644
index 121ac6d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/resources/krb5.conf
+++ /dev/null
@@ -1,28 +0,0 @@
-#
-# 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.
-# 
-[libdefaults]
-	default_realm = APACHE.ORG
-	udp_preference_limit = 1
-	extra_addresses = 127.0.0.1
-[realms]
-	APACHE.ORG = {
-		admin_server = localhost:88
-		kdc = localhost:88
-	}
-[domain_realm]
-	localhost = APACHE.ORG


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


[49/50] [abbrv] hadoop git commit: Addendum for YARN-5648.

Posted by va...@apache.org.
Addendum for YARN-5648.


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

Branch: refs/heads/YARN-5355_branch2
Commit: 4811d481c15237b4ba14221e3e82bcab3437a28c
Parents: 235006b
Author: Varun Saxena <va...@apache.org>
Authored: Tue Aug 1 19:04:09 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Tue Aug 1 19:04:09 2017 +0530

----------------------------------------------------------------------
 .../yarn/client/api/impl/TimelineV2ClientImpl.java |  4 ++--
 .../hadoop/yarn/server/TestRMNMSecretKeys.java     | 17 +++++++++++++++--
 .../security/TestTimelineAuthFilterForV2.java      |  4 ++--
 3 files changed, 19 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4811d481/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
index cd30d98..ad869e3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
@@ -203,8 +203,8 @@ public class TimelineV2ClientImpl extends TimelineV2Client {
         .put(ClientResponse.class, obj);
   }
 
-  protected void putObjects(URI base, String path,
-      MultivaluedMap<String, String> params, Object obj)
+  protected void putObjects(final URI base, final String path,
+      final MultivaluedMap<String, String> params, final Object obj)
       throws IOException, YarnException {
     ClientResponse resp = null;
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4811d481/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java
index ba14491..56872d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestRMNMSecretKeys.java
@@ -19,7 +19,11 @@
 package org.apache.hadoop.yarn.server;
 
 import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.charset.StandardCharsets;
 import java.util.UUID;
 
 import org.junit.AfterClass;
@@ -35,7 +39,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResp
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.kerby.util.IOUtil;
 import org.junit.Test;
 
 public class TestRMNMSecretKeys {
@@ -59,10 +62,20 @@ public class TestRMNMSecretKeys {
         "        kdc = localhost:88\n}\n" +
         "[domain_realm]\n" +
         "    localhost = APACHE.ORG";
-    IOUtil.writeFile(content, krb5ConfFile);
+    writeFile(content, krb5ConfFile);
     System.setProperty(KRB5_CONF, krb5ConfFile.getAbsolutePath());
   }
 
+  private static void writeFile(String content, File file) throws IOException {
+      FileOutputStream outputStream = new FileOutputStream(file);
+      FileChannel fc = outputStream.getChannel();
+
+      ByteBuffer buffer =
+          ByteBuffer.wrap(content.getBytes(StandardCharsets.UTF_8));
+      fc.write(buffer);
+      outputStream.close();
+  }
+
   @AfterClass
   public static void tearDown() throws IOException {
     KRB5_CONF_ROOT_DIR.delete();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4811d481/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
index 608ef67..da76958 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/security/TestTimelineAuthFilterForV2.java
@@ -243,10 +243,10 @@ public class TestTimelineAuthFilterForV2 {
 
   @Test
   public void testPutTimelineEntities() throws Exception {
-    ApplicationId appId = ApplicationId.newInstance(0, 1);
+    final ApplicationId appId = ApplicationId.newInstance(0, 1);
     auxService.addApplication(appId);
     final String entityType = "dummy_type";
-    File entityTypeDir = new File(TEST_ROOT_DIR.getAbsolutePath() +
+    final File entityTypeDir = new File(TEST_ROOT_DIR.getAbsolutePath() +
         File.separator + "entities" + File.separator +
         YarnConfiguration.DEFAULT_RM_CLUSTER_ID + File.separator + "test_user" +
         File.separator + "test_flow_name" + File.separator +


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


[03/50] [abbrv] hadoop git commit: YARN-6256. Add FROM_ID info key for timeline entities in reader response (Rohith Sharma K S via Varun Saxena)

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a391f54f/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 1a518d0..4a9e53e 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
@@ -74,14 +74,14 @@ final class TimelineReaderWebServicesUtils {
   static TimelineEntityFilters createTimelineEntityFilters(String limit,
       String createdTimeStart, String createdTimeEnd, String relatesTo,
       String isRelatedTo, String infofilters, String conffilters,
-      String metricfilters, String eventfilters, String fromidprefix,
+      String metricfilters, String eventfilters,
       String fromid) throws TimelineParseException {
     return new TimelineEntityFilters(parseLongStr(limit),
         parseLongStr(createdTimeStart), parseLongStr(createdTimeEnd),
         parseRelationFilters(relatesTo), parseRelationFilters(isRelatedTo),
         parseKVFilters(infofilters, false), parseKVFilters(conffilters, true),
         parseMetricFilters(metricfilters), parseEventFilters(eventfilters),
-        parseLongStr(fromidprefix), parseStr(fromid));
+        parseStr(fromid));
   }
 
   /**


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


[18/50] [abbrv] hadoop git commit: YARN-6424. TimelineCollector is not stopped when an app finishes in RM. Contributed by Varun Saxena.

Posted by va...@apache.org.
YARN-6424. TimelineCollector is not stopped when an app finishes in RM. Contributed by Varun Saxena.

(cherry picked from commit 1a9439e299910032ce6a1919dece3107c1c9de5b)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 4deb89d4288caf304de80ad97a255c0eb472613f
Parents: 6d9ae7f
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Thu Apr 6 10:15:22 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:45 2017 +0530

----------------------------------------------------------------------
 .../metrics/TimelineServiceV2Publisher.java              | 11 ++++++-----
 1 file changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4deb89d4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
index 8b1ad1c..1fc0c38 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
@@ -180,8 +180,9 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
         getTimelinelineAppMetrics(appMetrics, finishedTime);
     entity.setMetrics(entityMetrics);
 
-    getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
-        SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
+    getDispatcher().getEventHandler().handle(
+        new ApplicationFinishPublishEvent(SystemMetricsEventType.
+            PUBLISH_APPLICATION_FINISHED_ENTITY, entity, app));
   }
 
   private Set<TimelineMetric> getTimelinelineAppMetrics(
@@ -450,16 +451,16 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
   }
 
   private class ApplicationFinishPublishEvent extends TimelineV2PublishEvent {
-    private RMAppImpl app;
+    private RMApp app;
 
     public ApplicationFinishPublishEvent(SystemMetricsEventType type,
-        TimelineEntity entity, RMAppImpl app) {
+        TimelineEntity entity, RMApp app) {
       super(type, entity, app.getApplicationId());
       this.app = app;
     }
 
     public RMAppImpl getRMAppImpl() {
-      return app;
+      return (RMAppImpl) app;
     }
   }
 


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


[37/50] [abbrv] hadoop git commit: YARN-4455. Support fetching metrics by time range. Contributed by Varun Saxena.

Posted by va...@apache.org.
YARN-4455. Support fetching metrics by time range. Contributed by Varun Saxena.

(cherry picked from commit 743e4731781a7d9a4a5f1f09adc510f193182158)


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

Branch: refs/heads/YARN-5355_branch2
Commit: a6aafd2623b2265fda8adf633d5338ba70891b5f
Parents: bb4d7c6
Author: Rohith Sharma K S <ro...@apache.org>
Authored: Thu Jul 20 12:16:06 2017 +0530
Committer: Rohith Sharma K S <ro...@apache.org>
Committed: Thu Jul 20 14:11:54 2017 +0530

----------------------------------------------------------------------
 ...stTimelineReaderWebServicesHBaseStorage.java | 207 ++++++++++++++++-
 .../storage/DataGeneratorForTest.java           |  27 +--
 .../storage/TestHBaseTimelineStorageApps.java   | 209 ++++++++++++-----
 .../TestHBaseTimelineStorageEntities.java       | 166 ++++++++++----
 .../storage/flow/TestHBaseStorageFlowRun.java   |  18 +-
 .../storage/common/ColumnHelper.java            |   1 -
 .../common/HBaseTimelineStorageUtils.java       |  17 ++
 .../storage/reader/ApplicationEntityReader.java |  14 ++
 .../storage/reader/GenericEntityReader.java     |  12 +-
 .../reader/TimelineDataToRetrieve.java          |  35 ++-
 .../reader/TimelineReaderWebServices.java       | 226 ++++++++++++++++---
 .../reader/TimelineReaderWebServicesUtils.java  |   6 +-
 .../TestFileSystemTimelineReaderImpl.java       |  15 +-
 13 files changed, 777 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/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 6836cc1..b36eb9d 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
@@ -328,7 +328,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
       userEntity.setType("entitytype");
       userEntity.setId("entityid-" + i);
       userEntity.setIdPrefix(11 - i);
-      userEntity.setCreatedTime(System.currentTimeMillis());
+      userEntity.setCreatedTime(ts);
       userEntities.addEntity(userEntity);
     }
 
@@ -344,7 +344,7 @@ public class TestTimelineReaderWebServicesHBaseStorage
           flowVersion2, runid2, entity3.getId(), te3);
       hbi.write(cluster, user, flow, flowVersion, runid,
           "application_1111111111_1111", userEntities);
-      writeApplicationEntities(hbi);
+      writeApplicationEntities(hbi, ts);
       hbi.flush();
     } finally {
       if (hbi != null) {
@@ -353,26 +353,25 @@ public class TestTimelineReaderWebServicesHBaseStorage
     }
   }
 
-  static void writeApplicationEntities(HBaseTimelineWriterImpl hbi)
-      throws IOException {
-    long currentTimeMillis = System.currentTimeMillis();
+  static void writeApplicationEntities(HBaseTimelineWriterImpl hbi,
+      long timestamp) throws IOException {
     int count = 1;
     for (long i = 1; i <= 3; i++) {
       for (int j = 1; j <= 5; j++) {
         TimelineEntities te = new TimelineEntities();
         ApplicationId appId =
-            BuilderUtils.newApplicationId(currentTimeMillis, count++);
+            BuilderUtils.newApplicationId(timestamp, count++);
         ApplicationEntity appEntity = new ApplicationEntity();
         appEntity.setId(appId.toString());
-        appEntity.setCreatedTime(currentTimeMillis);
+        appEntity.setCreatedTime(timestamp);
 
         TimelineEvent created = new TimelineEvent();
         created.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-        created.setTimestamp(currentTimeMillis);
+        created.setTimestamp(timestamp);
         appEntity.addEvent(created);
         TimelineEvent finished = new TimelineEvent();
         finished.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
-        finished.setTimestamp(currentTimeMillis + i * j);
+        finished.setTimestamp(timestamp + i * j);
 
         appEntity.addEvent(finished);
         te.addEntity(appEntity);
@@ -1766,6 +1765,113 @@ public class TestTimelineReaderWebServicesHBaseStorage
     }
   }
 
+  private static void verifyMetricCount(TimelineEntity entity,
+      int expectedMetricsCnt, int expectedMeticsValCnt) {
+    int metricsValCnt = 0;
+    for (TimelineMetric m : entity.getMetrics()) {
+      metricsValCnt += m.getValues().size();
+    }
+    assertEquals(expectedMetricsCnt, entity.getMetrics().size());
+    assertEquals(expectedMeticsValCnt, metricsValCnt);
+  }
+
+  private static void verifyMetricsCount(Set<TimelineEntity> entities,
+      int expectedMetricsCnt, int expectedMeticsValCnt) {
+    int metricsCnt = 0;
+    int metricsValCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricsCnt += entity.getMetrics().size();
+      for (TimelineMetric m : entity.getMetrics()) {
+        metricsValCnt += m.getValues().size();
+      }
+    }
+    assertEquals(expectedMetricsCnt, metricsCnt);
+    assertEquals(expectedMeticsValCnt, metricsValCnt);
+  }
+
+  @Test
+  public void testGetEntitiesMetricsTimeRange() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?fields=ALL&metricslimit=100&metricstimestart=" +
+          (ts - 90000) + "&metricstimeend=" + (ts - 80000));
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      verifyMetricsCount(entities, 4, 4);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?fields=ALL&metricslimit=100&metricstimestart=" +
+          (ts - 100000) + "&metricstimeend=" + (ts - 80000));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      verifyMetricsCount(entities, 5, 9);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?fields=ALL&metricslimit=100&metricstimestart=" +
+          (ts - 100000));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      verifyMetricsCount(entities, 5, 9);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?fields=ALL&metricslimit=100&metricstimeend=" +
+          (ts - 90000));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      verifyMetricsCount(entities, 5, 5);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?fields=ALL&metricstimestart=" +
+          (ts - 100000));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      verifyMetricsCount(entities, 5, 5);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1/entity2?fields=ALL&metricstimestart=" +
+          (ts - 100000) + "&metricstimeend=" + (ts - 80000));
+      resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      verifyMetricCount(entity, 3, 3);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1/entity2?fields=ALL&metricslimit=5&metricstimestart=" +
+          (ts - 100000) + "&metricstimeend=" + (ts - 80000));
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      verifyMetricCount(entity, 3, 5);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?fields=ALL&metricslimit=100&metricstimestart=" +
+          (ts - 80000) + "&metricstimeend=" + (ts - 90000));
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+    } finally {
+      client.destroy();
+    }
+  }
+
   /**
    * Tests if specific configs and metrics are retrieve for getEntity call.
    */
@@ -2366,4 +2472,87 @@ public class TestTimelineReaderWebServicesHBaseStorage
       client.destroy();
     }
   }
+
+  @Test
+  public void testGetAppsMetricsRange() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" +
+          "1002345678919/apps?fields=ALL&metricslimit=100&metricstimestart=" +
+          (ts - 200000) + "&metricstimeend=" + (ts - 100000));
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      verifyMetricsCount(entities, 4, 4);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" +
+          "1002345678919/apps?fields=ALL&metricslimit=100");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      verifyMetricsCount(entities, 4, 10);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/" +
+          "apps?fields=ALL&metricslimit=100&metricstimestart=" +
+          (ts - 200000) + "&metricstimeend=" + (ts - 100000));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(3, entities.size());
+      verifyMetricsCount(entities, 5, 5);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/" +
+          "apps?fields=ALL&metricslimit=100");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(3, entities.size());
+      verifyMetricsCount(entities, 5, 12);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" +
+          "1002345678919/apps?fields=ALL&metricslimit=100&metricstimestart=" +
+          (ts - 200000));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      verifyMetricsCount(entities, 4, 10);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" +
+          "1002345678919/apps?fields=ALL&metricslimit=100&metricstimeend=" +
+          (ts - 100000));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      verifyMetricsCount(entities, 4, 4);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/apps/application_1111111111_1111?userid=user1&fields=ALL" +
+          "&flowname=flow_name&flowrunid=1002345678919&metricslimit=100" +
+          "&metricstimestart=" +(ts - 200000) + "&metricstimeend=" +
+          (ts - 100000));
+      resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      verifyMetricCount(entity, 3, 3);
+
+      uri = URI.create("http://localhost:" + getServerPort() + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/" +
+          "apps?fields=ALL&metricslimit=100&metricstimestart=" +
+          (ts - 100000) + "&metricstimeend=" + (ts - 200000));
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+    } finally {
+      client.destroy();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DataGeneratorForTest.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/storage/DataGeneratorForTest.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/storage/DataGeneratorForTest.java
index 1a522fa..926d8bb 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/storage/DataGeneratorForTest.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/storage/DataGeneratorForTest.java
@@ -58,7 +58,8 @@ public final class DataGeneratorForTest {
     TimelineSchemaCreator.createAllTables(conf, false);
   }
 
-  public static void loadApps(HBaseTestingUtility util) throws IOException {
+  public static void loadApps(HBaseTestingUtility util, long ts)
+      throws IOException {
     TimelineEntities te = new TimelineEntities();
     TimelineEntity entity = new TimelineEntity();
     String id = "application_1111111111_2222";
@@ -92,7 +93,6 @@ public final class DataGeneratorForTest {
     entity.addConfigs(conf);
     // add metrics
     Set<TimelineMetric> metrics = new HashSet<>();
-    long ts = System.currentTimeMillis();
     metrics.add(getMetric4(ts));
 
     TimelineMetric m12 = new TimelineMetric();
@@ -137,7 +137,7 @@ public final class DataGeneratorForTest {
     entity1.addConfigs(conf1);
 
     // add metrics
-    entity1.addMetrics(getMetrics4());
+    entity1.addMetrics(getMetrics4(ts));
     TimelineEvent event11 = new TimelineEvent();
     event11.setId("end_event");
     event11.setTimestamp(ts);
@@ -175,18 +175,17 @@ public final class DataGeneratorForTest {
     }
   }
 
-  private static Set<TimelineMetric> getMetrics4() {
+  private static Set<TimelineMetric> getMetrics4(long ts) {
     Set<TimelineMetric> metrics1 = new HashSet<>();
     TimelineMetric m2 = new TimelineMetric();
     m2.setId("MAP1_SLOT_MILLIS");
-    long ts1 = System.currentTimeMillis();
     Map<Long, Number> metricValues1 = new HashMap<>();
-    metricValues1.put(ts1 - 120000, 100000000);
-    metricValues1.put(ts1 - 100000, 200000000);
-    metricValues1.put(ts1 - 80000, 300000000);
-    metricValues1.put(ts1 - 60000, 400000000);
-    metricValues1.put(ts1 - 40000, 50000000000L);
-    metricValues1.put(ts1 - 20000, 60000000000L);
+    metricValues1.put(ts - 120000, 100000000);
+    metricValues1.put(ts - 100000, 200000000);
+    metricValues1.put(ts - 80000, 300000000);
+    metricValues1.put(ts - 60000, 400000000);
+    metricValues1.put(ts - 40000, 50000000000L);
+    metricValues1.put(ts - 20000, 60000000000L);
     m2.setType(Type.TIME_SERIES);
     m2.setValues(metricValues1);
     metrics1.add(m2);
@@ -307,7 +306,7 @@ public final class DataGeneratorForTest {
     return metricValues;
   }
 
-  public static void loadEntities(HBaseTestingUtility util)
+  public static void loadEntities(HBaseTestingUtility util, long ts)
       throws IOException {
     TimelineEntities te = new TimelineEntities();
     TimelineEntity entity = new TimelineEntity();
@@ -332,7 +331,6 @@ public final class DataGeneratorForTest {
     Set<TimelineMetric> metrics = new HashSet<>();
     TimelineMetric m1 = new TimelineMetric();
     m1.setId("MAP_SLOT_MILLIS");
-    long ts = System.currentTimeMillis();
     m1.setType(Type.TIME_SERIES);
     m1.setValues(getMetricValues1(ts));
     metrics.add(m1);
@@ -383,9 +381,8 @@ public final class DataGeneratorForTest {
     Set<TimelineMetric> metrics1 = new HashSet<>();
     TimelineMetric m2 = new TimelineMetric();
     m2.setId("MAP1_SLOT_MILLIS");
-    long ts1 = System.currentTimeMillis();
     m2.setType(Type.TIME_SERIES);
-    m2.setValues(getMetricValues2(ts1));
+    m2.setValues(getMetricValues2(ts));
     metrics1.add(m2);
     entity1.addMetrics(metrics1);
     te.addEntity(entity1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.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/storage/TestHBaseTimelineStorageApps.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/storage/TestHBaseTimelineStorageApps.java
index 321b72e..d6b0370 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/storage/TestHBaseTimelineStorageApps.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/storage/TestHBaseTimelineStorageApps.java
@@ -86,13 +86,14 @@ public class TestHBaseTimelineStorageApps {
 
   private static HBaseTestingUtility util;
   private HBaseTimelineReaderImpl reader;
+  private static final long CURRENT_TIME = System.currentTimeMillis();
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
     util = new HBaseTestingUtility();
     util.startMiniCluster();
     DataGeneratorForTest.createSchema(util.getConfiguration());
-    DataGeneratorForTest.loadApps(util);
+    DataGeneratorForTest.loadApps(util, CURRENT_TIME);
   }
 
   @Before
@@ -235,13 +236,12 @@ public class TestHBaseTimelineStorageApps {
     TimelineMetric m1 = new TimelineMetric();
     m1.setId("MAP_SLOT_MILLIS");
     Map<Long, Number> metricValues = new HashMap<Long, Number>();
-    long ts = System.currentTimeMillis();
-    metricValues.put(ts - 120000, 100000000);
-    metricValues.put(ts - 100000, 200000000);
-    metricValues.put(ts - 80000, 300000000);
-    metricValues.put(ts - 60000, 400000000);
-    metricValues.put(ts - 40000, 50000000000L);
-    metricValues.put(ts - 20000, 60000000000L);
+    metricValues.put(CURRENT_TIME - 120000, 100000000);
+    metricValues.put(CURRENT_TIME - 100000, 200000000);
+    metricValues.put(CURRENT_TIME - 80000, 300000000);
+    metricValues.put(CURRENT_TIME - 60000, 400000000);
+    metricValues.put(CURRENT_TIME - 40000, 50000000000L);
+    metricValues.put(CURRENT_TIME - 20000, 60000000000L);
     m1.setType(Type.TIME_SERIES);
     m1.setValues(metricValues);
     metrics.add(m1);
@@ -258,7 +258,7 @@ public class TestHBaseTimelineStorageApps {
     TimelineMetric aggMetric = new TimelineMetric();
     aggMetric.setId("MEM_USAGE");
     Map<Long, Number> aggMetricValues = new HashMap<Long, Number>();
-    long aggTs = ts;
+    long aggTs = CURRENT_TIME;
     aggMetricValues.put(aggTs - 120000, 102400000L);
     aggMetric.setType(Type.SINGLE_VALUE);
     aggMetric.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
@@ -379,7 +379,7 @@ public class TestHBaseTimelineStorageApps {
           new TimelineReaderContext(cluster, user, flow, runid, appId,
           entity.getType(), entity.getId()),
           new TimelineDataToRetrieve(null, null,
-          EnumSet.of(TimelineReader.Field.ALL), Integer.MAX_VALUE));
+          EnumSet.of(TimelineReader.Field.ALL), Integer.MAX_VALUE, null, null));
       assertNotNull(e1);
 
       // verify attributes
@@ -422,7 +422,7 @@ public class TestHBaseTimelineStorageApps {
       e1 = reader.getEntity(new TimelineReaderContext(cluster, user, flow,
           runid, appId, entity.getType(), entity.getId()),
           new TimelineDataToRetrieve(null, null,
-          EnumSet.of(TimelineReader.Field.ALL), 3));
+          EnumSet.of(TimelineReader.Field.ALL), 3, null, null));
       assertNotNull(e1);
       assertEquals(appId, e1.getId());
       assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
@@ -443,7 +443,7 @@ public class TestHBaseTimelineStorageApps {
       e1 = reader.getEntity(
           new TimelineReaderContext(cluster, user, flow, runid, appId,
          entity.getType(), entity.getId()), new TimelineDataToRetrieve(
-         null, null, EnumSet.of(TimelineReader.Field.ALL), null));
+         null, null, EnumSet.of(TimelineReader.Field.ALL), null, null, null));
       assertNotNull(e1);
       assertEquals(appId, e1.getId());
       assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
@@ -464,9 +464,9 @@ public class TestHBaseTimelineStorageApps {
             metric.getId().equals("MEM_USAGE"));
         assertEquals(1, metric.getValues().size());
         if (metric.getId().equals("MAP_SLOT_MILLIS")) {
-          assertTrue(metric.getValues().containsKey(ts - 20000));
-          assertEquals(metricValues.get(ts - 20000),
-              metric.getValues().get(ts - 20000));
+          assertTrue(metric.getValues().containsKey(CURRENT_TIME - 20000));
+          assertEquals(metricValues.get(CURRENT_TIME - 20000),
+              metric.getValues().get(CURRENT_TIME - 20000));
         }
         if (metric.getId().equals("MEM_USAGE")) {
           assertTrue(metric.getValues().containsKey(aggTs - 120000));
@@ -552,11 +552,13 @@ public class TestHBaseTimelineStorageApps {
       TimelineEntity e1 = reader.getEntity(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+          null, null));
       TimelineEntity e2 = reader.getEntity(
           new TimelineReaderContext(cluster, user, null, null, appName,
           entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+          null, null));
       assertNotNull(e1);
       assertNotNull(e2);
       assertEquals(e1, e2);
@@ -650,7 +652,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1111111111_2222",
         TimelineEntityType.YARN_APPLICATION.toString(), null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertNotNull(entity);
     assertEquals(3, entity.getConfigs().size());
     assertEquals(1, entity.getIsRelatedToEntities().size());
@@ -659,7 +662,8 @@ public class TestHBaseTimelineStorageApps {
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
         new TimelineEntityFilters.Builder().build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertEquals(3, entities.size());
     int cfgCnt = 0;
     int metricCnt = 0;
@@ -773,17 +777,17 @@ public class TestHBaseTimelineStorageApps {
         1002345678919L, "application_1111111111_2222",
         TimelineEntityType.YARN_APPLICATION.toString(), null),
         new TimelineDataToRetrieve(
-        null, null, EnumSet.of(Field.INFO, Field.CONFIGS), null));
+        null, null, EnumSet.of(Field.INFO, Field.CONFIGS), null, null, null));
     assertNotNull(e1);
     assertEquals(3, e1.getConfigs().size());
     assertEquals(0, e1.getIsRelatedToEntities().size());
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
-        null),
-        new TimelineEntityFilters.Builder().build(),
+        null), new TimelineEntityFilters.Builder().build(),
         new TimelineDataToRetrieve(
-        null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS), null));
+        null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS), null,
+        null, null));
     assertEquals(3, es1.size());
     int metricsCnt = 0;
     int isRelatedToCnt = 0;
@@ -812,7 +816,8 @@ public class TestHBaseTimelineStorageApps {
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
         new TimelineEntityFilters.Builder().isRelatedTo(irt).build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertEquals(2, entities.size());
     int isRelatedToCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -964,7 +969,8 @@ public class TestHBaseTimelineStorageApps {
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
         new TimelineEntityFilters.Builder().relatesTo(rt).build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertEquals(2, entities.size());
     int relatesToCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -1202,7 +1208,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineEntityFilters.Builder().configFilters(confFilterList)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-        null));
+        null, null, null));
     assertEquals(2, entities.size());
     int cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1216,7 +1222,8 @@ public class TestHBaseTimelineStorageApps {
         null),
         new TimelineEntityFilters.Builder().configFilters(confFilterList)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertEquals(2, entities.size());
     cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1234,7 +1241,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineEntityFilters.Builder().configFilters(confFilterList1)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-        null));
+        null, null, null));
     assertEquals(1, entities.size());
     cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1254,7 +1261,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineEntityFilters.Builder().configFilters(confFilterList2)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-        null));
+        null, null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList confFilterList3 = new TimelineFilterList(
@@ -1267,7 +1274,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineEntityFilters.Builder().configFilters(confFilterList3)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-        null));
+        null, null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList confFilterList4 = new TimelineFilterList(
@@ -1280,7 +1287,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineEntityFilters.Builder().configFilters(confFilterList4)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-        null));
+        null, null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList confFilterList5 = new TimelineFilterList(
@@ -1293,7 +1300,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineEntityFilters.Builder().configFilters(confFilterList5)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-        null));
+        null, null, null));
     assertEquals(3, entities.size());
   }
 
@@ -1309,7 +1316,8 @@ public class TestHBaseTimelineStorageApps {
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
         new TimelineEntityFilters.Builder().eventFilters(ef).build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertEquals(1, entities.size());
     int eventCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -1431,7 +1439,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1111111111_2222",
         TimelineEntityType.YARN_APPLICATION.toString(), null),
-        new TimelineDataToRetrieve(list, null, null, null));
+        new TimelineDataToRetrieve(list, null, null, null, null, null));
     assertNotNull(e1);
     assertEquals(1, e1.getConfigs().size());
     Set<TimelineEntity> es1 = reader.getEntities(
@@ -1439,7 +1447,7 @@ public class TestHBaseTimelineStorageApps {
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null) ,
         new TimelineEntityFilters.Builder().build(),
-        new TimelineDataToRetrieve(list, null, null, null));
+        new TimelineDataToRetrieve(list, null, null, null, null, null));
     int cfgCnt = 0;
     for (TimelineEntity entity : es1) {
       cfgCnt += entity.getConfigs().size();
@@ -1465,7 +1473,7 @@ public class TestHBaseTimelineStorageApps {
         null),
         new TimelineEntityFilters.Builder().configFilters(confFilterList)
             .build(),
-        new TimelineDataToRetrieve(list, null, null, null));
+        new TimelineDataToRetrieve(list, null, null, null, null, null));
     assertEquals(1, entities.size());
     int cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1498,7 +1506,8 @@ public class TestHBaseTimelineStorageApps {
         null),
         new TimelineEntityFilters.Builder().configFilters(confFilterList1)
             .build(),
-        new TimelineDataToRetrieve(confsToRetrieve, null, null, null));
+        new TimelineDataToRetrieve(confsToRetrieve, null, null, null, null,
+        null));
     assertEquals(2, entities.size());
     cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1530,7 +1539,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
-        null));
+        null, null, null));
     assertEquals(2, entities.size());
     int metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1544,7 +1553,8 @@ public class TestHBaseTimelineStorageApps {
         null),
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertEquals(2, entities.size());
     metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1564,7 +1574,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
-        null));
+        null, null, null));
     assertEquals(1, entities.size());
     metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1584,7 +1594,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList2)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
-        null));
+        null, null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList metricFilterList3 = new TimelineFilterList(
@@ -1597,7 +1607,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList3)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
-        null));
+        null, null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList metricFilterList4 = new TimelineFilterList(
@@ -1610,7 +1620,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList4)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
-        null));
+        null, null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList metricFilterList5 = new TimelineFilterList(
@@ -1623,7 +1633,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList5)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
-        null));
+        null, null, null));
     assertEquals(3, entities.size());
   }
 
@@ -1636,7 +1646,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1111111111_2222",
         TimelineEntityType.YARN_APPLICATION.toString(), null),
-        new TimelineDataToRetrieve(null, list, null, null));
+        new TimelineDataToRetrieve(null, list, null, null, null, null));
     assertNotNull(e1);
     assertEquals(1, e1.getMetrics().size());
     Set<TimelineEntity> es1 = reader.getEntities(
@@ -1644,7 +1654,7 @@ public class TestHBaseTimelineStorageApps {
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
         new TimelineEntityFilters.Builder().build(),
-        new TimelineDataToRetrieve(null, list, null, null));
+        new TimelineDataToRetrieve(null, list, null, null, null, null));
     int metricCnt = 0;
     for (TimelineEntity entity : es1) {
       metricCnt += entity.getMetrics().size();
@@ -1670,7 +1680,7 @@ public class TestHBaseTimelineStorageApps {
         null),
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
             .build(),
-        new TimelineDataToRetrieve(null, list, null, null));
+        new TimelineDataToRetrieve(null, list, null, null, null, null));
     int metricCnt = 0;
     assertEquals(1, entities.size());
     for (TimelineEntity entity : entities) {
@@ -1696,7 +1706,8 @@ public class TestHBaseTimelineStorageApps {
         null),
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
             .build(),
-        new TimelineDataToRetrieve(null, metricsToRetrieve, null, null));
+        new TimelineDataToRetrieve(null, metricsToRetrieve, null, null, null,
+        null));
     metricCnt = 0;
     assertEquals(2, entities.size());
     for (TimelineEntity entity : entities) {
@@ -1713,8 +1724,8 @@ public class TestHBaseTimelineStorageApps {
         TimelineEntityType.YARN_APPLICATION.toString(), null),
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
             .build(),
-        new TimelineDataToRetrieve(null,
-        metricsToRetrieve, EnumSet.of(Field.METRICS), Integer.MAX_VALUE));
+        new TimelineDataToRetrieve(null, metricsToRetrieve,
+        EnumSet.of(Field.METRICS), Integer.MAX_VALUE, null, null));
     metricCnt = 0;
     int metricValCnt = 0;
     assertEquals(2, entities.size());
@@ -1731,6 +1742,86 @@ public class TestHBaseTimelineStorageApps {
   }
 
   @Test
+  public void testReadAppsMetricTimeRange() throws Exception {
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null), new TimelineEntityFilters.Builder().build(),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        100, null, null));
+    assertEquals(3, entities.size());
+    int metricTimeSeriesCnt = 0;
+    int metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric m : entity.getMetrics()) {
+        metricTimeSeriesCnt += m.getValues().size();
+      }
+    }
+    assertEquals(3, metricCnt);
+    assertEquals(13, metricTimeSeriesCnt);
+
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null), new TimelineEntityFilters.Builder().build(),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        100, CURRENT_TIME - 40000, CURRENT_TIME));
+    assertEquals(3, entities.size());
+    metricCnt = 0;
+    metricTimeSeriesCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric m : entity.getMetrics()) {
+        for (Long ts : m.getValues().keySet()) {
+          assertTrue(ts >= CURRENT_TIME - 40000 && ts <= CURRENT_TIME);
+        }
+        metricTimeSeriesCnt += m.getValues().size();
+      }
+    }
+    assertEquals(3, metricCnt);
+    assertEquals(5, metricTimeSeriesCnt);
+
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null), new TimelineEntityFilters.Builder().build(),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null, CURRENT_TIME - 40000, CURRENT_TIME));
+    assertEquals(3, entities.size());
+    metricCnt = 0;
+    metricTimeSeriesCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric m : entity.getMetrics()) {
+        for (Long ts : m.getValues().keySet()) {
+          assertTrue(ts >= CURRENT_TIME - 40000 && ts <= CURRENT_TIME);
+        }
+        metricTimeSeriesCnt += m.getValues().size();
+      }
+    }
+    assertEquals(3, metricCnt);
+    assertEquals(3, metricTimeSeriesCnt);
+
+    TimelineEntity entity = reader.getEntity(new TimelineReaderContext(
+        "cluster1", "user1", "some_flow_name", 1002345678919L,
+        "application_1111111111_2222",
+        TimelineEntityType.YARN_APPLICATION.toString(), null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), 100,
+        CURRENT_TIME - 40000, CURRENT_TIME));
+    assertNotNull(entity);
+    assertEquals(2, entity.getMetrics().size());
+    metricTimeSeriesCnt = 0;
+    for (TimelineMetric m : entity.getMetrics()) {
+      for (Long ts : m.getValues().keySet()) {
+        assertTrue(ts >= CURRENT_TIME - 40000 && ts <= CURRENT_TIME);
+      }
+      metricTimeSeriesCnt += m.getValues().size();
+    }
+    assertEquals(3, metricTimeSeriesCnt);
+  }
+
+  @Test
   public void testReadAppsInfoFilters() throws Exception {
     TimelineFilterList list1 = new TimelineFilterList();
     list1.addFilter(new TimelineKeyValueFilter(
@@ -1749,7 +1840,8 @@ public class TestHBaseTimelineStorageApps {
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
         new TimelineEntityFilters.Builder().infoFilters(infoFilterList).build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null,
+        null, null));
     assertEquals(2, entities.size());
     int infoCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1766,7 +1858,8 @@ public class TestHBaseTimelineStorageApps {
         null),
         new TimelineEntityFilters.Builder().infoFilters(infoFilterList1)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null,
+        null, null));
     assertEquals(1, entities.size());
     infoCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1785,7 +1878,8 @@ public class TestHBaseTimelineStorageApps {
         null),
         new TimelineEntityFilters.Builder().infoFilters(infoFilterList2)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null,
+        null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList infoFilterList3 = new TimelineFilterList(
@@ -1797,7 +1891,8 @@ public class TestHBaseTimelineStorageApps {
         null),
         new TimelineEntityFilters.Builder().infoFilters(infoFilterList3)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null,
+        null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList infoFilterList4 = new TimelineFilterList(
@@ -1809,7 +1904,8 @@ public class TestHBaseTimelineStorageApps {
         null),
         new TimelineEntityFilters.Builder().infoFilters(infoFilterList4)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null,
+        null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList infoFilterList5 = new TimelineFilterList(
@@ -1821,7 +1917,8 @@ public class TestHBaseTimelineStorageApps {
         null),
         new TimelineEntityFilters.Builder().infoFilters(infoFilterList5)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null,
+        null, null));
     assertEquals(3, entities.size());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.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/storage/TestHBaseTimelineStorageEntities.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/storage/TestHBaseTimelineStorageEntities.java
index 9b47381..7ac5b36 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/storage/TestHBaseTimelineStorageEntities.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/storage/TestHBaseTimelineStorageEntities.java
@@ -93,13 +93,14 @@ public class TestHBaseTimelineStorageEntities {
 
   private static HBaseTestingUtility util;
   private HBaseTimelineReaderImpl reader;
+  private static final long CURRENT_TIME = System.currentTimeMillis();
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
     util = new HBaseTestingUtility();
     util.startMiniCluster();
     DataGeneratorForTest.createSchema(util.getConfiguration());
-    DataGeneratorForTest.loadEntities(util);
+    DataGeneratorForTest.loadEntities(util, CURRENT_TIME);
   }
 
   @Before
@@ -294,13 +295,13 @@ public class TestHBaseTimelineStorageEntities {
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), entity.getId()),
           new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL),
-          Integer.MAX_VALUE));
+          Integer.MAX_VALUE, null, null));
       Set<TimelineEntity> es1 = reader.getEntities(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), null),
           new TimelineEntityFilters.Builder().build(),
           new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL),
-          Integer.MAX_VALUE));
+          Integer.MAX_VALUE, null, null));
       assertNotNull(e1);
       assertEquals(1, es1.size());
 
@@ -331,7 +332,8 @@ public class TestHBaseTimelineStorageEntities {
 
       e1 = reader.getEntity(new TimelineReaderContext(cluster, user, flow,
           runid, appName, entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+          null, null));
       assertNotNull(e1);
       assertEquals(id, e1.getId());
       assertEquals(type, e1.getType());
@@ -449,12 +451,14 @@ public class TestHBaseTimelineStorageEntities {
       TimelineEntity e1 = reader.getEntity(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+          null, null));
       Set<TimelineEntity> es1 = reader.getEntities(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), null),
           new TimelineEntityFilters.Builder().build(),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+          null, null));
       assertNotNull(e1);
       assertEquals(1, es1.size());
 
@@ -513,7 +517,8 @@ public class TestHBaseTimelineStorageEntities {
       TimelineEntity e1 = reader.getEntity(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+          null, null));
       assertNotNull(e1);
       // check the events
       NavigableSet<TimelineEvent> events = e1.getEvents();
@@ -542,7 +547,8 @@ public class TestHBaseTimelineStorageEntities {
     TimelineEntity entity = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", "hello"),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertNotNull(entity);
     assertEquals(3, entity.getConfigs().size());
     assertEquals(1, entity.getIsRelatedToEntities().size());
@@ -550,7 +556,8 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world",
         null), new TimelineEntityFilters.Builder().build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertEquals(3, entities.size());
     int cfgCnt = 0;
     int metricCnt = 0;
@@ -677,7 +684,8 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().eventFilters(ef).build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertEquals(1, entities.size());
     int eventCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -797,7 +805,8 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().isRelatedTo(irt).build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertEquals(2, entities.size());
     int isRelatedToCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -939,7 +948,8 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().relatesTo(rt).build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertEquals(2, entities.size());
     int relatesToCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -1134,7 +1144,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", "hello"),
         new TimelineDataToRetrieve(
-        null, null, EnumSet.of(Field.INFO, Field.CONFIGS), null));
+        null, null, EnumSet.of(Field.INFO, Field.CONFIGS), null, null, null));
     assertNotNull(e1);
     assertEquals(3, e1.getConfigs().size());
     assertEquals(0, e1.getIsRelatedToEntities().size());
@@ -1142,8 +1152,8 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().build(),
-        new TimelineDataToRetrieve(
-        null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.IS_RELATED_TO,
+        Field.METRICS), null, null, null));
     assertEquals(3, es1.size());
     int metricsCnt = 0;
     int isRelatedToCnt = 0;
@@ -1166,14 +1176,14 @@ public class TestHBaseTimelineStorageEntities {
     TimelineEntity e1 = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", "hello"),
-        new TimelineDataToRetrieve(list, null, null, null));
+        new TimelineDataToRetrieve(list, null, null, null, null, null));
     assertNotNull(e1);
     assertEquals(1, e1.getConfigs().size());
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().build(),
-        new TimelineDataToRetrieve(list, null, null, null));
+        new TimelineDataToRetrieve(list, null, null, null, null, null));
     int cfgCnt = 0;
     for (TimelineEntity entity : es1) {
       cfgCnt += entity.getConfigs().size();
@@ -1205,7 +1215,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().configFilters(confFilterList)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-        null));
+        null, null, null));
     assertEquals(2, entities.size());
     int cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1218,7 +1228,8 @@ public class TestHBaseTimelineStorageEntities {
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().configFilters(confFilterList)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertEquals(2, entities.size());
     cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1235,7 +1246,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().configFilters(confFilterList1)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-        null));
+        null, null, null));
     assertEquals(1, entities.size());
     cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1254,7 +1265,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().configFilters(confFilterList2)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-        null));
+        null, null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList confFilterList3 = new TimelineFilterList(
@@ -1266,7 +1277,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().configFilters(confFilterList3)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-        null));
+        null, null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList confFilterList4 = new TimelineFilterList(
@@ -1278,7 +1289,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().configFilters(confFilterList4)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-            null));
+            null, null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList confFilterList5 = new TimelineFilterList(
@@ -1290,7 +1301,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().configFilters(confFilterList5)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
-        null));
+        null, null, null));
     assertEquals(3, entities.size());
   }
 
@@ -1307,7 +1318,7 @@ public class TestHBaseTimelineStorageEntities {
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().configFilters(confFilterList)
             .build(),
-        new TimelineDataToRetrieve(list, null, null, null));
+        new TimelineDataToRetrieve(list, null, null, null, null, null));
     assertEquals(1, entities.size());
     int cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1338,7 +1349,8 @@ public class TestHBaseTimelineStorageEntities {
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().configFilters(confFilterList1)
             .build(),
-        new TimelineDataToRetrieve(confsToRetrieve, null, null, null));
+        new TimelineDataToRetrieve(confsToRetrieve, null, null, null, null,
+        null));
     assertEquals(2, entities.size());
     cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1359,14 +1371,14 @@ public class TestHBaseTimelineStorageEntities {
     TimelineEntity e1 = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", "hello"),
-        new TimelineDataToRetrieve(null, list, null, null));
+        new TimelineDataToRetrieve(null, list, null, null, null, null));
     assertNotNull(e1);
     assertEquals(1, e1.getMetrics().size());
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().build(),
-        new TimelineDataToRetrieve(null, list, null, null));
+        new TimelineDataToRetrieve(null, list, null, null, null, null));
     int metricCnt = 0;
     for (TimelineEntity entity : es1) {
       metricCnt += entity.getMetrics().size();
@@ -1379,6 +1391,63 @@ public class TestHBaseTimelineStorageEntities {
   }
 
   @Test
+  public void testReadEntitiesMetricTimeRange() throws Exception {
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters.Builder().build(),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        100, null, null));
+    assertEquals(3, entities.size());
+    int metricTimeSeriesCnt = 0;
+    int metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric m : entity.getMetrics()) {
+        metricTimeSeriesCnt += m.getValues().size();
+      }
+    }
+    assertEquals(3, metricCnt);
+    assertEquals(13, metricTimeSeriesCnt);
+
+    entities = reader.getEntities(new TimelineReaderContext("cluster1", "user1",
+        "some_flow_name", 1002345678919L, "application_1231111111_1111",
+        "world", null), new TimelineEntityFilters.Builder().build(),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        100, CURRENT_TIME - 40000, CURRENT_TIME));
+    assertEquals(3, entities.size());
+    metricCnt = 0;
+    metricTimeSeriesCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric m : entity.getMetrics()) {
+        for (Long ts : m.getValues().keySet()) {
+          assertTrue(ts >= CURRENT_TIME - 40000 && ts <= CURRENT_TIME);
+        }
+        metricTimeSeriesCnt += m.getValues().size();
+      }
+    }
+    assertEquals(3, metricCnt);
+    assertEquals(5, metricTimeSeriesCnt);
+
+    TimelineEntity entity = reader.getEntity(new TimelineReaderContext(
+        "cluster1", "user1", "some_flow_name", 1002345678919L,
+        "application_1231111111_1111", "world", "hello"),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS), 100,
+        CURRENT_TIME - 40000, CURRENT_TIME));
+    assertNotNull(entity);
+    assertEquals(2, entity.getMetrics().size());
+    metricTimeSeriesCnt = 0;
+    for (TimelineMetric m : entity.getMetrics()) {
+      for (Long ts : m.getValues().keySet()) {
+        assertTrue(ts >= CURRENT_TIME - 40000 && ts <= CURRENT_TIME);
+      }
+      metricTimeSeriesCnt += m.getValues().size();
+    }
+    assertEquals(3, metricTimeSeriesCnt);
+  }
+
+  @Test
   public void testReadEntitiesMetricFilters() throws Exception {
     TimelineFilterList list1 = new TimelineFilterList();
     list1.addFilter(new TimelineCompareFilter(
@@ -1396,7 +1465,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
-        null));
+        null, null, null));
     assertEquals(2, entities.size());
     int metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1409,7 +1478,8 @@ public class TestHBaseTimelineStorageEntities {
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     assertEquals(2, entities.size());
     metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1428,7 +1498,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
-        null));
+        null, null, null));
     assertEquals(1, entities.size());
     metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1447,7 +1517,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList2)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
-        null));
+        null, null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList metricFilterList3 = new TimelineFilterList(
@@ -1459,7 +1529,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList3)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
-        null));
+        null, null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList metricFilterList4 = new TimelineFilterList(
@@ -1471,7 +1541,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList4)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
-        null));
+        null, null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList metricFilterList5 = new TimelineFilterList(
@@ -1483,7 +1553,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList5)
             .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
-        null));
+        null, null, null));
     assertEquals(3, entities.size());
   }
 
@@ -1500,7 +1570,7 @@ public class TestHBaseTimelineStorageEntities {
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
             .build(),
-        new TimelineDataToRetrieve(null, list, null, null));
+        new TimelineDataToRetrieve(null, list, null, null, null, null));
     assertEquals(1, entities.size());
     int metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1530,7 +1600,7 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
             .build(),
         new TimelineDataToRetrieve(
-        null, metricsToRetrieve, EnumSet.of(Field.METRICS), null));
+        null, metricsToRetrieve, EnumSet.of(Field.METRICS), null, null, null));
     assertEquals(2, entities.size());
     metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1549,8 +1619,8 @@ public class TestHBaseTimelineStorageEntities {
         "world", null),
         new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
             .build(),
-        new TimelineDataToRetrieve(null,
-        metricsToRetrieve, EnumSet.of(Field.METRICS), Integer.MAX_VALUE));
+        new TimelineDataToRetrieve(null, metricsToRetrieve,
+        EnumSet.of(Field.METRICS), Integer.MAX_VALUE, null, null));
     assertEquals(2, entities.size());
     metricCnt = 0;
     int metricValCnt = 0;
@@ -1584,7 +1654,8 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().infoFilters(infoFilterList).build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null,
+        null, null));
     assertEquals(2, entities.size());
     int infoCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1600,7 +1671,8 @@ public class TestHBaseTimelineStorageEntities {
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().infoFilters(infoFilterList1)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null,
+        null, null));
     assertEquals(1, entities.size());
     infoCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -1618,7 +1690,8 @@ public class TestHBaseTimelineStorageEntities {
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().infoFilters(infoFilterList2)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null,
+        null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList infoFilterList3 = new TimelineFilterList(
@@ -1629,7 +1702,8 @@ public class TestHBaseTimelineStorageEntities {
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().infoFilters(infoFilterList3)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null,
+        null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList infoFilterList4 = new TimelineFilterList(
@@ -1640,7 +1714,8 @@ public class TestHBaseTimelineStorageEntities {
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().infoFilters(infoFilterList4)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null,
+        null, null));
     assertEquals(0, entities.size());
 
     TimelineFilterList infoFilterList5 = new TimelineFilterList(
@@ -1651,7 +1726,8 @@ public class TestHBaseTimelineStorageEntities {
         1002345678919L, "application_1231111111_1111", "world", null),
         new TimelineEntityFilters.Builder().infoFilters(infoFilterList5)
             .build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null,
+        null, null));
     assertEquals(3, entities.size());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.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/storage/flow/TestHBaseStorageFlowRun.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/storage/flow/TestHBaseStorageFlowRun.java
index e1309e7..acfdc4d 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/storage/flow/TestHBaseStorageFlowRun.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/storage/flow/TestHBaseStorageFlowRun.java
@@ -584,7 +584,8 @@ public class TestHBaseStorageFlowRun {
       TimelineEntity entity = hbr.getEntity(
           new TimelineReaderContext(cluster, user, flow, 1002345678919L, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineDataToRetrieve(null, metricsToRetrieve, null, null));
+          new TimelineDataToRetrieve(null, metricsToRetrieve, null, null, null,
+          null));
       assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
       Set<TimelineMetric> metrics = entity.getMetrics();
       assertEquals(1, metrics.size());
@@ -609,7 +610,8 @@ public class TestHBaseStorageFlowRun {
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
           new TimelineEntityFilters.Builder().build(),
-          new TimelineDataToRetrieve(null, metricsToRetrieve, null, null));
+          new TimelineDataToRetrieve(null, metricsToRetrieve, null, null, null,
+          null));
       assertEquals(2, entities.size());
       int metricCnt = 0;
       for (TimelineEntity timelineEntity : entities) {
@@ -681,7 +683,7 @@ public class TestHBaseStorageFlowRun {
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
           new TimelineEntityFilters.Builder().build(),
           new TimelineDataToRetrieve(null, null,
-              EnumSet.of(Field.METRICS), null));
+              EnumSet.of(Field.METRICS), null, null, null));
       assertEquals(1, entities.size());
       for (TimelineEntity timelineEntity : entities) {
         Set<TimelineMetric> timelineMetrics = timelineEntity.getMetrics();
@@ -948,7 +950,7 @@ public class TestHBaseStorageFlowRun {
           new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
               .build(),
           new TimelineDataToRetrieve(null, null,
-          EnumSet.of(Field.METRICS), null));
+          EnumSet.of(Field.METRICS), null, null, null));
       assertEquals(2, entities.size());
       int metricCnt = 0;
       for (TimelineEntity entity : entities) {
@@ -966,7 +968,7 @@ public class TestHBaseStorageFlowRun {
           new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
               .build(),
           new TimelineDataToRetrieve(null, null,
-          EnumSet.of(Field.METRICS), null));
+          EnumSet.of(Field.METRICS), null, null, null));
       assertEquals(1, entities.size());
       metricCnt = 0;
       for (TimelineEntity entity : entities) {
@@ -983,7 +985,7 @@ public class TestHBaseStorageFlowRun {
           new TimelineEntityFilters.Builder().metricFilters(metricFilterList2)
               .build(),
           new TimelineDataToRetrieve(null, null,
-          EnumSet.of(Field.METRICS), null));
+          EnumSet.of(Field.METRICS), null, null, null));
       assertEquals(0, entities.size());
 
       TimelineFilterList metricFilterList3 = new TimelineFilterList(
@@ -994,7 +996,7 @@ public class TestHBaseStorageFlowRun {
           new TimelineEntityFilters.Builder().metricFilters(metricFilterList3)
               .build(),
           new TimelineDataToRetrieve(null, null,
-          EnumSet.of(Field.METRICS), null));
+          EnumSet.of(Field.METRICS), null, null, null));
       assertEquals(0, entities.size());
 
       TimelineFilterList list3 = new TimelineFilterList();
@@ -1016,7 +1018,7 @@ public class TestHBaseStorageFlowRun {
           new TimelineEntityFilters.Builder().metricFilters(metricFilterList4)
               .build(),
           new TimelineDataToRetrieve(null, metricsToRetrieve,
-          EnumSet.of(Field.ALL), null));
+          EnumSet.of(Field.ALL), null, null, null));
       assertEquals(2, entities.size());
       metricCnt = 0;
       for (TimelineEntity entity : entities) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnHelper.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/common/ColumnHelper.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/common/ColumnHelper.java
index b9815eb..496b0ae 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/common/ColumnHelper.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/common/ColumnHelper.java
@@ -190,7 +190,6 @@ public class ColumnHelper<T> {
 
       NavigableMap<byte[], NavigableMap<Long, byte[]>> columnCellMap =
           resultMap.get(columnFamilyBytes);
-
       // could be that there is no such column family.
       if (columnCellMap != null) {
         for (Entry<byte[], NavigableMap<Long, byte[]>> entry : columnCellMap

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.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/common/HBaseTimelineStorageUtils.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/common/HBaseTimelineStorageUtils.java
index 865a70d..d3ef847 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/common/HBaseTimelineStorageUtils.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/common/HBaseTimelineStorageUtils.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.client.Query;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
@@ -308,4 +309,20 @@ public final class HBaseTimelineStorageUtils {
     return (obj instanceof Short) || (obj instanceof Integer) ||
         (obj instanceof Long);
   }
+
+  public static void setMetricsTimeRange(Query query, byte[] metricsCf,
+      long tsBegin, long tsEnd) {
+    if (tsBegin != 0 || tsEnd != Long.MAX_VALUE) {
+      long supplementedTsBegin = tsBegin == 0 ? 0 :
+          TimestampGenerator.getSupplementedTimestamp(tsBegin, null);
+      long supplementedTsEnd =
+          (tsEnd == Long.MAX_VALUE) ? Long.MAX_VALUE :
+          TimestampGenerator.getSupplementedTimestamp(tsEnd + 1, null);
+      // Handle overflow by resetting time begin to 0 and time end to
+      // Long#MAX_VALUE, if required.
+      query.setColumnFamilyTimeRange(metricsCf,
+          ((supplementedTsBegin < 0) ? 0 : supplementedTsBegin),
+          ((supplementedTsEnd < 0) ? Long.MAX_VALUE : supplementedTsEnd));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.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/ApplicationEntityReader.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/ApplicationEntityReader.java
index b4bb005..cda4510 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/ApplicationEntityReader.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/ApplicationEntityReader.java
@@ -24,6 +24,7 @@ import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Get;
+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;
@@ -315,6 +316,8 @@ class ApplicationEntityReader extends GenericEntityReader {
             context.getFlowName(), context.getFlowRunId(), context.getAppId());
     byte[] rowKey = applicationRowKey.getRowKey();
     Get get = new Get(rowKey);
+    // Set time range for metric values.
+    setMetricsTimeRange(get);
     get.setMaxVersions(getDataToRetrieve().getMetricsLimit());
     if (filterList != null && !filterList.getFilters().isEmpty()) {
       get.setFilter(filterList);
@@ -357,6 +360,14 @@ class ApplicationEntityReader extends GenericEntityReader {
     }
   }
 
+  private void setMetricsTimeRange(Query query) {
+    // Set time range for metric values.
+    HBaseTimelineStorageUtils.
+        setMetricsTimeRange(query, ApplicationColumnFamily.METRICS.getBytes(),
+            getDataToRetrieve().getMetricsTimeBegin(),
+            getDataToRetrieve().getMetricsTimeEnd());
+  }
+
   @Override
   protected ResultScanner getResults(Configuration hbaseConf,
       Connection conn, FilterList filterList) throws IOException {
@@ -405,6 +416,9 @@ class ApplicationEntityReader extends GenericEntityReader {
       newList.addFilter(filterList);
     }
     scan.setFilter(newList);
+
+    // Set time range for metric values.
+    setMetricsTimeRange(scan);
     scan.setMaxVersions(getDataToRetrieve().getMetricsLimit());
     return getTable().getResultScanner(hbaseConf, conn, scan);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/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 39013d9..6b740e2 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
@@ -26,6 +26,7 @@ import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Get;
+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;
@@ -434,8 +435,8 @@ class GenericEntityReader extends TimelineEntityReader {
           context.getUserId(), context.getFlowName(), context.getFlowRunId(),
           context.getAppId(), context.getEntityType(),
           context.getEntityIdPrefix(), context.getEntityId()).getRowKey();
-
       Get get = new Get(rowKey);
+      setMetricsTimeRange(get);
       get.setMaxVersions(getDataToRetrieve().getMetricsLimit());
       if (filterList != null && !filterList.getFilters().isEmpty()) {
         get.setFilter(filterList);
@@ -468,6 +469,14 @@ class GenericEntityReader extends TimelineEntityReader {
     return result;
   }
 
+  private void setMetricsTimeRange(Query query) {
+    // Set time range for metric values.
+    HBaseTimelineStorageUtils.
+        setMetricsTimeRange(query, EntityColumnFamily.METRICS.getBytes(),
+            getDataToRetrieve().getMetricsTimeBegin(),
+            getDataToRetrieve().getMetricsTimeEnd());
+  }
+
   @Override
   protected ResultScanner getResults(Configuration hbaseConf, Connection conn,
       FilterList filterList) throws IOException {
@@ -513,6 +522,7 @@ class GenericEntityReader extends TimelineEntityReader {
       // mode.
       filterList.addFilter(new PageFilter(getFilters().getLimit()));
     }
+    setMetricsTimeRange(scan);
     scan.setMaxVersions(getDataToRetrieve().getMetricsLimit());
     if (filterList != null && !filterList.getFilters().isEmpty()) {
       scan.setFilter(filterList);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.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/TimelineDataToRetrieve.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java
index 325050a..8d09c00 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java
@@ -57,6 +57,11 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Fiel
  * metricsToRetrieve is specified, this limit defines an upper limit to the
  * number of metrics to return. This parameter is ignored if METRICS are not to
  * be fetched.</li>
+ * <li><b>metricsTimeStart</b> - Metric values before this timestamp would not
+ * be retrieved. If null or {@literal <0}, defaults to 0.</li>
+ * <li><b>metricsTimeEnd</b> - Metric values after this timestamp would not
+ * be retrieved. If null or {@literal <0}, defaults to {@link Long#MAX_VALUE}.
+ * </li>
  * </ul>
  */
 @Private
@@ -66,6 +71,10 @@ public class TimelineDataToRetrieve {
   private TimelineFilterList metricsToRetrieve;
   private EnumSet<Field> fieldsToRetrieve;
   private Integer metricsLimit;
+  private Long metricsTimeBegin;
+  private Long metricsTimeEnd;
+  private static final long DEFAULT_METRICS_BEGIN_TIME = 0L;
+  private static final long DEFAULT_METRICS_END_TIME = Long.MAX_VALUE;
 
   /**
    * Default limit of number of metrics to return.
@@ -73,12 +82,12 @@ public class TimelineDataToRetrieve {
   public static final Integer DEFAULT_METRICS_LIMIT = 1;
 
   public TimelineDataToRetrieve() {
-    this(null, null, null, null);
+    this(null, null, null, null, null, null);
   }
 
   public TimelineDataToRetrieve(TimelineFilterList confs,
       TimelineFilterList metrics, EnumSet<Field> fields,
-      Integer limitForMetrics) {
+      Integer limitForMetrics, Long metricTimeBegin, Long metricTimeEnd) {
     this.confsToRetrieve = confs;
     this.metricsToRetrieve = metrics;
     this.fieldsToRetrieve = fields;
@@ -91,6 +100,20 @@ public class TimelineDataToRetrieve {
     if (this.fieldsToRetrieve == null) {
       this.fieldsToRetrieve = EnumSet.noneOf(Field.class);
     }
+    if (metricTimeBegin == null || metricTimeBegin < 0) {
+      this.metricsTimeBegin = DEFAULT_METRICS_BEGIN_TIME;
+    } else {
+      this.metricsTimeBegin = metricTimeBegin;
+    }
+    if (metricTimeEnd == null || metricTimeEnd < 0) {
+      this.metricsTimeEnd = DEFAULT_METRICS_END_TIME;
+    } else {
+      this.metricsTimeEnd = metricTimeEnd;
+    }
+    if (this.metricsTimeBegin > this.metricsTimeEnd) {
+      throw new IllegalArgumentException("metricstimebegin should not be " +
+          "greater than metricstimeend");
+    }
   }
 
   public TimelineFilterList getConfsToRetrieve() {
@@ -137,6 +160,14 @@ public class TimelineDataToRetrieve {
     return metricsLimit;
   }
 
+  public Long getMetricsTimeBegin() {
+    return this.metricsTimeBegin;
+  }
+
+  public Long getMetricsTimeEnd() {
+    return metricsTimeEnd;
+  }
+
   public void setMetricsLimit(Integer limit) {
     if (limit == null || limit < 1) {
       this.metricsLimit = DEFAULT_METRICS_LIMIT;


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


[06/50] [abbrv] hadoop git commit: YARN-6159. Documentation changes for TimelineV2Client (Naganarasimha G R via Varun Saxena)

Posted by va...@apache.org.
YARN-6159. Documentation changes for TimelineV2Client (Naganarasimha G R via Varun Saxena)

(cherry picked from commit 6ba61d20d3f65e40ea8e3a49d5beebe34f04aab4)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 4aad6683f7b9936089679f7dd16c838a13f8f16c
Parents: d573ce4
Author: Varun Saxena <va...@apache.org>
Authored: Tue Feb 21 12:25:37 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:44 2017 +0530

----------------------------------------------------------------------
 .../src/site/markdown/TimelineServiceV2.md      | 44 ++++++++------------
 1 file changed, 18 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aad6683/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
index 02b0562..7d36a4a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
@@ -340,56 +340,48 @@ To write MapReduce framework data to Timeline Service v.2, enable the following
 
 This section is for YARN application developers that want to integrate with Timeline Service v.2.
 
-Developers can continue to use the `TimelineClient` API to publish per-framework data to the
-Timeline Service v.2. You only need to instantiate the right type of the client to write to v.2.
-On the other hand, the entity/object API for v.2 is different than v.1 as the object model is
-significantly changed. The v.2 timeline entity class is
-`org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity` whereas the v.1 class is
-`org.apache.hadoop.yarn.api.records.timeline.TimelineEntity`. The methods on `TimelineClient`
-suitable for writing to Timeline Service v.2 are clearly delineated, and they use the v.2
-types as arguments.
+Developers need to use the `TimelineV2Client` API to publish per-framework data to the
+Timeline Service v.2. The entity/object API for v.2 is different than v.1 as
+the object model is significantly changed. The v.2 timeline entity class is
+`org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity`.
 
 Timeline Service v.2 `putEntities` methods come in 2 varieties: `putEntities` and
 `putEntitiesAsync`. The former is a blocking operation which must be used for writing more
 critical data (e.g. lifecycle events). The latter is a non-blocking operation. Note that neither
 has a return value.
 
-Creating a `TimelineClient` for v.2 involves passing in the application id to the factory method.
+Creating a `TimelineV2Client` involves passing in the application id to the static method
+`TimelineV2Client.createTimelineClient`.
 
 For example:
 
 
     // Create and start the Timeline client v.2
-    TimelineClient client = TimelineClient.createTimelineClient(appId);
-    client.init(conf);
-    client.start();
+    TimelineV2Client timelineClient =
+        TimelineV2Client.createTimelineClient(appId);
+    timelineClient.init(conf);
+    timelineClient.start();
 
     try {
       TimelineEntity myEntity = new TimelineEntity();
-      myEntity.setEntityType("MY_APPLICATION");
-      myEntity.setEntityId("MyApp1")
+      myEntity.setType("MY_APPLICATION");
+      myEntity.setId("MyApp1");
       // Compose other entity info
 
       // Blocking write
-      client.putEntities(entity);
+      timelineClient.putEntities(myEntity);
 
       TimelineEntity myEntity2 = new TimelineEntity();
       // Compose other info
 
       // Non-blocking write
-      timelineClient.putEntitiesAsync(entity);
+      timelineClient.putEntitiesAsync(myEntity2);
 
-    } catch (IOException e) {
-      // Handle the exception
-    } catch (RuntimeException e) {
-      // In Hadoop 2.6, if attempts submit information to the Timeline Server fail more than the retry limit,
-      // a RuntimeException will be raised. This may change in future releases, being
-      // replaced with a IOException that is (or wraps) that which triggered retry failures.
-    } catch (YarnException e) {
+    } catch (IOException | YarnException e) {
       // Handle the exception
     } finally {
       // Stop the Timeline client
-      client.stop();
+      timelineClient.stop();
     }
 
 As evidenced above, you need to specify the YARN application id to be able to write to the Timeline
@@ -397,9 +389,9 @@ Service v.2. Note that currently you need to be on the cluster to be able to wri
 Service. For example, an application master or code in the container can write to the Timeline
 Service, while an off-cluster MapReduce job submitter cannot.
 
-After creating the timeline client, user also needs to set the timeline collector address for the application. If `AMRMClient` is used then by registering the timeline client by calling `AMRMClient#registerTimelineClient` is sufficient.
+After creating the timeline v2 client, user also needs to set the timeline collector address for the application. If `AMRMClient` is used then by registering the timeline client by calling `AMRMClient#registerTimelineV2Client` is sufficient.
 
-    amRMClient.registerTimelineClient(timelineClient);
+    amRMClient.registerTimelineV2Client(timelineClient);
 
 Else address needs to be retrieved from the AM allocate response and need to be set in timeline client explicitly.
 


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


[20/50] [abbrv] hadoop git commit: YARN-6146. Add Builder methods for TimelineEntityFilters (Haibo Chen via Varun Saxena)

Posted by va...@apache.org.
YARN-6146. Add Builder methods for TimelineEntityFilters (Haibo Chen via Varun Saxena)

(cherry picked from commit 7f09cd1d5ee70e0bdf24cdecd78cd3bc258a40f8)


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

Branch: refs/heads/YARN-5355_branch2
Commit: e59486a376e41757350b9d387026a537fb01b52c
Parents: 7e6fdeb
Author: Varun Saxena <va...@apache.org>
Authored: Thu Mar 23 14:35:37 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:45 2017 +0530

----------------------------------------------------------------------
 .../storage/TestHBaseTimelineStorageApps.java   | 190 ++++++++---------
 .../TestHBaseTimelineStorageEntities.java       | 202 +++++++++----------
 .../flow/TestHBaseStorageFlowActivity.java      |   9 +-
 .../storage/flow/TestHBaseStorageFlowRun.java   |  48 +++--
 .../storage/reader/TimelineEntityReader.java    |   2 +-
 .../reader/TimelineEntityFilters.java           | 160 ++++++++-------
 .../reader/TimelineReaderWebServices.java       |   6 +-
 .../reader/TimelineReaderWebServicesUtils.java  |  44 +++-
 .../TestFileSystemTimelineReaderImpl.java       | 100 +++++----
 9 files changed, 381 insertions(+), 380 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e59486a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.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/storage/TestHBaseTimelineStorageApps.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/storage/TestHBaseTimelineStorageApps.java
index 6bb8de1..321b72e 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/storage/TestHBaseTimelineStorageApps.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/storage/TestHBaseTimelineStorageApps.java
@@ -658,7 +658,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(),
+        new TimelineEntityFilters.Builder().build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(3, entities.size());
     int cfgCnt = 0;
@@ -695,8 +695,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, 1425016502000L, 1425016502040L, null,
-        null, null, null, null, null),
+        new TimelineEntityFilters.Builder().createdTimeBegin(1425016502000L)
+            .createTimeEnd(1425016502040L).build(),
         new TimelineDataToRetrieve());
     assertEquals(3, entities.size());
     for (TimelineEntity entity : entities) {
@@ -712,8 +712,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, 1425016502015L, null, null, null, null,
-        null, null, null),
+        new TimelineEntityFilters.Builder().createdTimeBegin(1425016502015L)
+            .build(),
         new TimelineDataToRetrieve());
     assertEquals(2, entities.size());
     for (TimelineEntity entity : entities) {
@@ -727,8 +727,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, 1425016502015L, null, null, null,
-        null, null, null),
+        new TimelineEntityFilters.Builder().createTimeEnd(1425016502015L)
+            .build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     for (TimelineEntity entity : entities) {
@@ -755,7 +755,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(),
+        new TimelineEntityFilters.Builder().build(),
         new TimelineDataToRetrieve());
     assertEquals(3, es1.size());
     for (TimelineEntity e : es1) {
@@ -781,7 +781,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(),
+        new TimelineEntityFilters.Builder().build(),
         new TimelineDataToRetrieve(
         null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS), null));
     assertEquals(3, es1.size());
@@ -811,8 +811,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt).build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     int isRelatedToCnt = 0;
@@ -837,8 +836,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt1, null, null,
-        null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt1).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     isRelatedToCnt = 0;
@@ -861,8 +859,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt2, null, null,
-        null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt2).build(),
         new TimelineDataToRetrieve());
     assertEquals(2, entities.size());
     isRelatedToCnt = 0;
@@ -884,8 +881,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt3, null, null,
-        null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt3).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     isRelatedToCnt = 0;
@@ -908,8 +904,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt4, null, null,
-        null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt4).build(),
         new TimelineDataToRetrieve());
     assertEquals(0, entities.size());
 
@@ -921,8 +916,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt5, null, null,
-        null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt5).build(),
         new TimelineDataToRetrieve());
     assertEquals(0, entities.size());
 
@@ -942,8 +936,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, irt6, null, null,
-        null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt6).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     isRelatedToCnt = 0;
@@ -970,8 +963,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, rt, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt).build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     int relatesToCnt = 0;
@@ -996,8 +988,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, rt1, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt1).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     relatesToCnt = 0;
@@ -1020,8 +1011,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, rt2, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt2).build(),
         new TimelineDataToRetrieve());
     assertEquals(2, entities.size());
     relatesToCnt = 0;
@@ -1043,8 +1033,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, rt3, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt3).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     relatesToCnt = 0;
@@ -1067,8 +1056,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, rt4, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt4).build(),
         new TimelineDataToRetrieve());
     assertEquals(0, entities.size());
 
@@ -1080,8 +1068,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, rt5, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt5).build(),
         new TimelineDataToRetrieve());
     assertEquals(0, entities.size());
 
@@ -1101,8 +1088,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, rt6, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt6).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     relatesToCnt = 0;
@@ -1138,8 +1124,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, rt7, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt7).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     relatesToCnt = 0;
@@ -1176,8 +1161,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, relatesTo, isRelatedTo,
-        null, null, null, eventFilter),
+        new TimelineEntityFilters.Builder().relatesTo(relatesTo)
+            .isRelatedTo(isRelatedTo).eventFilters(eventFilter).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     int eventCnt = 0;
@@ -1214,8 +1199,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
         null));
     assertEquals(2, entities.size());
@@ -1229,8 +1214,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     cfgCnt = 0;
@@ -1246,8 +1231,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList1, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList1)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
         null));
     assertEquals(1, entities.size());
@@ -1266,8 +1251,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList2, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList2)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
         null));
     assertEquals(0, entities.size());
@@ -1279,8 +1264,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList3, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList3)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
         null));
     assertEquals(0, entities.size());
@@ -1292,8 +1277,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList4, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList4)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
         null));
     assertEquals(0, entities.size());
@@ -1305,8 +1290,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList5, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList5)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
         null));
     assertEquals(3, entities.size());
@@ -1323,8 +1308,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef),
+        new TimelineEntityFilters.Builder().eventFilters(ef).build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(1, entities.size());
     int eventCnt = 0;
@@ -1345,8 +1329,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef1), new TimelineDataToRetrieve());
+        new TimelineEntityFilters.Builder().eventFilters(ef1).build(),
+        new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     eventCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -1364,8 +1348,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef2),
+        new TimelineEntityFilters.Builder().eventFilters(ef2).build(),
         new TimelineDataToRetrieve());
     assertEquals(2, entities.size());
     eventCnt = 0;
@@ -1388,8 +1371,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef3),
+        new TimelineEntityFilters.Builder().eventFilters(ef3).build(),
         new TimelineDataToRetrieve());
     assertEquals(0, entities.size());
 
@@ -1406,8 +1388,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef4),
+        new TimelineEntityFilters.Builder().eventFilters(ef4).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     eventCnt = 0;
@@ -1428,8 +1409,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef5),
+        new TimelineEntityFilters.Builder().eventFilters(ef5).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     eventCnt = 0;
@@ -1458,7 +1438,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null) ,
-        new TimelineEntityFilters(),
+        new TimelineEntityFilters.Builder().build(),
         new TimelineDataToRetrieve(list, null, null, null));
     int cfgCnt = 0;
     for (TimelineEntity entity : es1) {
@@ -1483,8 +1463,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList)
+            .build(),
         new TimelineDataToRetrieve(list, null, null, null));
     assertEquals(1, entities.size());
     int cfgCnt = 0;
@@ -1516,8 +1496,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList1, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList1)
+            .build(),
         new TimelineDataToRetrieve(confsToRetrieve, null, null, null));
     assertEquals(2, entities.size());
     cfgCnt = 0;
@@ -1547,8 +1527,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
         null));
     assertEquals(2, entities.size());
@@ -1562,8 +1542,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     metricCnt = 0;
@@ -1581,8 +1561,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList1, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
         null));
     assertEquals(1, entities.size());
@@ -1601,8 +1581,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList2, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList2)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
         null));
     assertEquals(0, entities.size());
@@ -1614,8 +1594,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList3, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList3)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
         null));
     assertEquals(0, entities.size());
@@ -1627,8 +1607,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList4, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList4)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
         null));
     assertEquals(0, entities.size());
@@ -1640,8 +1620,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList5, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList5)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
         null));
     assertEquals(3, entities.size());
@@ -1663,7 +1643,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(),
+        new TimelineEntityFilters.Builder().build(),
         new TimelineDataToRetrieve(null, list, null, null));
     int metricCnt = 0;
     for (TimelineEntity entity : es1) {
@@ -1688,8 +1668,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
+            .build(),
         new TimelineDataToRetrieve(null, list, null, null));
     int metricCnt = 0;
     assertEquals(1, entities.size());
@@ -1714,8 +1694,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList1, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
+            .build(),
         new TimelineDataToRetrieve(null, metricsToRetrieve, null, null));
     metricCnt = 0;
     assertEquals(2, entities.size());
@@ -1731,8 +1711,9 @@ public class TestHBaseTimelineStorageApps {
     entities = reader.getEntities(new TimelineReaderContext("cluster1", "user1",
         "some_flow_name", 1002345678919L, null,
         TimelineEntityType.YARN_APPLICATION.toString(), null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList1, null), new TimelineDataToRetrieve(null,
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
+            .build(),
+        new TimelineDataToRetrieve(null,
         metricsToRetrieve, EnumSet.of(Field.METRICS), Integer.MAX_VALUE));
     metricCnt = 0;
     int metricValCnt = 0;
@@ -1767,8 +1748,7 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList).build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(2, entities.size());
     int infoCnt = 0;
@@ -1784,8 +1764,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList1,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList1)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(1, entities.size());
     infoCnt = 0;
@@ -1803,8 +1783,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList2,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList2)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(0, entities.size());
 
@@ -1815,8 +1795,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList3,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList3)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(0, entities.size());
 
@@ -1827,8 +1807,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList4,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList4)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(0, entities.size());
 
@@ -1839,8 +1819,8 @@ public class TestHBaseTimelineStorageApps {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList5,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList5)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(3, entities.size());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e59486a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.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/storage/TestHBaseTimelineStorageEntities.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/storage/TestHBaseTimelineStorageEntities.java
index 7211450..9b47381 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/storage/TestHBaseTimelineStorageEntities.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/storage/TestHBaseTimelineStorageEntities.java
@@ -298,7 +298,7 @@ public class TestHBaseTimelineStorageEntities {
       Set<TimelineEntity> es1 = reader.getEntities(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), null),
-          new TimelineEntityFilters(),
+          new TimelineEntityFilters.Builder().build(),
           new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL),
           Integer.MAX_VALUE));
       assertNotNull(e1);
@@ -453,7 +453,7 @@ public class TestHBaseTimelineStorageEntities {
       Set<TimelineEntity> es1 = reader.getEntities(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), null),
-          new TimelineEntityFilters(),
+          new TimelineEntityFilters.Builder().build(),
           new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
       assertNotNull(e1);
       assertEquals(1, es1.size());
@@ -549,7 +549,7 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world",
-        null), new TimelineEntityFilters(),
+        null), new TimelineEntityFilters.Builder().build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(3, entities.size());
     int cfgCnt = 0;
@@ -585,8 +585,9 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, 1425016502000L, 1425016502040L, null,
-        null, null, null, null, null), new TimelineDataToRetrieve());
+        new TimelineEntityFilters.Builder().createdTimeBegin(1425016502000L)
+            .createTimeEnd(1425016502040L).build(),
+        new TimelineDataToRetrieve());
     assertEquals(3, entities.size());
     for (TimelineEntity entity : entities) {
       if (!entity.getId().equals("hello") && !entity.getId().equals("hello1") &&
@@ -598,8 +599,9 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, 1425016502015L, null, null, null, null,
-        null, null, null), new TimelineDataToRetrieve());
+        new TimelineEntityFilters.Builder().createdTimeBegin(1425016502015L)
+            .build(),
+        new TimelineDataToRetrieve());
     assertEquals(2, entities.size());
     for (TimelineEntity entity : entities) {
       if (!entity.getId().equals("hello1") &&
@@ -610,8 +612,9 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world",  null),
-        new TimelineEntityFilters(null, null, 1425016502015L, null, null, null,
-        null, null, null), new TimelineDataToRetrieve());
+        new TimelineEntityFilters.Builder().createTimeEnd(1425016502015L)
+            .build(),
+        new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     for (TimelineEntity entity : entities) {
       if (!entity.getId().equals("hello")) {
@@ -643,8 +646,9 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, relatesTo, isRelatedTo,
-        null, null, null, eventFilter), new TimelineDataToRetrieve());
+        new TimelineEntityFilters.Builder().relatesTo(relatesTo)
+            .isRelatedTo(isRelatedTo).eventFilters(eventFilter).build(),
+        new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     int eventCnt = 0;
     int isRelatedToCnt = 0;
@@ -672,8 +676,7 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef),
+        new TimelineEntityFilters.Builder().eventFilters(ef).build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(1, entities.size());
     int eventCnt = 0;
@@ -693,8 +696,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef1),
+        new TimelineEntityFilters.Builder().eventFilters(ef1).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     eventCnt = 0;
@@ -712,8 +714,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef2),
+        new TimelineEntityFilters.Builder().eventFilters(ef2).build(),
         new TimelineDataToRetrieve());
     assertEquals(2, entities.size());
     eventCnt = 0;
@@ -734,8 +735,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef3),
+        new TimelineEntityFilters.Builder().eventFilters(ef3).build(),
         new TimelineDataToRetrieve());
     assertEquals(0, entities.size());
 
@@ -751,8 +751,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef4),
+        new TimelineEntityFilters.Builder().eventFilters(ef4).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     eventCnt = 0;
@@ -772,8 +771,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef5),
+        new TimelineEntityFilters.Builder().eventFilters(ef5).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     eventCnt = 0;
@@ -798,8 +796,7 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt).build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     int isRelatedToCnt = 0;
@@ -822,8 +819,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt1, null, null,
-        null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt1).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     isRelatedToCnt = 0;
@@ -845,8 +841,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt2, null, null,
-        null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt2).build(),
         new TimelineDataToRetrieve());
     assertEquals(2, entities.size());
     isRelatedToCnt = 0;
@@ -866,8 +861,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt3, null, null,
-        null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt3).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     isRelatedToCnt = 0;
@@ -889,8 +883,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt4, null, null,
-        null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt4).build(),
         new TimelineDataToRetrieve());
     assertEquals(0, entities.size());
 
@@ -901,8 +894,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt5, null, null,
-        null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt5).build(),
         new TimelineDataToRetrieve());
     assertEquals(0, entities.size());
 
@@ -921,8 +913,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt6, null, null,
-        null, null),
+        new TimelineEntityFilters.Builder().isRelatedTo(irt6).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     isRelatedToCnt = 0;
@@ -947,8 +938,7 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt).build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     int relatesToCnt = 0;
@@ -971,8 +961,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt1, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt1).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     relatesToCnt = 0;
@@ -994,8 +983,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt2, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt2).build(),
         new TimelineDataToRetrieve());
     assertEquals(2, entities.size());
     relatesToCnt = 0;
@@ -1015,8 +1003,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt3, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt3).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     relatesToCnt = 0;
@@ -1038,8 +1025,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt4, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt4).build(),
         new TimelineDataToRetrieve());
     assertEquals(0, entities.size());
 
@@ -1050,8 +1036,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt5, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt5).build(),
         new TimelineDataToRetrieve());
     assertEquals(0, entities.size());
 
@@ -1070,8 +1055,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt6, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt6).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     relatesToCnt = 0;
@@ -1106,8 +1090,7 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt7, null, null, null, null,
-        null),
+        new TimelineEntityFilters.Builder().relatesTo(rt7).build(),
         new TimelineDataToRetrieve());
     assertEquals(1, entities.size());
     relatesToCnt = 0;
@@ -1134,7 +1117,7 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(),
+        new TimelineEntityFilters.Builder().build(),
         new TimelineDataToRetrieve());
     assertEquals(3, es1.size());
     for (TimelineEntity e : es1) {
@@ -1158,7 +1141,7 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(),
+        new TimelineEntityFilters.Builder().build(),
         new TimelineDataToRetrieve(
         null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS), null));
     assertEquals(3, es1.size());
@@ -1189,7 +1172,7 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(),
+        new TimelineEntityFilters.Builder().build(),
         new TimelineDataToRetrieve(list, null, null, null));
     int cfgCnt = 0;
     for (TimelineEntity entity : es1) {
@@ -1219,8 +1202,8 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
         null));
     assertEquals(2, entities.size());
@@ -1233,8 +1216,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     cfgCnt = 0;
@@ -1249,8 +1232,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList1, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList1)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
         null));
     assertEquals(1, entities.size());
@@ -1268,8 +1251,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList2, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList2)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
         null));
     assertEquals(0, entities.size());
@@ -1280,8 +1263,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList3, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList3)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
         null));
     assertEquals(0, entities.size());
@@ -1290,11 +1273,11 @@ public class TestHBaseTimelineStorageEntities {
         new TimelineKeyValueFilter(
         TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1"));
     entities = reader.getEntities(
-            new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
             1002345678919L, "application_1231111111_1111", "world", null),
-            new TimelineEntityFilters(null, null, null, null, null, null,
-            confFilterList4, null, null),
-            new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList4)
+            .build(),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
             null));
     assertEquals(0, entities.size());
 
@@ -1304,8 +1287,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList5, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList5)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
         null));
     assertEquals(3, entities.size());
@@ -1322,8 +1305,8 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList)
+            .build(),
         new TimelineDataToRetrieve(list, null, null, null));
     assertEquals(1, entities.size());
     int cfgCnt = 0;
@@ -1353,8 +1336,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList1, null, null),
+        new TimelineEntityFilters.Builder().configFilters(confFilterList1)
+            .build(),
         new TimelineDataToRetrieve(confsToRetrieve, null, null, null));
     assertEquals(2, entities.size());
     cfgCnt = 0;
@@ -1382,7 +1365,7 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(),
+        new TimelineEntityFilters.Builder().build(),
         new TimelineDataToRetrieve(null, list, null, null));
     int metricCnt = 0;
     for (TimelineEntity entity : es1) {
@@ -1410,8 +1393,8 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
         null));
     assertEquals(2, entities.size());
@@ -1424,8 +1407,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     metricCnt = 0;
@@ -1442,8 +1425,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList1, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
         null));
     assertEquals(1, entities.size());
@@ -1461,8 +1444,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList2, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList2)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
         null));
     assertEquals(0, entities.size());
@@ -1473,8 +1456,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList3, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList3)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
         null));
     assertEquals(0, entities.size());
@@ -1485,8 +1468,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList4, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList4)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
         null));
     assertEquals(0, entities.size());
@@ -1497,8 +1480,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList5, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList5)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
         null));
     assertEquals(3, entities.size());
@@ -1515,8 +1498,8 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
+            .build(),
         new TimelineDataToRetrieve(null, list, null, null));
     assertEquals(1, entities.size());
     int metricCnt = 0;
@@ -1544,8 +1527,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList1, null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
+            .build(),
         new TimelineDataToRetrieve(
         null, metricsToRetrieve, EnumSet.of(Field.METRICS), null));
     assertEquals(2, entities.size());
@@ -1563,8 +1546,10 @@ public class TestHBaseTimelineStorageEntities {
 
     entities = reader.getEntities(new TimelineReaderContext("cluster1", "user1",
         "some_flow_name", 1002345678919L, "application_1231111111_1111",
-        "world", null), new TimelineEntityFilters(null, null, null, null, null,
-        null, null, metricFilterList1, null), new TimelineDataToRetrieve(null,
+        "world", null),
+        new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
+            .build(),
+        new TimelineDataToRetrieve(null,
         metricsToRetrieve, EnumSet.of(Field.METRICS), Integer.MAX_VALUE));
     assertEquals(2, entities.size());
     metricCnt = 0;
@@ -1598,8 +1583,7 @@ public class TestHBaseTimelineStorageEntities {
     Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList).build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(2, entities.size());
     int infoCnt = 0;
@@ -1614,8 +1598,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList1,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList1)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(1, entities.size());
     infoCnt = 0;
@@ -1632,8 +1616,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList2,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList2)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(0, entities.size());
 
@@ -1643,8 +1627,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList3,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList3)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(0, entities.size());
 
@@ -1654,8 +1638,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList4,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList4)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(0, entities.size());
 
@@ -1665,8 +1649,8 @@ public class TestHBaseTimelineStorageEntities {
     entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilterList5,
-        null, null, null),
+        new TimelineEntityFilters.Builder().infoFilters(infoFilterList5)
+            .build(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(3, entities.size());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e59486a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.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/storage/flow/TestHBaseStorageFlowActivity.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/storage/flow/TestHBaseStorageFlowActivity.java
index 1af47a6..0923105 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/storage/flow/TestHBaseStorageFlowActivity.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/storage/flow/TestHBaseStorageFlowActivity.java
@@ -184,8 +184,7 @@ public class TestHBaseStorageFlowActivity {
       Set<TimelineEntity> entities = hbr.getEntities(
           new TimelineReaderContext(cluster, null, null, null, null,
           TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
-          new TimelineEntityFilters(10L, null, null, null, null, null,
-          null, null, null),
+          new TimelineEntityFilters.Builder().entityLimit(10L).build(),
           new TimelineDataToRetrieve());
       assertEquals(1, entities.size());
       for (TimelineEntity e : entities) {
@@ -249,8 +248,7 @@ public class TestHBaseStorageFlowActivity {
       Set<TimelineEntity> entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
-          new TimelineEntityFilters(10L, null, null, null, null, null,
-          null, null, null),
+          new TimelineEntityFilters.Builder().entityLimit(10L).build(),
           new TimelineDataToRetrieve());
       assertEquals(1, entities.size());
       for (TimelineEntity e : entities) {
@@ -377,8 +375,7 @@ public class TestHBaseStorageFlowActivity {
       Set<TimelineEntity> entities = hbr.getEntities(
           new TimelineReaderContext(cluster, null, null, null, null,
           TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
-          new TimelineEntityFilters(10L, null, null, null, null, null,
-          null, null, null),
+          new TimelineEntityFilters.Builder().entityLimit(10L).build(),
           new TimelineDataToRetrieve());
       assertEquals(1, entities.size());
       for (TimelineEntity e : entities) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e59486a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.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/storage/flow/TestHBaseStorageFlowRun.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/storage/flow/TestHBaseStorageFlowRun.java
index e376c6c..e1309e7 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/storage/flow/TestHBaseStorageFlowRun.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/storage/flow/TestHBaseStorageFlowRun.java
@@ -608,7 +608,7 @@ public class TestHBaseStorageFlowRun {
       Set<TimelineEntity> entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(),
+          new TimelineEntityFilters.Builder().build(),
           new TimelineDataToRetrieve(null, metricsToRetrieve, null, null));
       assertEquals(2, entities.size());
       int metricCnt = 0;
@@ -669,7 +669,7 @@ public class TestHBaseStorageFlowRun {
       Set<TimelineEntity> entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, runid, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(),
+          new TimelineEntityFilters.Builder().build(),
           new TimelineDataToRetrieve());
       assertEquals(1, entities.size());
       for (TimelineEntity timelineEntity : entities) {
@@ -679,8 +679,9 @@ public class TestHBaseStorageFlowRun {
       entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, runid, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(), new TimelineDataToRetrieve(null, null,
-          EnumSet.of(Field.METRICS), null));
+          new TimelineEntityFilters.Builder().build(),
+          new TimelineDataToRetrieve(null, null,
+              EnumSet.of(Field.METRICS), null));
       assertEquals(1, entities.size());
       for (TimelineEntity timelineEntity : entities) {
         Set<TimelineMetric> timelineMetrics = timelineEntity.getMetrics();
@@ -850,8 +851,9 @@ public class TestHBaseStorageFlowRun {
       Set<TimelineEntity> entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow,
           null, null, TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, 1425016501000L, 1425016502001L, null,
-          null, null, null, null, null), new TimelineDataToRetrieve());
+          new TimelineEntityFilters.Builder().createdTimeBegin(1425016501000L)
+              .createTimeEnd(1425016502001L).build(),
+          new TimelineDataToRetrieve());
       assertEquals(2, entities.size());
       for (TimelineEntity entity : entities) {
         if (!entity.getId().equals("user2@flow_name2/1002345678918") &&
@@ -863,8 +865,9 @@ public class TestHBaseStorageFlowRun {
       entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, 1425016501050L, null, null, null,
-          null, null, null, null), new TimelineDataToRetrieve());
+          new TimelineEntityFilters.Builder().createdTimeBegin(1425016501050L)
+              .build(),
+          new TimelineDataToRetrieve());
       assertEquals(1, entities.size());
       for (TimelineEntity entity : entities) {
         if (!entity.getId().equals("user2@flow_name2/1002345678918")) {
@@ -874,8 +877,9 @@ public class TestHBaseStorageFlowRun {
       entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, null, 1425016501050L, null, null,
-          null, null, null, null), new TimelineDataToRetrieve());
+          new TimelineEntityFilters.Builder().createTimeEnd(1425016501050L)
+              .build(),
+          new TimelineDataToRetrieve());
       assertEquals(1, entities.size());
       for (TimelineEntity entity : entities) {
         if (!entity.getId().equals("user2@flow_name2/1002345678919")) {
@@ -941,8 +945,9 @@ public class TestHBaseStorageFlowRun {
       Set<TimelineEntity> entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, null,
           null, TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList, null), new TimelineDataToRetrieve(null, null,
+          new TimelineEntityFilters.Builder().metricFilters(metricFilterList)
+              .build(),
+          new TimelineDataToRetrieve(null, null,
           EnumSet.of(Field.METRICS), null));
       assertEquals(2, entities.size());
       int metricCnt = 0;
@@ -958,8 +963,9 @@ public class TestHBaseStorageFlowRun {
       entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList1, null), new TimelineDataToRetrieve(null, null,
+          new TimelineEntityFilters.Builder().metricFilters(metricFilterList1)
+              .build(),
+          new TimelineDataToRetrieve(null, null,
           EnumSet.of(Field.METRICS), null));
       assertEquals(1, entities.size());
       metricCnt = 0;
@@ -974,8 +980,9 @@ public class TestHBaseStorageFlowRun {
       entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList2, null), new TimelineDataToRetrieve(null, null,
+          new TimelineEntityFilters.Builder().metricFilters(metricFilterList2)
+              .build(),
+          new TimelineDataToRetrieve(null, null,
           EnumSet.of(Field.METRICS), null));
       assertEquals(0, entities.size());
 
@@ -984,8 +991,9 @@ public class TestHBaseStorageFlowRun {
       entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList3, null), new TimelineDataToRetrieve(null, null,
+          new TimelineEntityFilters.Builder().metricFilters(metricFilterList3)
+              .build(),
+          new TimelineDataToRetrieve(null, null,
           EnumSet.of(Field.METRICS), null));
       assertEquals(0, entities.size());
 
@@ -1005,8 +1013,8 @@ public class TestHBaseStorageFlowRun {
       entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList4, null),
+          new TimelineEntityFilters.Builder().metricFilters(metricFilterList4)
+              .build(),
           new TimelineDataToRetrieve(null, metricsToRetrieve,
           EnumSet.of(Field.ALL), null));
       assertEquals(2, entities.size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e59486a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.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/TimelineEntityReader.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/TimelineEntityReader.java
index 4c88cd3..d1a4dc5 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/TimelineEntityReader.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/TimelineEntityReader.java
@@ -185,7 +185,7 @@ public abstract class TimelineEntityReader extends
    */
   protected void createFiltersIfNull() {
     if (filters == null) {
-      filters = new TimelineEntityFilters();
+      filters = new TimelineEntityFilters.Builder().build();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e59486a3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.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/TimelineEntityFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
index dc3e3ec..a415d34 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
@@ -107,17 +107,17 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyVa
  */
 @Private
 @Unstable
-public class TimelineEntityFilters {
-  private long limit;
+public final class TimelineEntityFilters {
+  private final long limit;
   private long createdTimeBegin;
   private long createdTimeEnd;
-  private TimelineFilterList relatesTo;
-  private TimelineFilterList isRelatedTo;
-  private TimelineFilterList infoFilters;
-  private TimelineFilterList configFilters;
-  private TimelineFilterList metricFilters;
-  private TimelineFilterList eventFilters;
-  private String fromId;
+  private final TimelineFilterList relatesTo;
+  private final TimelineFilterList isRelatedTo;
+  private final TimelineFilterList infoFilters;
+  private final TimelineFilterList configFilters;
+  private final TimelineFilterList metricFilters;
+  private final TimelineFilterList eventFilters;
+  private final String fromId;
   private static final long DEFAULT_BEGIN_TIME = 0L;
   private static final long DEFAULT_END_TIME = Long.MAX_VALUE;
 
@@ -127,30 +127,14 @@ public class TimelineEntityFilters {
    */
   public static final long DEFAULT_LIMIT = 100;
 
-  public TimelineEntityFilters() {
-    this(null, null, null, null, null, null, null, null, null);
-  }
-
-  public TimelineEntityFilters(Long entityLimit, Long timeBegin, Long timeEnd,
-      TimelineFilterList entityRelatesTo, TimelineFilterList entityIsRelatedTo,
-      TimelineFilterList entityInfoFilters,
-      TimelineFilterList entityConfigFilters,
-      TimelineFilterList entityMetricFilters,
-      TimelineFilterList entityEventFilters, String fromid) {
-    this(entityLimit, timeBegin, timeEnd, entityRelatesTo, entityIsRelatedTo,
-        entityInfoFilters, entityConfigFilters, entityMetricFilters,
-        entityEventFilters);
-    this.fromId = fromid;
-  }
-
-  public TimelineEntityFilters(
+  private TimelineEntityFilters(
       Long entityLimit, Long timeBegin, Long timeEnd,
       TimelineFilterList entityRelatesTo,
       TimelineFilterList entityIsRelatedTo,
       TimelineFilterList entityInfoFilters,
       TimelineFilterList entityConfigFilters,
       TimelineFilterList  entityMetricFilters,
-      TimelineFilterList entityEventFilters) {
+      TimelineFilterList entityEventFilters, String fromId) {
     if (entityLimit == null || entityLimit < 0) {
       this.limit = DEFAULT_LIMIT;
     } else {
@@ -172,97 +156,119 @@ public class TimelineEntityFilters {
     this.configFilters = entityConfigFilters;
     this.metricFilters = entityMetricFilters;
     this.eventFilters = entityEventFilters;
+    this.fromId = fromId;
   }
 
   public long getLimit() {
     return limit;
   }
 
-  public void setLimit(Long entityLimit) {
-    if (entityLimit == null || entityLimit < 0) {
-      this.limit = DEFAULT_LIMIT;
-    } else {
-      this.limit = entityLimit;
-    }
-  }
-
   public long getCreatedTimeBegin() {
     return createdTimeBegin;
   }
 
-  public void setCreatedTimeBegin(Long timeBegin) {
-    if (timeBegin == null || timeBegin < 0) {
-      this.createdTimeBegin = DEFAULT_BEGIN_TIME;
-    } else {
-      this.createdTimeBegin = timeBegin;
-    }
-  }
-
   public long getCreatedTimeEnd() {
     return createdTimeEnd;
   }
 
-  public void setCreatedTimeEnd(Long timeEnd) {
-    if (timeEnd == null || timeEnd < 0) {
-      this.createdTimeEnd = DEFAULT_END_TIME;
-    } else {
-      this.createdTimeEnd = timeEnd;
-    }
-  }
-
   public TimelineFilterList getRelatesTo() {
     return relatesTo;
   }
 
-  public void setRelatesTo(TimelineFilterList relations) {
-    this.relatesTo = relations;
-  }
-
   public TimelineFilterList getIsRelatedTo() {
     return isRelatedTo;
   }
 
-  public void setIsRelatedTo(TimelineFilterList relations) {
-    this.isRelatedTo = relations;
-  }
-
   public TimelineFilterList getInfoFilters() {
     return infoFilters;
   }
 
-  public void setInfoFilters(TimelineFilterList filters) {
-    this.infoFilters = filters;
-  }
-
   public TimelineFilterList getConfigFilters() {
     return configFilters;
   }
 
-  public void setConfigFilters(TimelineFilterList filters) {
-    this.configFilters = filters;
-  }
-
   public TimelineFilterList getMetricFilters() {
     return metricFilters;
   }
 
-  public void setMetricFilters(TimelineFilterList filters) {
-    this.metricFilters = filters;
-  }
-
   public TimelineFilterList getEventFilters() {
     return eventFilters;
   }
 
-  public void setEventFilters(TimelineFilterList filters) {
-    this.eventFilters = filters;
-  }
-
   public String getFromId() {
     return fromId;
   }
 
-  public void setFromId(String fromId) {
-    this.fromId = fromId;
+  /**
+   * A builder class to build an instance of TimelineEntityFilters.
+   */
+  public static class Builder {
+    private Long entityLimit;
+    private Long createdTimeBegin;
+    private Long createdTimeEnd;
+    private TimelineFilterList relatesToFilters;
+    private TimelineFilterList isRelatedToFilters;
+    private TimelineFilterList entityInfoFilters;
+    private TimelineFilterList entityConfigFilters;
+    private TimelineFilterList entityMetricFilters;
+    private TimelineFilterList entityEventFilters;
+    private String entityFromId;
+
+    public Builder entityLimit(Long limit) {
+      this.entityLimit = limit;
+      return this;
+    }
+
+    public Builder createdTimeBegin(Long timeBegin) {
+      this.createdTimeBegin = timeBegin;
+      return this;
+    }
+
+    public Builder createTimeEnd(Long timeEnd) {
+      this.createdTimeEnd = timeEnd;
+      return this;
+    }
+
+    public Builder relatesTo(TimelineFilterList relatesTo) {
+      this.relatesToFilters = relatesTo;
+      return this;
+    }
+
+    public Builder isRelatedTo(TimelineFilterList isRelatedTo) {
+      this.isRelatedToFilters = isRelatedTo;
+      return this;
+    }
+
+    public Builder infoFilters(TimelineFilterList infoFilters) {
+      this.entityInfoFilters = infoFilters;
+      return this;
+    }
+
+    public Builder configFilters(TimelineFilterList configFilters) {
+      this.entityConfigFilters = configFilters;
+      return this;
+    }
+
+    public Builder metricFilters(TimelineFilterList metricFilters) {
+      this.entityMetricFilters = metricFilters;
+      return this;
+    }
+
+    public Builder eventFilters(TimelineFilterList eventFilters) {
+      this.entityEventFilters = eventFilters;
+      return this;
+    }
+
+    public Builder fromId(String fromId) {
+      this.entityFromId = fromId;
+      return this;
+    }
+
+    public TimelineEntityFilters build() {
+      return new TimelineEntityFilters(entityLimit, createdTimeBegin,
+          createdTimeEnd, relatesToFilters, isRelatedToFilters,
+          entityInfoFilters, entityConfigFilters, entityMetricFilters,
+          entityEventFilters, entityFromId);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e59486a3/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 490a3cf..3a9e15a 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
@@ -1392,10 +1392,8 @@ public class TimelineReaderWebServices {
       DateRange range = parseDateRange(dateRange);
       TimelineEntityFilters entityFilters =
           TimelineReaderWebServicesUtils.createTimelineEntityFilters(
-              limit, null, null, null, null, null, null, null, null,
-              fromId);
-      entityFilters.setCreatedTimeBegin(range.dateStart);
-      entityFilters.setCreatedTimeEnd(range.dateEnd);
+              limit, range.dateStart, range.dateEnd,
+              null, null, null, null, null, null, fromId);
       entities = timelineReaderManager.getEntities(
           TimelineReaderWebServicesUtils.createTimelineReaderContext(
           clusterId, null, null, null, null,


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


[28/50] [abbrv] hadoop git commit: YARN-6563 ConcurrentModificationException in TimelineCollectorManager while stopping RM (Contributed by Haibo Chen via Vrushali C)

Posted by va...@apache.org.
YARN-6563 ConcurrentModificationException in TimelineCollectorManager while stopping RM (Contributed by Haibo Chen via Vrushali C)


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

Branch: refs/heads/YARN-5355_branch2
Commit: f07a97c1afcd4ee049b4b2af0163221800946a1a
Parents: ab20af1
Author: Vrushali Channapattan <vr...@apache.org>
Authored: Tue May 9 11:15:24 2017 -0700
Committer: Vrushali Channapattan <vr...@apache.org>
Committed: Tue May 9 11:15:24 2017 -0700

----------------------------------------------------------------------
 .../timelineservice/collector/TimelineCollectorManager.java    | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f07a97c1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.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/collector/TimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
index 8ef9b43..07cbb2b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollectorManager.java
@@ -219,8 +219,10 @@ public class TimelineCollectorManager extends AbstractService {
   @Override
   protected void serviceStop() throws Exception {
     if (collectors != null && collectors.size() > 1) {
-      for (TimelineCollector c : collectors.values()) {
-        c.serviceStop();
+      synchronized (collectors) {
+        for (TimelineCollector c : collectors.values()) {
+          c.serviceStop();
+        }
       }
     }
     // stop the flusher first


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


[36/50] [abbrv] hadoop git commit: YARN-4455. Support fetching metrics by time range. Contributed by Varun Saxena.

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/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 3a9e15a..5134bfb 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
@@ -264,6 +264,11 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the entities
+   *     would not contain metric values before this timestamp(Optional query
+   *     param).
+   * @param metricsTimeEnd If specified, returned metrics for the entities would
+   *     not contain metric values after this timestamp(Optional query param).
    * @param fromId If specified, retrieve the next set of entities from the
    *     given fromId. The set of entities retrieved is inclusive of specified
    *     fromId. fromId should be taken from the value associated with FROM_ID
@@ -299,6 +304,8 @@ public class TimelineReaderWebServices {
       @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 ? "" :
@@ -325,7 +332,8 @@ public class TimelineReaderWebServices {
               infofilters, conffilters, metricfilters, eventfilters,
               fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+          metricsTimeStart, metricsTimeEnd));
     } catch (Exception e) {
       handleException(e, url, startTime,
           "createdTime start/end or limit or flowrunid");
@@ -406,6 +414,11 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the entities
+   *     would not contain metric values before this timestamp(Optional query
+   *     param).
+   * @param metricsTimeEnd If specified, returned metrics for the entities would
+   *     not contain metric values after this timestamp(Optional query param).
    * @param fromId If specified, retrieve the next set of entities from the
    *     given fromId. The set of entities retrieved is inclusive of specified
    *     fromId. fromId should be taken from the value associated with FROM_ID
@@ -446,12 +459,14 @@ public class TimelineReaderWebServices {
       @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 getEntities(req, res, null, appId, entityType, userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
         confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
-        fromId);
+        metricsTimeStart, metricsTimeEnd, fromId);
   }
 
   /**
@@ -522,6 +537,11 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the entities
+   *     would not contain metric values before this timestamp(Optional query
+   *     param).
+   * @param metricsTimeEnd If specified, returned metrics for the entities would
+   *     not contain metric values after this timestamp(Optional query param).
    * @param fromId If specified, retrieve the next set of entities from the
    *     given fromId. The set of entities retrieved is inclusive of specified
    *     fromId. fromId should be taken from the value associated with FROM_ID
@@ -563,6 +583,8 @@ public class TimelineReaderWebServices {
       @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 ? "" :
@@ -585,7 +607,8 @@ public class TimelineReaderWebServices {
               infofilters, conffilters, metricfilters, eventfilters,
               fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+          metricsTimeStart, metricsTimeEnd));
     } catch (Exception e) {
       handleException(e, url, startTime,
           "createdTime start/end or limit or flowrunid");
@@ -627,6 +650,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the entity would
+   *     not contain metric values before this timestamp(Optional query param).
+   * @param metricsTimeEnd If specified, returned metrics for the entity would
+   *     not contain metric values after this timestamp(Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>TimelineEntity</cite> instance is returned.<br>
@@ -648,7 +675,9 @@ public class TimelineReaderWebServices {
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
-      @QueryParam("metricslimit") String metricsLimit) {
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("metricstimestart") String metricsTimeStart,
+      @QueryParam("metricstimeend") String metricsTimeEnd) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -668,7 +697,8 @@ public class TimelineReaderWebServices {
       }
       entity = timelineReaderManager.getEntity(context,
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+          metricsTimeStart, metricsTimeEnd));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -722,6 +752,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the entity would
+   *     not contain metric values before this timestamp(Optional query param).
+   * @param metricsTimeEnd If specified, returned metrics for the entity would
+   *     not contain metric values after this timestamp(Optional query param).
    * @param entityIdPrefix Defines the id prefix for the entity to be fetched.
    *     If specified, then entity retrieval will be faster.
    *
@@ -751,10 +785,12 @@ public class TimelineReaderWebServices {
       @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 getEntity(req, res, null, appId, entityType, entityId, userId,
         flowName, flowRunId, confsToRetrieve, metricsToRetrieve, fields,
-        metricsLimit, entityIdPrefix);
+        metricsLimit, metricsTimeStart, metricsTimeEnd, entityIdPrefix);
   }
 
   /**
@@ -796,6 +832,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the entity would
+   *     not contain metric values before this timestamp(Optional query param).
+   * @param metricsTimeEnd If specified, returned metrics for the entity would
+   *     not contain metric values after this timestamp(Optional query param).
    * @param entityIdPrefix Defines the id prefix for the entity to be fetched.
    *     If specified, then entity retrieval will be faster.
    *
@@ -826,6 +866,8 @@ public class TimelineReaderWebServices {
       @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 ? "" :
@@ -844,7 +886,8 @@ public class TimelineReaderWebServices {
               clusterId, userId, flowName, flowRunId, appId, entityType,
               entityIdPrefix, entityId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+          metricsTimeStart, metricsTimeEnd));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -911,7 +954,7 @@ public class TimelineReaderWebServices {
       context.setEntityType(TimelineEntityType.YARN_FLOW_RUN.toString());
       entity = timelineReaderManager.getEntity(context,
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, metricsToRetrieve, null, null));
+          null, metricsToRetrieve, null, null, null, null));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -1020,7 +1063,7 @@ public class TimelineReaderWebServices {
           clusterId, userId, flowName, flowRunId, null,
               TimelineEntityType.YARN_FLOW_RUN.toString(), null, null),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, metricsToRetrieve, null, null));
+          null, metricsToRetrieve, null, null, null, null));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -1114,7 +1157,7 @@ public class TimelineReaderWebServices {
           limit, createdTimeStart, createdTimeEnd, null, null, null,
               null, null, null, fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, metricsToRetrieve, fields, null));
+          null, metricsToRetrieve, fields, null, null, null));
     } catch (Exception e) {
       handleException(e, url, startTime,
           "createdTime start/end or limit or fromId");
@@ -1264,7 +1307,7 @@ public class TimelineReaderWebServices {
           limit, createdTimeStart, createdTimeEnd, null, null, null,
               null, null, null, fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, metricsToRetrieve, fields, null));
+          null, metricsToRetrieve, fields, null, null, null));
     } catch (Exception e) {
       handleException(e, url, startTime,
           "createdTime start/end or limit or fromId");
@@ -1399,7 +1442,7 @@ public class TimelineReaderWebServices {
           clusterId, null, null, null, null,
               TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null, null),
           entityFilters, TimelineReaderWebServicesUtils.
-          createTimelineDataToRetrieve(null, null, null, null));
+              createTimelineDataToRetrieve(null, null, null, null, null, null));
     } catch (Exception e) {
       handleException(e, url, startTime, "limit");
     }
@@ -1440,6 +1483,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the apps would
+   *     not contain metric values before this timestamp(Optional query param).
+   * @param metricsTimeEnd If specified, returned metrics for the apps would
+   *     not contain metric values after this timestamp(Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>TimelineEntity</cite> instance is returned.<br>
@@ -1461,7 +1508,9 @@ public class TimelineReaderWebServices {
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
-      @QueryParam("metricslimit") String metricsLimit) {
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("metricstimestart") String metricsTimeStart,
+      @QueryParam("metricstimeend") String metricsTimeEnd) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -1482,7 +1531,8 @@ public class TimelineReaderWebServices {
       context.setEntityType(TimelineEntityType.YARN_APPLICATION.toString());
       entity = timelineReaderManager.getEntity(context,
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+          metricsTimeStart, metricsTimeEnd));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -1531,6 +1581,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the app would
+   *     not contain metric values before this timestamp(Optional query param).
+   * @param metricsTimeEnd If specified, returned metrics for the app would
+   *     not contain metric values after this timestamp(Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>TimelineEntity</cite> instance is returned.<br>
@@ -1555,9 +1609,12 @@ public class TimelineReaderWebServices {
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
-      @QueryParam("metricslimit") String metricsLimit) {
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("metricstimestart") String metricsTimeStart,
+      @QueryParam("metricstimeend") String metricsTimeEnd) {
     return getApp(req, res, null, appId, flowName, flowRunId, userId,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+        metricsTimeStart, metricsTimeEnd);
   }
 
   /**
@@ -1595,6 +1652,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the app would
+   *     not contain metric values before this timestamp(Optional query param).
+   * @param metricsTimeEnd If specified, returned metrics for the app would
+   *     not contain metric values after this timestamp(Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>TimelineEntity</cite> instance is returned.<br>
@@ -1620,7 +1681,9 @@ public class TimelineReaderWebServices {
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields,
-      @QueryParam("metricslimit") String metricsLimit) {
+      @QueryParam("metricslimit") String metricsLimit,
+      @QueryParam("metricstimestart") String metricsTimeStart,
+      @QueryParam("metricstimeend") String metricsTimeEnd) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -1638,7 +1701,8 @@ public class TimelineReaderWebServices {
           clusterId, userId, flowName, flowRunId, appId,
               TimelineEntityType.YARN_APPLICATION.toString(), null, null),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+          metricsTimeStart, metricsTimeEnd));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -1711,6 +1775,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the apps would
+   *     not contain metric values before this timestamp(Optional query param).
+   * @param metricsTimeEnd If specified, returned metrics for the apps would
+   *     not contain metric values after this timestamp(Optional query param).
    * @param fromId If specified, retrieve the next set of applications
    *     from the given fromId. The set of applications retrieved is inclusive
    *     of specified fromId. fromId should be taken from the value associated
@@ -1745,6 +1813,8 @@ public class TimelineReaderWebServices {
       @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 ? "" :
@@ -1770,7 +1840,8 @@ public class TimelineReaderWebServices {
               infofilters, conffilters, metricfilters, eventfilters,
               fromId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+          metricsTimeStart, metricsTimeEnd));
     } catch (Exception e) {
       handleException(e, url, startTime,
           "createdTime start/end or limit or flowrunid");
@@ -1844,6 +1915,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the apps would
+   *     not contain metric values before this timestamp(Optional query param).
+   * @param metricsTimeEnd If specified, returned metrics for the apps would
+   *     not contain metric values after this timestamp(Optional query param).
    * @param fromId If specified, retrieve the next set of applications
    *     from the given fromId. The set of applications retrieved is inclusive
    *     of specified fromId. fromId should be taken from the value associated
@@ -1880,12 +1955,15 @@ public class TimelineReaderWebServices {
       @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 getEntities(req, res, null, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, fromId);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+        metricsTimeStart, metricsTimeEnd, fromId);
   }
 
   /**
@@ -1949,6 +2027,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the apps would
+   *     not contain metric values before this timestamp(Optional query param).
+   * @param metricsTimeEnd If specified, returned metrics for the apps would
+   *     not contain metric values after this timestamp(Optional query param).
    * @param fromId If specified, retrieve the next set of applications
    *     from the given fromId. The set of applications retrieved is inclusive
    *     of specified fromId. fromId should be taken from the value associated
@@ -1987,12 +2069,15 @@ public class TimelineReaderWebServices {
       @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 getEntities(req, res, clusterId, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, fromId);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+        metricsTimeStart, metricsTimeEnd, fromId);
   }
 
   /**
@@ -2053,6 +2138,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the apps would
+   *     not contain metric values before this timestamp(Optional query param).
+   * @param metricsTimeEnd If specified, returned metrics for the apps would
+   *     not contain metric values after this timestamp(Optional query param).
    * @param fromId If specified, retrieve the next set of applications
    *     from the given fromId. The set of applications retrieved is inclusive
    *     of specified fromId. fromId should be taken from the value associated
@@ -2088,12 +2177,15 @@ public class TimelineReaderWebServices {
       @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 getEntities(req, res, null, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         null, limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
         infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, fromId);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+        metricsTimeStart, metricsTimeEnd, fromId);
   }
 
   /**
@@ -2155,6 +2247,10 @@ public class TimelineReaderWebServices {
    *     or has a value less than 1, and metrics have to be retrieved, then
    *     metricsLimit will be considered as 1 i.e. latest single value of
    *     metric(s) will be returned. (Optional query param).
+   * @param metricsTimeStart If specified, returned metrics for the apps would
+   *     not contain metric values before this timestamp(Optional query param).
+   * @param metricsTimeEnd If specified, returned metrics for the apps would
+   *     not contain metric values after this timestamp(Optional query param).
    * @param fromId If specified, retrieve the next set of applications
    *     from the given fromId. The set of applications retrieved is inclusive
    *     of specified fromId. fromId should be taken from the value associated
@@ -2191,12 +2287,15 @@ public class TimelineReaderWebServices {
       @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 getEntities(req, res, clusterId, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         null, limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
         infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields, metricsLimit, fromId);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
+        metricsTimeStart, metricsTimeEnd, fromId);
   }
 
   /**
@@ -2267,6 +2366,12 @@ public class TimelineReaderWebServices {
    *          have to be retrieved, then metricsLimit will be considered as 1
    *          i.e. latest single value of metric(s) will be returned. (Optional
    *          query param).
+   * @param metricsTimeStart If specified, returned metrics for the app attempts
+   *          would not contain metric values before this timestamp(Optional
+   *          query param).
+   * @param metricsTimeEnd If specified, returned metrics for the app attempts
+   *          would not contain metric values after this timestamp(Optional
+   *          query param).
    * @param fromId If specified, retrieve the next set of application-attempt
    *         entities from the given fromId. The set of application-attempt
    *         entities retrieved is inclusive of specified fromId. fromId should
@@ -2305,12 +2410,15 @@ public class TimelineReaderWebServices {
       @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 getAppAttempts(req, res, null, appId, userId, flowName, flowRunId,
         limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
         infofilters, conffilters, metricfilters, eventfilters, confsToRetrieve,
-        metricsToRetrieve, fields, metricsLimit, fromId);
+        metricsToRetrieve, fields, metricsLimit, metricsTimeStart,
+        metricsTimeEnd, fromId);
   }
 
   /**
@@ -2382,6 +2490,12 @@ public class TimelineReaderWebServices {
    *          have to be retrieved, then metricsLimit will be considered as 1
    *          i.e. latest single value of metric(s) will be returned. (Optional
    *          query param).
+   * @param metricsTimeStart If specified, returned metrics for the app attempts
+   *          would not contain metric values before this timestamp(Optional
+   *          query param).
+   * @param metricsTimeEnd If specified, returned metrics for the app attempts
+   *          would not contain metric values after this timestamp(Optional
+   *          query param).
    * @param fromId If specified, retrieve the next set of application-attempt
    *         entities from the given fromId. The set of application-attempt
    *         entities retrieved is inclusive of specified fromId. fromId should
@@ -2421,6 +2535,8 @@ public class TimelineReaderWebServices {
       @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 getEntities(req, res, clusterId, appId,
@@ -2428,7 +2544,7 @@ public class TimelineReaderWebServices {
         flowName, flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
         confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
-        fromId);
+        metricsTimeStart, metricsTimeEnd, fromId);
   }
 
   /**
@@ -2471,6 +2587,12 @@ public class TimelineReaderWebServices {
    *          have to be retrieved, then metricsLimit will be considered as 1
    *          i.e. latest single value of metric(s) will be returned. (Optional
    *          query param).
+   * @param metricsTimeStart If specified, returned metrics for the app attempt
+   *          would not contain metric values before this timestamp(Optional
+   *          query param).
+   * @param metricsTimeEnd If specified, returned metrics for the app attempt
+   *          would not contain metric values after this timestamp(Optional
+   *          query param).
    * @param entityIdPrefix Defines the id prefix for the entity to be fetched.
    *          If specified, then entity retrieval will be faster.
    *
@@ -2498,10 +2620,12 @@ public class TimelineReaderWebServices {
       @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 getAppAttempt(req, res, null, appId, appAttemptId, userId, flowName,
         flowRunId, confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
-        entityIdPrefix);
+        metricsTimeStart, metricsTimeEnd, entityIdPrefix);
   }
 
   /**
@@ -2544,6 +2668,12 @@ public class TimelineReaderWebServices {
    *          have to be retrieved, then metricsLimit will be considered as 1
    *          i.e. latest single value of metric(s) will be returned. (Optional
    *          query param).
+   * @param metricsTimeStart If specified, returned metrics for the app attempt
+   *          would not contain metric values before this timestamp(Optional
+   *          query param).
+   * @param metricsTimeEnd If specified, returned metrics for the app attempt
+   *          would not contain metric values after this timestamp(Optional
+   *          query param).
    * @param entityIdPrefix Defines the id prefix for the entity to be fetched.
    *          If specified, then entity retrieval will be faster.
    *
@@ -2573,11 +2703,13 @@ public class TimelineReaderWebServices {
       @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 getEntity(req, res, clusterId, appId,
         TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(), appAttemptId,
         userId, flowName, flowRunId, confsToRetrieve, metricsToRetrieve, fields,
-        metricsLimit, entityIdPrefix);
+        metricsLimit, metricsTimeStart, metricsTimeEnd, entityIdPrefix);
   }
 
   /**
@@ -2650,6 +2782,12 @@ public class TimelineReaderWebServices {
    *          have to be retrieved, then metricsLimit will be considered as 1
    *          i.e. latest single value of metric(s) will be returned. (Optional
    *          query param).
+   * @param metricsTimeStart If specified, returned metrics for the containers
+   *          would not contain metric values before this timestamp(Optional
+   *          query param).
+   * @param metricsTimeEnd If specified, returned metrics for the containers
+   *          would not contain metric values after this timestamp(Optional
+   *          query param).
    * @param fromId If specified, retrieve the next set of container
    *         entities from the given fromId. The set of container
    *         entities retrieved is inclusive of specified fromId. fromId should
@@ -2689,12 +2827,14 @@ public class TimelineReaderWebServices {
       @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 getContainers(req, res, null, appId, appattemptId, userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
         confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
-        fromId);
+        metricsTimeStart, metricsTimeEnd, fromId);
   }
 
   /**
@@ -2768,6 +2908,12 @@ public class TimelineReaderWebServices {
    *          have to be retrieved, then metricsLimit will be considered as 1
    *          i.e. latest single value of metric(s) will be returned. (Optional
    *          query param).
+   * @param metricsTimeStart If specified, returned metrics for the containers
+   *          would not contain metric values before this timestamp(Optional
+   *          query param).
+   * @param metricsTimeEnd If specified, returned metrics for the containers
+   *          would not contain metric values after this timestamp(Optional
+   *          query param).
    * @param fromId If specified, retrieve the next set of container
    *         entities from the given fromId. The set of container
    *         entities retrieved is inclusive of specified fromId. fromId should
@@ -2809,6 +2955,8 @@ public class TimelineReaderWebServices {
       @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 entityType = TimelineEntityType.YARN_CONTAINER.toString();
@@ -2828,7 +2976,7 @@ public class TimelineReaderWebServices {
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilter, conffilters, metricfilters, eventfilters,
         confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
-        fromId);
+        metricsTimeStart, metricsTimeEnd, fromId);
   }
 
   /**
@@ -2870,6 +3018,12 @@ public class TimelineReaderWebServices {
    *          have to be retrieved, then metricsLimit will be considered as 1
    *          i.e. latest single value of metric(s) will be returned. (Optional
    *          query param).
+   * @param metricsTimeStart If specified, returned metrics for the container
+   *          would not contain metric values before this timestamp(Optional
+   *          query param).
+   * @param metricsTimeEnd If specified, returned metrics for the container
+   *          would not contain metric values after this timestamp(Optional
+   *          query param).
    * @param entityIdPrefix Defines the id prefix for the entity to be fetched.
    *          If specified, then entity retrieval will be faster.
    *
@@ -2897,10 +3051,12 @@ public class TimelineReaderWebServices {
       @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 getContainer(req, res, null, appId, containerId, userId, flowName,
         flowRunId, confsToRetrieve, metricsToRetrieve, fields, metricsLimit,
-        entityIdPrefix);
+        entityIdPrefix, metricsTimeStart, metricsTimeEnd);
   }
 
   /**
@@ -2943,6 +3099,12 @@ public class TimelineReaderWebServices {
    *          have to be retrieved, then metricsLimit will be considered as 1
    *          i.e. latest single value of metric(s) will be returned. (Optional
    *          query param).
+   * @param metricsTimeStart If specified, returned metrics for the container
+   *          would not contain metric values before this timestamp(Optional
+   *          query param).
+   * @param metricsTimeEnd If specified, returned metrics for the container
+   *          would not contain metric values after this timestamp(Optional
+   *          query param).
    * @param entityIdPrefix Defines the id prefix for the entity to be fetched.
    *          If specified, then entity retrieval will be faster.
    *
@@ -2972,11 +3134,13 @@ public class TimelineReaderWebServices {
       @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 getEntity(req, res, clusterId, appId,
         TimelineEntityType.YARN_CONTAINER.toString(), containerId, userId,
         flowName, flowRunId, confsToRetrieve, metricsToRetrieve, fields,
-        metricsLimit, entityIdPrefix);
+        metricsLimit, metricsTimeStart, metricsTimeEnd, entityIdPrefix);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/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 4d3e769..cded3a1 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
@@ -127,11 +127,13 @@ final class TimelineReaderWebServicesUtils {
    * @throws TimelineParseException if any problem occurs during parsing.
    */
   static TimelineDataToRetrieve createTimelineDataToRetrieve(String confs,
-      String metrics, String fields, String metricsLimit)
+      String metrics, String fields, String metricsLimit,
+      String metricsTimeBegin, String metricsTimeEnd)
       throws TimelineParseException {
     return new TimelineDataToRetrieve(parseDataToRetrieve(confs),
         parseDataToRetrieve(metrics), parseFieldsStr(fields,
-        TimelineParseConstants.COMMA_DELIMITER), parseIntStr(metricsLimit));
+        TimelineParseConstants.COMMA_DELIMITER), parseIntStr(metricsLimit),
+        parseLongStr(metricsTimeBegin), parseLongStr(metricsTimeEnd));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6aafd26/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.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/storage/TestFileSystemTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java
index 1bc66db..46873ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java
@@ -319,7 +319,7 @@ public class TestFileSystemTimelineReaderImpl {
     TimelineEntity result = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", "id_1"),
-        new TimelineDataToRetrieve(null, null, null, null));
+        new TimelineDataToRetrieve(null, null, null, null, null, null));
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
@@ -334,7 +334,7 @@ public class TestFileSystemTimelineReaderImpl {
     TimelineEntity result = reader.getEntity(
         new TimelineReaderContext("cluster1", null, null, null, "app1", "app",
         "id_1"),
-        new TimelineDataToRetrieve(null, null, null, null));
+        new TimelineDataToRetrieve(null, null, null, null, null, null));
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
@@ -351,7 +351,7 @@ public class TestFileSystemTimelineReaderImpl {
     TimelineEntity result = reader.getEntity(
         new TimelineReaderContext("cluster1", null, null, null, "app2",
         "app", "id_5"),
-        new TimelineDataToRetrieve(null, null, null, null));
+        new TimelineDataToRetrieve(null, null, null, null, null, null));
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_5")).toString(),
         result.getIdentifier().toString());
@@ -365,7 +365,8 @@ public class TestFileSystemTimelineReaderImpl {
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", "id_1"),
         new TimelineDataToRetrieve(null, null,
-        EnumSet.of(Field.INFO, Field.CONFIGS, Field.METRICS), null));
+        EnumSet.of(Field.INFO, Field.CONFIGS, Field.METRICS), null, null,
+        null));
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
@@ -383,7 +384,8 @@ public class TestFileSystemTimelineReaderImpl {
     TimelineEntity result = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", "id_1"),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
@@ -399,7 +401,8 @@ public class TestFileSystemTimelineReaderImpl {
     Set<TimelineEntity> result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null), new TimelineEntityFilters.Builder().build(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null,
+        null, null));
     // All 4 entities will be returned
     Assert.assertEquals(4, result.size());
   }


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


[23/50] [abbrv] hadoop git commit: Addendum for YARN-4218. Metric for resource*time that was preempted.

Posted by va...@apache.org.
Addendum for YARN-4218. Metric for resource*time that was preempted.


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

Branch: refs/heads/YARN-5355_branch2
Commit: 39b383aaffb4509fe51300b6339dc6d9a8a25d3c
Parents: 4deb89d
Author: Varun Saxena <va...@apache.org>
Authored: Wed Apr 26 11:29:12 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 11:29:12 2017 +0530

----------------------------------------------------------------------
 .../resourcemanager/metrics/TimelineServiceV1Publisher.java    | 4 ++++
 .../resourcemanager/metrics/TimelineServiceV2Publisher.java    | 6 ++++++
 .../metrics/TestSystemMetricsPublisherForV2.java               | 2 +-
 3 files changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/39b383aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
index cbf6a73..89a8616 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
@@ -144,6 +144,10 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
         appMetrics.getVcoreSeconds());
     entity.addOtherInfo(ApplicationMetricsConstants.APP_MEM_METRICS,
         appMetrics.getMemorySeconds());
+    entity.addOtherInfo(ApplicationMetricsConstants.APP_MEM_PREEMPT_METRICS,
+        appMetrics.getPreemptedMemorySeconds());
+    entity.addOtherInfo(ApplicationMetricsConstants.APP_CPU_PREEMPT_METRICS,
+        appMetrics.getPreemptedVcoreSeconds());
     tEvent.setEventInfo(eventInfo);
 
     entity.addEvent(tEvent);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39b383aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
index 1fc0c38..495c36c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
@@ -196,6 +196,12 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
         ApplicationMetricsConstants.APP_MEM_METRICS, timestamp,
         appMetrics.getMemorySeconds()));
     entityMetrics.add(getTimelineMetric(
+        ApplicationMetricsConstants.APP_MEM_PREEMPT_METRICS, timestamp,
+        appMetrics.getPreemptedMemorySeconds()));
+    entityMetrics.add(getTimelineMetric(
+        ApplicationMetricsConstants.APP_CPU_PREEMPT_METRICS, timestamp,
+        appMetrics.getPreemptedVcoreSeconds()));
+    entityMetrics.add(getTimelineMetric(
         ApplicationMetricsConstants.APP_RESOURCE_PREEMPTED_CPU, timestamp,
         appMetrics.getResourcePreempted().getVirtualCores()));
     entityMetrics.add(getTimelineMetric(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/39b383aa/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
index 5aaddbd..0f7d9d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
@@ -213,7 +213,7 @@ public class TestSystemMetricsPublisherForV2 {
     File appFile = new File(outputDirApp, timelineServiceFileName);
     Assert.assertTrue(appFile.exists());
     verifyEntity(
-        appFile, 3, ApplicationMetricsConstants.CREATED_EVENT_TYPE, 6, 0);
+        appFile, 3, ApplicationMetricsConstants.CREATED_EVENT_TYPE, 8, 0);
   }
 
   @Test(timeout = 10000)


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


[31/50] [abbrv] hadoop git commit: YARN-6658. Remove columnFor() methods of Columns in HBaseTimeline backend (Haibo Chen via Varun Saxena)

Posted by va...@apache.org.
YARN-6658. Remove columnFor() methods of Columns in HBaseTimeline backend (Haibo Chen via Varun Saxena)

(cherry picked from commit f7cba98504a5e968bb6ae976d20b71c84a4664f2)

Conflicts:
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumn.java


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

Branch: refs/heads/YARN-5355_branch2
Commit: ee6a4d0e1905d25c14c0c02e8cdad47013d0252f
Parents: 303d7e0
Author: Varun Saxena <va...@apache.org>
Authored: Wed May 31 10:14:35 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed May 31 10:27:16 2017 +0530

----------------------------------------------------------------------
 .../storage/application/ApplicationColumn.java  | 48 -----------------
 .../application/ApplicationColumnPrefix.java    | 52 ------------------
 .../storage/entity/EntityColumn.java            | 48 -----------------
 .../storage/entity/EntityColumnPrefix.java      | 51 ------------------
 .../storage/flow/FlowActivityColumnPrefix.java  | 56 --------------------
 .../storage/flow/FlowRunColumn.java             | 51 ------------------
 .../storage/flow/FlowRunColumnPrefix.java       | 51 ------------------
 7 files changed, 357 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6a4d0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.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/application/ApplicationColumn.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/application/ApplicationColumn.java
index dde3911..00eaa7e 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/application/ApplicationColumn.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/application/ApplicationColumn.java
@@ -105,52 +105,4 @@ public enum ApplicationColumn implements Column<ApplicationTable> {
     return column.getValueConverter();
   }
 
-  /**
-   * Retrieve an {@link ApplicationColumn} given a name, or null if there is no
-   * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
-   * and only if {@code x.equals(y)} or {@code (x == y == null)}.
-   *
-   * @param columnQualifier Name of the column to retrieve
-   * @return the corresponding {@link ApplicationColumn} or null
-   */
-  public static final ApplicationColumn columnFor(String columnQualifier) {
-
-    // Match column based on value, assume column family matches.
-    for (ApplicationColumn ac : ApplicationColumn.values()) {
-      // Find a match based only on name.
-      if (ac.getColumnQualifier().equals(columnQualifier)) {
-        return ac;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-
-  /**
-   * Retrieve an {@link ApplicationColumn} given a name, or null if there is no
-   * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
-   * if and only if {@code a.equals(b) & x.equals(y)} or
-   * {@code (x == y == null)}
-   *
-   * @param columnFamily The columnFamily for which to retrieve the column.
-   * @param name Name of the column to retrieve
-   * @return the corresponding {@link ApplicationColumn} or null if both
-   *         arguments don't match.
-   */
-  public static final ApplicationColumn columnFor(
-      ApplicationColumnFamily columnFamily, String name) {
-
-    for (ApplicationColumn ac : ApplicationColumn.values()) {
-      // Find a match based column family and on name.
-      if (ac.columnFamily.equals(columnFamily)
-          && ac.getColumnQualifier().equals(name)) {
-        return ac;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6a4d0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.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/application/ApplicationColumnPrefix.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/application/ApplicationColumnPrefix.java
index 42488f4..8297dc5 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/application/ApplicationColumnPrefix.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/application/ApplicationColumnPrefix.java
@@ -233,56 +233,4 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
         keyConverter);
   }
 
-  /**
-   * Retrieve an {@link ApplicationColumnPrefix} given a name, or null if there
-   * is no match. The following holds true: {@code columnFor(x) == columnFor(y)}
-   * if and only if {@code x.equals(y)} or {@code (x == y == null)}
-   *
-   * @param columnPrefix Name of the column to retrieve
-   * @return the corresponding {@link ApplicationColumnPrefix} or null
-   */
-  public static final ApplicationColumnPrefix columnFor(String columnPrefix) {
-
-    // Match column based on value, assume column family matches.
-    for (ApplicationColumnPrefix acp : ApplicationColumnPrefix.values()) {
-      // Find a match based only on name.
-      if (acp.getColumnPrefix().equals(columnPrefix)) {
-        return acp;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-
-  /**
-   * Retrieve an {@link ApplicationColumnPrefix} given a name, or null if there
-   * is no match. The following holds true:
-   * {@code columnFor(a,x) == columnFor(b,y)} if and only if
-   * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)}
-   *
-   * @param columnFamily The columnFamily for which to retrieve the column.
-   * @param columnPrefix Name of the column to retrieve
-   * @return the corresponding {@link ApplicationColumnPrefix} or null if both
-   *         arguments don't match.
-   */
-  public static final ApplicationColumnPrefix columnFor(
-      ApplicationColumnFamily columnFamily, String columnPrefix) {
-
-    // TODO: needs unit test to confirm and need to update javadoc to explain
-    // null prefix case.
-
-    for (ApplicationColumnPrefix acp : ApplicationColumnPrefix.values()) {
-      // Find a match based column family and on name.
-      if (acp.columnFamily.equals(columnFamily)
-          && (((columnPrefix == null) && (acp.getColumnPrefix() == null)) ||
-          (acp.getColumnPrefix().equals(columnPrefix)))) {
-        return acp;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6a4d0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.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/entity/EntityColumn.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/entity/EntityColumn.java
index 93b4b36..b228d84 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/entity/EntityColumn.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/entity/EntityColumn.java
@@ -94,28 +94,6 @@ public enum EntityColumn implements Column<EntityTable> {
     return column.readResult(result, columnQualifierBytes);
   }
 
-  /**
-   * Retrieve an {@link EntityColumn} given a name, or null if there is no
-   * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
-   * and only if {@code x.equals(y)} or {@code (x == y == null)}
-   *
-   * @param columnQualifier Name of the column to retrieve
-   * @return the corresponding {@link EntityColumn} or null
-   */
-  public static final EntityColumn columnFor(String columnQualifier) {
-
-    // Match column based on value, assume column family matches.
-    for (EntityColumn ec : EntityColumn.values()) {
-      // Find a match based only on name.
-      if (ec.getColumnQualifier().equals(columnQualifier)) {
-        return ec;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-
   @Override
   public byte[] getColumnQualifierBytes() {
     return columnQualifierBytes.clone();
@@ -131,30 +109,4 @@ public enum EntityColumn implements Column<EntityTable> {
     return column.getValueConverter();
   }
 
-  /**
-   * Retrieve an {@link EntityColumn} given a name, or null if there is no
-   * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
-   * if and only if {@code a.equals(b) & x.equals(y)} or
-   * {@code (x == y == null)}
-   *
-   * @param columnFamily The columnFamily for which to retrieve the column.
-   * @param name Name of the column to retrieve
-   * @return the corresponding {@link EntityColumn} or null if both arguments
-   *         don't match.
-   */
-  public static final EntityColumn columnFor(EntityColumnFamily columnFamily,
-      String name) {
-
-    for (EntityColumn ec : EntityColumn.values()) {
-      // Find a match based column family and on name.
-      if (ec.columnFamily.equals(columnFamily)
-          && ec.getColumnQualifier().equals(name)) {
-        return ec;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6a4d0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.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/entity/EntityColumnPrefix.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/entity/EntityColumnPrefix.java
index e410549..d385108 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/entity/EntityColumnPrefix.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/entity/EntityColumnPrefix.java
@@ -246,55 +246,4 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
         keyConverter);
   }
 
-  /**
-   * Retrieve an {@link EntityColumnPrefix} given a name, or null if there is no
-   * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
-   * and only if {@code x.equals(y)} or {@code (x == y == null)}
-   *
-   * @param columnPrefix Name of the column to retrieve
-   * @return the corresponding {@link EntityColumnPrefix} or null
-   */
-  public static final EntityColumnPrefix columnFor(String columnPrefix) {
-
-    // Match column based on value, assume column family matches.
-    for (EntityColumnPrefix ecp : EntityColumnPrefix.values()) {
-      // Find a match based only on name.
-      if (ecp.getColumnPrefix().equals(columnPrefix)) {
-        return ecp;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-
-  /**
-   * Retrieve an {@link EntityColumnPrefix} given a name, or null if there is no
-   * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
-   * if and only if {@code (x == y == null)} or
-   * {@code a.equals(b) & x.equals(y)}
-   *
-   * @param columnFamily The columnFamily for which to retrieve the column.
-   * @param columnPrefix Name of the column to retrieve
-   * @return the corresponding {@link EntityColumnPrefix} or null if both
-   *         arguments don't match.
-   */
-  public static final EntityColumnPrefix columnFor(
-      EntityColumnFamily columnFamily, String columnPrefix) {
-
-    // TODO: needs unit test to confirm and need to update javadoc to explain
-    // null prefix case.
-
-    for (EntityColumnPrefix ecp : EntityColumnPrefix.values()) {
-      // Find a match based column family and on name.
-      if (ecp.columnFamily.equals(columnFamily)
-          && (((columnPrefix == null) && (ecp.getColumnPrefix() == null)) ||
-          (ecp.getColumnPrefix().equals(columnPrefix)))) {
-        return ecp;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6a4d0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.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/flow/FlowActivityColumnPrefix.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/flow/FlowActivityColumnPrefix.java
index 5e7a5d6..706b002 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/flow/FlowActivityColumnPrefix.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/flow/FlowActivityColumnPrefix.java
@@ -191,62 +191,6 @@ public enum FlowActivityColumnPrefix
         keyConverter);
   }
 
-  /**
-   * Retrieve an {@link FlowActivityColumnPrefix} given a name, or null if there
-   * is no match. The following holds true: {@code columnFor(x) == columnFor(y)}
-   * if and only if {@code x.equals(y)} or {@code (x == y == null)}
-   *
-   * @param columnPrefix
-   *          Name of the column to retrieve
-   * @return the corresponding {@link FlowActivityColumnPrefix} or null
-   */
-  public static final FlowActivityColumnPrefix columnFor(String columnPrefix) {
-
-    // Match column based on value, assume column family matches.
-    for (FlowActivityColumnPrefix flowActivityColPrefix :
-        FlowActivityColumnPrefix.values()) {
-      // Find a match based only on name.
-      if (flowActivityColPrefix.getColumnPrefix().equals(columnPrefix)) {
-        return flowActivityColPrefix;
-      }
-    }
-    // Default to null
-    return null;
-  }
-
-  /**
-   * Retrieve an {@link FlowActivityColumnPrefix} given a name, or null if there
-   * is no match. The following holds true:
-   * {@code columnFor(a,x) == columnFor(b,y)} if and only if
-   * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)}
-   *
-   * @param columnFamily
-   *          The columnFamily for which to retrieve the column.
-   * @param columnPrefix
-   *          Name of the column to retrieve
-   * @return the corresponding {@link FlowActivityColumnPrefix} or null if both
-   *         arguments don't match.
-   */
-  public static final FlowActivityColumnPrefix columnFor(
-      FlowActivityColumnFamily columnFamily, String columnPrefix) {
-
-    // TODO: needs unit test to confirm and need to update javadoc to explain
-    // null prefix case.
-
-    for (FlowActivityColumnPrefix flowActivityColumnPrefix :
-        FlowActivityColumnPrefix.values()) {
-      // Find a match based column family and on name.
-      if (flowActivityColumnPrefix.columnFamily.equals(columnFamily)
-          && (((columnPrefix == null) && (flowActivityColumnPrefix
-              .getColumnPrefix() == null)) || (flowActivityColumnPrefix
-              .getColumnPrefix().equals(columnPrefix)))) {
-        return flowActivityColumnPrefix;
-      }
-    }
-    // Default to null
-    return null;
-  }
-
   /*
    * (non-Javadoc)
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6a4d0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.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/flow/FlowRunColumn.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/flow/FlowRunColumn.java
index 90dd345..7a39120 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/flow/FlowRunColumn.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/flow/FlowRunColumn.java
@@ -123,60 +123,9 @@ public enum FlowRunColumn implements Column<FlowRunTable> {
     return column.readResult(result, columnQualifierBytes);
   }
 
-  /**
-   * Retrieve an {@link FlowRunColumn} given a name, or null if there is no
-   * match. The following holds true: {@code columnFor(x) == columnFor(y)} if
-   * and only if {@code x.equals(y)} or {@code (x == y == null)}
-   *
-   * @param columnQualifier
-   *          Name of the column to retrieve
-   * @return the corresponding {@link FlowRunColumn} or null
-   */
-  public static final FlowRunColumn columnFor(String columnQualifier) {
-
-    // Match column based on value, assume column family matches.
-    for (FlowRunColumn ec : FlowRunColumn.values()) {
-      // Find a match based only on name.
-      if (ec.getColumnQualifier().equals(columnQualifier)) {
-        return ec;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-
   @Override
   public ValueConverter getValueConverter() {
     return column.getValueConverter();
   }
 
-  /**
-   * Retrieve an {@link FlowRunColumn} given a name, or null if there is no
-   * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}
-   * if and only if {@code a.equals(b) & x.equals(y)} or
-   * {@code (x == y == null)}
-   *
-   * @param columnFamily
-   *          The columnFamily for which to retrieve the column.
-   * @param name
-   *          Name of the column to retrieve
-   * @return the corresponding {@link FlowRunColumn} or null if both arguments
-   *         don't match.
-   */
-  public static final FlowRunColumn columnFor(FlowRunColumnFamily columnFamily,
-      String name) {
-
-    for (FlowRunColumn ec : FlowRunColumn.values()) {
-      // Find a match based column family and on name.
-      if (ec.columnFamily.equals(columnFamily)
-          && ec.getColumnQualifier().equals(name)) {
-        return ec;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee6a4d0e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.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/flow/FlowRunColumnPrefix.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/flow/FlowRunColumnPrefix.java
index 278d18e..103674e 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/flow/FlowRunColumnPrefix.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/flow/FlowRunColumnPrefix.java
@@ -209,60 +209,9 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
         keyConverter);
   }
 
-  /**
-   * Retrieve an {@link FlowRunColumnPrefix} given a name, or null if there is
-   * no match. The following holds true: {@code columnFor(x) == columnFor(y)} if
-   * and only if {@code x.equals(y)} or {@code (x == y == null)}
-   *
-   * @param columnPrefix Name of the column to retrieve
-   * @return the corresponding {@link FlowRunColumnPrefix} or null
-   */
-  public static final FlowRunColumnPrefix columnFor(String columnPrefix) {
-
-    // Match column based on value, assume column family matches.
-    for (FlowRunColumnPrefix frcp : FlowRunColumnPrefix.values()) {
-      // Find a match based only on name.
-      if (frcp.getColumnPrefix().equals(columnPrefix)) {
-        return frcp;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
-
   @Override
   public ValueConverter getValueConverter() {
     return column.getValueConverter();
   }
 
-  /**
-   * Retrieve an {@link FlowRunColumnPrefix} given a name, or null if there is
-   * no match. The following holds true:
-   * {@code columnFor(a,x) == columnFor(b,y)} if and only if
-   * {@code (x == y == null)} or {@code a.equals(b) & x.equals(y)}
-   *
-   * @param columnFamily The columnFamily for which to retrieve the column.
-   * @param columnPrefix Name of the column to retrieve
-   * @return the corresponding {@link FlowRunColumnPrefix} or null if both
-   *         arguments don't match.
-   */
-  public static final FlowRunColumnPrefix columnFor(
-      FlowRunColumnFamily columnFamily, String columnPrefix) {
-
-    // TODO: needs unit test to confirm and need to update javadoc to explain
-    // null prefix case.
-
-    for (FlowRunColumnPrefix frcp : FlowRunColumnPrefix.values()) {
-      // Find a match based column family and on name.
-      if (frcp.columnFamily.equals(columnFamily)
-          && (((columnPrefix == null) && (frcp.getColumnPrefix() == null)) ||
-          (frcp.getColumnPrefix().equals(columnPrefix)))) {
-        return frcp;
-      }
-    }
-
-    // Default to null
-    return null;
-  }
 }


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


[17/50] [abbrv] hadoop git commit: YARN-6377. NMTimelinePublisher#serviceStop does not stop timeline clients (Haibo Chen via Varun Saxena)

Posted by va...@apache.org.
YARN-6377. NMTimelinePublisher#serviceStop does not stop timeline clients (Haibo Chen via Varun Saxena)

(cherry picked from commit a4b5aa8493e0bd9006f44291d265c28ab86497e1)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 6d9ae7f1448bcb1b5dca81cf823235f8ccf714b3
Parents: fd2c8d2
Author: Varun Saxena <va...@apache.org>
Authored: Sun Apr 2 04:54:12 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:45 2017 +0530

----------------------------------------------------------------------
 .../nodemanager/timelineservice/NMTimelinePublisher.java  |  8 ++++++++
 .../timelineservice/TestNMTimelinePublisher.java          | 10 +++++++++-
 2 files changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d9ae7f1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
index 851ba53..ced41c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
@@ -102,6 +102,14 @@ public class NMTimelinePublisher extends CompositeService {
     this.nodeId = context.getNodeId();
   }
 
+  @Override
+  protected void serviceStop() throws Exception {
+    for(ApplicationId app : appToClientMap.keySet()) {
+      stopTimelineClient(app);
+    }
+    super.serviceStop();
+  }
+
   @VisibleForTesting
   Map<ApplicationId, TimelineV2Client> getAppToClientMap() {
     return appToClientMap;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d9ae7f1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
index e116122..0b8eaa9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.client.api.impl.TimelineV2ClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
@@ -53,14 +54,21 @@ public class TestNMTimelinePublisher {
     final DummyTimelineClient timelineClient = new DummyTimelineClient(null);
     when(context.getNodeId()).thenReturn(NodeId.newInstance("localhost", 0));
     when(context.getHttpPort()).thenReturn(0);
+
+    Configuration conf = new Configuration();
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+
     NMTimelinePublisher publisher = new NMTimelinePublisher(context) {
       public void createTimelineClient(ApplicationId appId) {
         if (!getAppToClientMap().containsKey(appId)) {
+          timelineClient.init(getConfig());
+          timelineClient.start();
           getAppToClientMap().put(appId, timelineClient);
         }
       }
     };
-    publisher.init(new Configuration());
+    publisher.init(conf);
     publisher.start();
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     publisher.createTimelineClient(appId);


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


[08/50] [abbrv] hadoop git commit: YARN-4675. Reorganize TimelineClient and TimelineClientImpl into separate classes for ATSv1.x and ATSv2. Contributed by Naganarasimha G R.

Posted by va...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineConnector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineConnector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineConnector.java
new file mode 100644
index 0000000..bb29d6c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineConnector.java
@@ -0,0 +1,442 @@
+/**
+ * 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.client.api.impl;
+
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.net.ConnectException;
+import java.net.HttpURLConnection;
+import java.net.SocketException;
+import java.net.SocketTimeoutException;
+import java.net.URI;
+import java.net.URL;
+import java.net.URLConnection;
+import java.security.GeneralSecurityException;
+import java.security.PrivilegedExceptionAction;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.SSLSocketFactory;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL.Token;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator;
+import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticator;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.ClientRequest;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.api.client.filter.ClientFilter;
+import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+
+/**
+ * Utility Connector class which is used by timeline clients to securely get
+ * connected to the timeline server.
+ *
+ */
+public class TimelineConnector extends AbstractService {
+
+  private static final Joiner JOINER = Joiner.on("");
+  private static final Log LOG = LogFactory.getLog(TimelineConnector.class);
+  public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
+
+  private SSLFactory sslFactory;
+  private Client client;
+  private ConnectionConfigurator connConfigurator;
+  private DelegationTokenAuthenticator authenticator;
+  private DelegationTokenAuthenticatedURL.Token token;
+  private UserGroupInformation authUgi;
+  private String doAsUser;
+  @VisibleForTesting
+  TimelineClientConnectionRetry connectionRetry;
+  private boolean requireConnectionRetry;
+
+  public TimelineConnector(boolean requireConnectionRetry,
+      UserGroupInformation authUgi, String doAsUser,
+      DelegationTokenAuthenticatedURL.Token token) {
+    super("TimelineConnector");
+    this.requireConnectionRetry = requireConnectionRetry;
+    this.authUgi = authUgi;
+    this.doAsUser = doAsUser;
+    this.token = token;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    ClientConfig cc = new DefaultClientConfig();
+    cc.getClasses().add(YarnJacksonJaxbJsonProvider.class);
+
+    sslFactory = getSSLFactory(conf);
+    connConfigurator = getConnConfigurator(sslFactory);
+
+    if (UserGroupInformation.isSecurityEnabled()) {
+      authenticator = new KerberosDelegationTokenAuthenticator();
+    } else {
+      authenticator = new PseudoDelegationTokenAuthenticator();
+    }
+    authenticator.setConnectionConfigurator(connConfigurator);
+
+    connectionRetry = new TimelineClientConnectionRetry(conf);
+    client =
+        new Client(
+            new URLConnectionClientHandler(new TimelineURLConnectionFactory(
+                authUgi, authenticator, connConfigurator, token, doAsUser)),
+            cc);
+    if (requireConnectionRetry) {
+      TimelineJerseyRetryFilter retryFilter =
+          new TimelineJerseyRetryFilter(connectionRetry);
+      client.addFilter(retryFilter);
+    }
+  }
+
+  private static final ConnectionConfigurator DEFAULT_TIMEOUT_CONN_CONFIGURATOR
+    = new ConnectionConfigurator() {
+        @Override
+        public HttpURLConnection configure(HttpURLConnection conn)
+            throws IOException {
+          setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT);
+          return conn;
+        }
+      };
+
+  private ConnectionConfigurator getConnConfigurator(SSLFactory sslFactoryObj) {
+    try {
+      return initSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, sslFactoryObj);
+    } catch (Exception e) {
+      LOG.debug("Cannot load customized ssl related configuration. "
+          + "Fallback to system-generic settings.", e);
+      return DEFAULT_TIMEOUT_CONN_CONFIGURATOR;
+    }
+  }
+
+  private static ConnectionConfigurator initSslConnConfigurator(
+      final int timeout, SSLFactory sslFactory)
+      throws IOException, GeneralSecurityException {
+    final SSLSocketFactory sf;
+    final HostnameVerifier hv;
+
+    sf = sslFactory.createSSLSocketFactory();
+    hv = sslFactory.getHostnameVerifier();
+
+    return new ConnectionConfigurator() {
+      @Override
+      public HttpURLConnection configure(HttpURLConnection conn)
+          throws IOException {
+        if (conn instanceof HttpsURLConnection) {
+          HttpsURLConnection c = (HttpsURLConnection) conn;
+          c.setSSLSocketFactory(sf);
+          c.setHostnameVerifier(hv);
+        }
+        setTimeouts(conn, timeout);
+        return conn;
+      }
+    };
+  }
+
+  protected SSLFactory getSSLFactory(Configuration conf)
+      throws GeneralSecurityException, IOException {
+    SSLFactory newSSLFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
+    newSSLFactory.init();
+    return newSSLFactory;
+  }
+
+  private static void setTimeouts(URLConnection connection, int socketTimeout) {
+    connection.setConnectTimeout(socketTimeout);
+    connection.setReadTimeout(socketTimeout);
+  }
+
+  public static URI constructResURI(Configuration conf, String address,
+      String uri) {
+    return URI.create(
+        JOINER.join(YarnConfiguration.useHttps(conf) ? "https://" : "http://",
+            address, uri));
+  }
+
+  DelegationTokenAuthenticatedURL getDelegationTokenAuthenticatedURL() {
+    return new DelegationTokenAuthenticatedURL(authenticator, connConfigurator);
+  }
+
+  protected void serviceStop() {
+    if (this.sslFactory != null) {
+      this.sslFactory.destroy();
+    }
+  }
+
+  public Client getClient() {
+    return client;
+  }
+
+  public Object operateDelegationToken(
+      final PrivilegedExceptionAction<?> action)
+      throws IOException, YarnException {
+    // Set up the retry operation
+    TimelineClientRetryOp tokenRetryOp =
+        createRetryOpForOperateDelegationToken(action);
+
+    return connectionRetry.retryOn(tokenRetryOp);
+  }
+
+  @Private
+  @VisibleForTesting
+  TimelineClientRetryOp createRetryOpForOperateDelegationToken(
+      final PrivilegedExceptionAction<?> action) throws IOException {
+    return new TimelineClientRetryOpForOperateDelegationToken(this.authUgi,
+        action);
+  }
+
+  /**
+   * Abstract class for an operation that should be retried by timeline client.
+   */
+  @Private
+  @VisibleForTesting
+  public static abstract class TimelineClientRetryOp {
+    // The operation that should be retried
+    public abstract Object run() throws IOException;
+
+    // The method to indicate if we should retry given the incoming exception
+    public abstract boolean shouldRetryOn(Exception e);
+  }
+
+  private static class TimelineURLConnectionFactory
+      implements HttpURLConnectionFactory {
+    private DelegationTokenAuthenticator authenticator;
+    private UserGroupInformation authUgi;
+    private ConnectionConfigurator connConfigurator;
+    private Token token;
+    private String doAsUser;
+
+    public TimelineURLConnectionFactory(UserGroupInformation authUgi,
+        DelegationTokenAuthenticator authenticator,
+        ConnectionConfigurator connConfigurator,
+        DelegationTokenAuthenticatedURL.Token token, String doAsUser) {
+      this.authUgi = authUgi;
+      this.authenticator = authenticator;
+      this.connConfigurator = connConfigurator;
+      this.token = token;
+      this.doAsUser = doAsUser;
+    }
+
+    @Override
+    public HttpURLConnection getHttpURLConnection(final URL url)
+        throws IOException {
+      authUgi.checkTGTAndReloginFromKeytab();
+      try {
+        return new DelegationTokenAuthenticatedURL(authenticator,
+            connConfigurator).openConnection(url, token, doAsUser);
+      } catch (UndeclaredThrowableException e) {
+        throw new IOException(e.getCause());
+      } catch (AuthenticationException ae) {
+        throw new IOException(ae);
+      }
+    }
+
+  }
+
+  // Class to handle retry
+  // Outside this class, only visible to tests
+  @Private
+  @VisibleForTesting
+  static class TimelineClientConnectionRetry {
+
+    // maxRetries < 0 means keep trying
+    @Private
+    @VisibleForTesting
+    public int maxRetries;
+
+    @Private
+    @VisibleForTesting
+    public long retryInterval;
+
+    // Indicates if retries happened last time. Only tests should read it.
+    // In unit tests, retryOn() calls should _not_ be concurrent.
+    private boolean retried = false;
+
+    @Private
+    @VisibleForTesting
+    boolean getRetired() {
+      return retried;
+    }
+
+    // Constructor with default retry settings
+    public TimelineClientConnectionRetry(Configuration conf) {
+      Preconditions.checkArgument(
+          conf.getInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
+              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES)
+              >= -1,
+          "%s property value should be greater than or equal to -1",
+          YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
+      Preconditions.checkArgument(
+          conf.getLong(
+              YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
+              YarnConfiguration.
+                DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS) > 0,
+          "%s property value should be greater than zero",
+          YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
+      maxRetries =
+          conf.getInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
+              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
+      retryInterval = conf.getLong(
+          YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
+          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
+    }
+
+    public Object retryOn(TimelineClientRetryOp op)
+        throws RuntimeException, IOException {
+      int leftRetries = maxRetries;
+      retried = false;
+
+      // keep trying
+      while (true) {
+        try {
+          // try perform the op, if fail, keep retrying
+          return op.run();
+        } catch (IOException | RuntimeException e) {
+          // break if there's no retries left
+          if (leftRetries == 0) {
+            break;
+          }
+          if (op.shouldRetryOn(e)) {
+            logException(e, leftRetries);
+          } else {
+            throw e;
+          }
+        }
+        if (leftRetries > 0) {
+          leftRetries--;
+        }
+        retried = true;
+        try {
+          // sleep for the given time interval
+          Thread.sleep(retryInterval);
+        } catch (InterruptedException ie) {
+          LOG.warn("Client retry sleep interrupted! ");
+        }
+      }
+      throw new RuntimeException("Failed to connect to timeline server. "
+          + "Connection retries limit exceeded. "
+          + "The posted timeline event may be missing");
+    };
+
+    private void logException(Exception e, int leftRetries) {
+      if (leftRetries > 0) {
+        LOG.info(
+            "Exception caught by TimelineClientConnectionRetry," + " will try "
+                + leftRetries + " more time(s).\nMessage: " + e.getMessage());
+      } else {
+        // note that maxRetries may be -1 at the very beginning
+        LOG.info("ConnectionException caught by TimelineClientConnectionRetry,"
+            + " will keep retrying.\nMessage: " + e.getMessage());
+      }
+    }
+  }
+
+  private static class TimelineJerseyRetryFilter extends ClientFilter {
+    private TimelineClientConnectionRetry connectionRetry;
+
+    public TimelineJerseyRetryFilter(
+        TimelineClientConnectionRetry connectionRetry) {
+      this.connectionRetry = connectionRetry;
+    }
+
+    @Override
+    public ClientResponse handle(final ClientRequest cr)
+        throws ClientHandlerException {
+      // Set up the retry operation
+      TimelineClientRetryOp jerseyRetryOp = new TimelineClientRetryOp() {
+        @Override
+        public Object run() {
+          // Try pass the request, if fail, keep retrying
+          return getNext().handle(cr);
+        }
+
+        @Override
+        public boolean shouldRetryOn(Exception e) {
+          // Only retry on connection exceptions
+          return (e instanceof ClientHandlerException)
+              && (e.getCause() instanceof ConnectException
+                  || e.getCause() instanceof SocketTimeoutException
+                  || e.getCause() instanceof SocketException);
+        }
+      };
+      try {
+        return (ClientResponse) connectionRetry.retryOn(jerseyRetryOp);
+      } catch (IOException e) {
+        throw new ClientHandlerException(
+            "Jersey retry failed!\nMessage: " + e.getMessage());
+      }
+    }
+  }
+
+  @Private
+  @VisibleForTesting
+  public static class TimelineClientRetryOpForOperateDelegationToken
+      extends TimelineClientRetryOp {
+
+    private final UserGroupInformation authUgi;
+    private final PrivilegedExceptionAction<?> action;
+
+    public TimelineClientRetryOpForOperateDelegationToken(
+        UserGroupInformation authUgi, PrivilegedExceptionAction<?> action) {
+      this.authUgi = authUgi;
+      this.action = action;
+    }
+
+    @Override
+    public Object run() throws IOException {
+      // Try pass the request, if fail, keep retrying
+      authUgi.checkTGTAndReloginFromKeytab();
+      try {
+        return authUgi.doAs(action);
+      } catch (UndeclaredThrowableException e) {
+        throw new IOException(e.getCause());
+      } catch (InterruptedException e) {
+        throw new IOException(e);
+      }
+    }
+
+    @Override
+    public boolean shouldRetryOn(Exception e) {
+      // retry on connection exceptions
+      // and SocketTimeoutException
+      return (e instanceof ConnectException
+          || e instanceof SocketTimeoutException);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
new file mode 100644
index 0000000..848e238
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineV2ClientImpl.java
@@ -0,0 +1,459 @@
+/**
+ * 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.client.api.impl;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.core.util.MultivaluedMapImpl;
+
+/**
+ * Implementation of timeline v2 client interface.
+ *
+ */
+public class TimelineV2ClientImpl extends TimelineV2Client {
+  private static final Log LOG = LogFactory.getLog(TimelineV2ClientImpl.class);
+
+  private static final String RESOURCE_URI_STR_V2 = "/ws/v2/timeline/";
+
+  private TimelineEntityDispatcher entityDispatcher;
+  private volatile String timelineServiceAddress;
+
+  // Retry parameters for identifying new timeline service
+  // TODO consider to merge with connection retry
+  private int maxServiceRetries;
+  private long serviceRetryInterval;
+
+  private TimelineConnector connector;
+
+  private ApplicationId contextAppId;
+
+  public TimelineV2ClientImpl(ApplicationId appId) {
+    super(TimelineV2ClientImpl.class.getName());
+    this.contextAppId = appId;
+  }
+
+  public ApplicationId getContextAppId() {
+    return contextAppId;
+  }
+
+  protected void serviceInit(Configuration conf) throws Exception {
+    if (!YarnConfiguration.timelineServiceEnabled(conf)
+        || (int) YarnConfiguration.getTimelineServiceVersion(conf) != 2) {
+      throw new IOException("Timeline V2 client is not properly configured. "
+          + "Either timeline service is not enabled or version is not set to"
+          + " 2");
+    }
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    UserGroupInformation realUgi = ugi.getRealUser();
+    String doAsUser = null;
+    UserGroupInformation authUgi = null;
+    if (realUgi != null) {
+      authUgi = realUgi;
+      doAsUser = ugi.getShortUserName();
+    } else {
+      authUgi = ugi;
+      doAsUser = null;
+    }
+
+    // TODO need to add/cleanup filter retry later for ATSV2. similar to V1
+    DelegationTokenAuthenticatedURL.Token token =
+        new DelegationTokenAuthenticatedURL.Token();
+    connector = new TimelineConnector(false, authUgi, doAsUser, token);
+    addIfService(connector);
+
+    // new version need to auto discovery (with retry till ATS v2 address is
+    // got).
+    maxServiceRetries =
+        conf.getInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
+    serviceRetryInterval = conf.getLong(
+        YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
+    entityDispatcher = new TimelineEntityDispatcher(conf);
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    entityDispatcher.start();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    entityDispatcher.stop();
+    super.serviceStop();
+  }
+
+  @Override
+  public void putEntities(TimelineEntity... entities)
+      throws IOException, YarnException {
+    entityDispatcher.dispatchEntities(true, entities);
+  }
+
+  @Override
+  public void putEntitiesAsync(TimelineEntity... entities)
+      throws IOException, YarnException {
+    entityDispatcher.dispatchEntities(false, entities);
+  }
+
+  @Override
+  public void setTimelineServiceAddress(String address) {
+    this.timelineServiceAddress = address;
+  }
+
+  @Private
+  protected void putObjects(String path, MultivaluedMap<String, String> params,
+      Object obj) throws IOException, YarnException {
+
+    int retries = verifyRestEndPointAvailable();
+
+    // timelineServiceAddress could be stale, add retry logic here.
+    boolean needRetry = true;
+    while (needRetry) {
+      try {
+        URI uri = TimelineConnector.constructResURI(getConfig(),
+            timelineServiceAddress, RESOURCE_URI_STR_V2);
+        putObjects(uri, path, params, obj);
+        needRetry = false;
+      } catch (IOException e) {
+        // handle exception for timelineServiceAddress being updated.
+        checkRetryWithSleep(retries, e);
+        retries--;
+      }
+    }
+  }
+
+  /**
+   * Check if reaching to maximum of retries.
+   *
+   * @param retries
+   * @param e
+   */
+  private void checkRetryWithSleep(int retries, IOException e)
+      throws YarnException, IOException {
+    if (retries > 0) {
+      try {
+        Thread.sleep(this.serviceRetryInterval);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        throw new YarnException("Interrupted while retrying to connect to ATS");
+      }
+    } else {
+      StringBuilder msg =
+          new StringBuilder("TimelineClient has reached to max retry times : ");
+      msg.append(this.maxServiceRetries);
+      msg.append(" for service address: ");
+      msg.append(timelineServiceAddress);
+      LOG.error(msg.toString());
+      throw new IOException(msg.toString(), e);
+    }
+  }
+
+  protected void putObjects(URI base, String path,
+      MultivaluedMap<String, String> params, Object obj)
+      throws IOException, YarnException {
+    ClientResponse resp;
+    try {
+      resp = connector.getClient().resource(base).path(path).queryParams(params)
+          .accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_JSON)
+          .put(ClientResponse.class, obj);
+    } catch (RuntimeException re) {
+      // runtime exception is expected if the client cannot connect the server
+      String msg = "Failed to get the response from the timeline server.";
+      LOG.error(msg, re);
+      throw new IOException(re);
+    }
+    if (resp == null ||
+        resp.getClientResponseStatus() != ClientResponse.Status.OK) {
+      String msg =
+          "Response from the timeline server is " + ((resp == null) ? "null"
+              : "not successful," + " HTTP error code: " + resp.getStatus()
+                  + ", Server response:\n" + resp.getEntity(String.class));
+      LOG.error(msg);
+      throw new YarnException(msg);
+    }
+  }
+
+  private int verifyRestEndPointAvailable() throws YarnException {
+    // timelineServiceAddress could haven't be initialized yet
+    // or stale (only for new timeline service)
+    int retries = pollTimelineServiceAddress(this.maxServiceRetries);
+    if (timelineServiceAddress == null) {
+      String errMessage = "TimelineClient has reached to max retry times : "
+          + this.maxServiceRetries
+          + ", but failed to fetch timeline service address. Please verify"
+          + " Timeline Auxiliary Service is configured in all the NMs";
+      LOG.error(errMessage);
+      throw new YarnException(errMessage);
+    }
+    return retries;
+  }
+
+  /**
+   * Poll TimelineServiceAddress for maximum of retries times if it is null.
+   *
+   * @param retries
+   * @return the left retry times
+   * @throws IOException
+   */
+  private int pollTimelineServiceAddress(int retries) throws YarnException {
+    while (timelineServiceAddress == null && retries > 0) {
+      try {
+        Thread.sleep(this.serviceRetryInterval);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new YarnException("Interrupted while trying to connect ATS");
+      }
+      retries--;
+    }
+    return retries;
+  }
+
+  private final class EntitiesHolder extends FutureTask<Void> {
+    private final TimelineEntities entities;
+    private final boolean isSync;
+
+    EntitiesHolder(final TimelineEntities entities, final boolean isSync) {
+      super(new Callable<Void>() {
+        // publishEntities()
+        public Void call() throws Exception {
+          MultivaluedMap<String, String> params = new MultivaluedMapImpl();
+          params.add("appid", getContextAppId().toString());
+          params.add("async", Boolean.toString(!isSync));
+          putObjects("entities", params, entities);
+          return null;
+        }
+      });
+      this.entities = entities;
+      this.isSync = isSync;
+    }
+
+    public boolean isSync() {
+      return isSync;
+    }
+
+    public TimelineEntities getEntities() {
+      return entities;
+    }
+  }
+
+  /**
+   * This class is responsible for collecting the timeline entities and
+   * publishing them in async.
+   */
+  private class TimelineEntityDispatcher {
+    /**
+     * Time period for which the timelineclient will wait for draining after
+     * stop.
+     */
+    private static final long DRAIN_TIME_PERIOD = 2000L;
+
+    private int numberOfAsyncsToMerge;
+    private final BlockingQueue<EntitiesHolder> timelineEntityQueue;
+    private ExecutorService executor;
+
+    TimelineEntityDispatcher(Configuration conf) {
+      timelineEntityQueue = new LinkedBlockingQueue<EntitiesHolder>();
+      numberOfAsyncsToMerge =
+          conf.getInt(YarnConfiguration.NUMBER_OF_ASYNC_ENTITIES_TO_MERGE,
+              YarnConfiguration.DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE);
+    }
+
+    Runnable createRunnable() {
+      return new Runnable() {
+        @Override
+        public void run() {
+          try {
+            EntitiesHolder entitiesHolder;
+            while (!Thread.currentThread().isInterrupted()) {
+              // Merge all the async calls and make one push, but if its sync
+              // call push immediately
+              try {
+                entitiesHolder = timelineEntityQueue.take();
+              } catch (InterruptedException ie) {
+                LOG.info("Timeline dispatcher thread was interrupted ");
+                Thread.currentThread().interrupt();
+                return;
+              }
+              if (entitiesHolder != null) {
+                publishWithoutBlockingOnQueue(entitiesHolder);
+              }
+            }
+          } finally {
+            if (!timelineEntityQueue.isEmpty()) {
+              LOG.info("Yet to publish " + timelineEntityQueue.size()
+                  + " timelineEntities, draining them now. ");
+            }
+            // Try to drain the remaining entities to be published @ the max for
+            // 2 seconds
+            long timeTillweDrain =
+                System.currentTimeMillis() + DRAIN_TIME_PERIOD;
+            while (!timelineEntityQueue.isEmpty()) {
+              publishWithoutBlockingOnQueue(timelineEntityQueue.poll());
+              if (System.currentTimeMillis() > timeTillweDrain) {
+                // time elapsed stop publishing further....
+                if (!timelineEntityQueue.isEmpty()) {
+                  LOG.warn("Time to drain elapsed! Remaining "
+                      + timelineEntityQueue.size() + "timelineEntities will not"
+                      + " be published");
+                  // if some entities were not drained then we need interrupt
+                  // the threads which had put sync EntityHolders to the queue.
+                  EntitiesHolder nextEntityInTheQueue = null;
+                  while ((nextEntityInTheQueue =
+                      timelineEntityQueue.poll()) != null) {
+                    nextEntityInTheQueue.cancel(true);
+                  }
+                }
+                break;
+              }
+            }
+          }
+        }
+
+        /**
+         * Publishes the given EntitiesHolder and return immediately if sync
+         * call, else tries to fetch the EntitiesHolder from the queue in non
+         * blocking fashion and collate the Entities if possible before
+         * publishing through REST.
+         *
+         * @param entitiesHolder
+         */
+        private void publishWithoutBlockingOnQueue(
+            EntitiesHolder entitiesHolder) {
+          if (entitiesHolder.isSync()) {
+            entitiesHolder.run();
+            return;
+          }
+          int count = 1;
+          while (true) {
+            // loop till we find a sync put Entities or there is nothing
+            // to take
+            EntitiesHolder nextEntityInTheQueue = timelineEntityQueue.poll();
+            if (nextEntityInTheQueue == null) {
+              // Nothing in the queue just publish and get back to the
+              // blocked wait state
+              entitiesHolder.run();
+              break;
+            } else if (nextEntityInTheQueue.isSync()) {
+              // flush all the prev async entities first
+              entitiesHolder.run();
+              // and then flush the sync entity
+              nextEntityInTheQueue.run();
+              break;
+            } else {
+              // append all async entities together and then flush
+              entitiesHolder.getEntities().addEntities(
+                  nextEntityInTheQueue.getEntities().getEntities());
+              count++;
+              if (count == numberOfAsyncsToMerge) {
+                // Flush the entities if the number of the async
+                // putEntites merged reaches the desired limit. To avoid
+                // collecting multiple entities and delaying for a long
+                // time.
+                entitiesHolder.run();
+                break;
+              }
+            }
+          }
+        }
+      };
+    }
+
+    public void dispatchEntities(boolean sync,
+        TimelineEntity[] entitiesTobePublished) throws YarnException {
+      if (executor.isShutdown()) {
+        throw new YarnException("Timeline client is in the process of stopping,"
+            + " not accepting any more TimelineEntities");
+      }
+
+      // wrap all TimelineEntity into TimelineEntities object
+      TimelineEntities entities = new TimelineEntities();
+      for (TimelineEntity entity : entitiesTobePublished) {
+        entities.addEntity(entity);
+      }
+
+      // created a holder and place it in queue
+      EntitiesHolder entitiesHolder = new EntitiesHolder(entities, sync);
+      try {
+        timelineEntityQueue.put(entitiesHolder);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new YarnException(
+            "Failed while adding entity to the queue for publishing", e);
+      }
+
+      if (sync) {
+        // In sync call we need to wait till its published and if any error then
+        // throw it back
+        try {
+          entitiesHolder.get();
+        } catch (ExecutionException e) {
+          throw new YarnException("Failed while publishing entity",
+              e.getCause());
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new YarnException("Interrupted while publishing entity", e);
+        }
+      }
+    }
+
+    public void start() {
+      executor = Executors.newSingleThreadExecutor();
+      executor.execute(createRunnable());
+    }
+
+    public void stop() {
+      LOG.info("Stopping TimelineClient.");
+      executor.shutdownNow();
+      try {
+        executor.awaitTermination(DRAIN_TIME_PERIOD, TimeUnit.MILLISECONDS);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        e.printStackTrace();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
index a1d4449..a44a8ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClient.java
@@ -215,11 +215,11 @@ public class TestTimelineClient {
           + "Timeline server should be off to run this test. ");
     } catch (RuntimeException ce) {
       Assert.assertTrue(
-        "Handler exception for reason other than retry: " + ce.getMessage(),
-        ce.getMessage().contains("Connection retries limit exceeded"));
+          "Handler exception for reason other than retry: " + ce.getMessage(),
+          ce.getMessage().contains("Connection retries limit exceeded"));
       // we would expect this exception here, check if the client has retried
-      Assert.assertTrue("Retry filter didn't perform any retries! ", client
-        .connectionRetry.getRetired());
+      Assert.assertTrue("Retry filter didn't perform any retries! ",
+          client.connector.connectionRetry.getRetired());
     }
   }
 
@@ -318,7 +318,7 @@ public class TestTimelineClient {
             .getMessage().contains("Connection retries limit exceeded"));
     // we would expect this exception here, check if the client has retried
     Assert.assertTrue("Retry filter didn't perform any retries! ",
-        client.connectionRetry.getRetired());
+        client.connector.connectionRetry.getRetired());
   }
 
   public static ClientResponse mockEntityClientResponse(
@@ -419,17 +419,26 @@ public class TestTimelineClient {
   private TimelineClientImpl createTimelineClientFakeTimelineClientRetryOp(
       YarnConfiguration conf) {
     TimelineClientImpl client = new TimelineClientImpl() {
-
       @Override
-      public TimelineClientRetryOp
-          createTimelineClientRetryOpForOperateDelegationToken(
-              final PrivilegedExceptionAction<?> action) throws IOException {
-        TimelineClientRetryOpForOperateDelegationToken op =
-            spy(new TimelineClientRetryOpForOperateDelegationToken(
-            UserGroupInformation.getCurrentUser(), action));
-        doThrow(new SocketTimeoutException("Test socketTimeoutException"))
-            .when(op).run();
-        return op;
+      protected TimelineConnector createTimelineConnector() {
+        TimelineConnector connector =
+            new TimelineConnector(true, authUgi, doAsUser, token) {
+              @Override
+              public TimelineClientRetryOp
+                createRetryOpForOperateDelegationToken(
+                  final PrivilegedExceptionAction<?> action)
+                  throws IOException {
+                TimelineClientRetryOpForOperateDelegationToken op =
+                    spy(new TimelineClientRetryOpForOperateDelegationToken(
+                        UserGroupInformation.getCurrentUser(), action));
+                doThrow(
+                    new SocketTimeoutException("Test socketTimeoutException"))
+                        .when(op).run();
+                return op;
+              }
+            };
+        addIfService(connector);
+        return connector;
       }
     };
     client.init(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java
index 5813340..c5b02fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestTimelineClientV2Impl.java
@@ -50,7 +50,7 @@ public class TestTimelineClientV2Impl {
   public void setup() {
     conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
-    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.0f);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
     conf.setInt(YarnConfiguration.NUMBER_OF_ASYNC_ENTITIES_TO_MERGE, 3);
     if (!currTestName.getMethodName()
         .contains("testRetryOnConnectionFailure")) {
@@ -71,7 +71,7 @@ public class TestTimelineClientV2Impl {
   }
 
   private class TestV2TimelineClientForExceptionHandling
-      extends TimelineClientImpl {
+      extends TimelineV2ClientImpl {
     public TestV2TimelineClientForExceptionHandling(ApplicationId id) {
       super(id);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
index 45b9213..851ba53 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelinePublisher.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -77,7 +77,7 @@ public class NMTimelinePublisher extends CompositeService {
 
   private String httpAddress;
 
-  private final Map<ApplicationId, TimelineClient> appToClientMap;
+  private final Map<ApplicationId, TimelineV2Client> appToClientMap;
 
   public NMTimelinePublisher(Context context) {
     super(NMTimelinePublisher.class.getName());
@@ -103,7 +103,7 @@ public class NMTimelinePublisher extends CompositeService {
   }
 
   @VisibleForTesting
-  Map<ApplicationId, TimelineClient> getAppToClientMap() {
+  Map<ApplicationId, TimelineV2Client> getAppToClientMap() {
     return appToClientMap;
   }
 
@@ -148,7 +148,7 @@ public class NMTimelinePublisher extends CompositeService {
       try {
         // no need to put it as part of publisher as timeline client already has
         // Queuing concept
-        TimelineClient timelineClient = getTimelineClient(appId);
+        TimelineV2Client timelineClient = getTimelineClient(appId);
         if (timelineClient != null) {
           timelineClient.putEntitiesAsync(entity);
         } else {
@@ -242,7 +242,7 @@ public class NMTimelinePublisher extends CompositeService {
     try {
       // no need to put it as part of publisher as timeline client already has
       // Queuing concept
-      TimelineClient timelineClient = getTimelineClient(appId);
+      TimelineV2Client timelineClient = getTimelineClient(appId);
       if (timelineClient != null) {
         timelineClient.putEntitiesAsync(entity);
       } else {
@@ -273,7 +273,7 @@ public class NMTimelinePublisher extends CompositeService {
         LOG.debug("Publishing the entity " + entity + ", JSON-style content: "
             + TimelineUtils.dumpTimelineRecordtoJSON(entity));
       }
-      TimelineClient timelineClient = getTimelineClient(appId);
+      TimelineV2Client timelineClient = getTimelineClient(appId);
       if (timelineClient != null) {
         timelineClient.putEntities(entity);
       } else {
@@ -390,8 +390,8 @@ public class NMTimelinePublisher extends CompositeService {
 
   public void createTimelineClient(ApplicationId appId) {
     if (!appToClientMap.containsKey(appId)) {
-      TimelineClient timelineClient =
-          TimelineClient.createTimelineClient(appId);
+      TimelineV2Client timelineClient =
+          TimelineV2Client.createTimelineClient(appId);
       timelineClient.init(getConfig());
       timelineClient.start();
       appToClientMap.put(appId, timelineClient);
@@ -399,7 +399,7 @@ public class NMTimelinePublisher extends CompositeService {
   }
 
   public void stopTimelineClient(ApplicationId appId) {
-    TimelineClient client = appToClientMap.remove(appId);
+    TimelineV2Client client = appToClientMap.remove(appId);
     if (client != null) {
       client.stop();
     }
@@ -407,13 +407,13 @@ public class NMTimelinePublisher extends CompositeService {
 
   public void setTimelineServiceAddress(ApplicationId appId,
       String collectorAddr) {
-    TimelineClient client = appToClientMap.get(appId);
+    TimelineV2Client client = appToClientMap.get(appId);
     if (client != null) {
       client.setTimelineServiceAddress(collectorAddr);
     }
   }
 
-  private TimelineClient getTimelineClient(ApplicationId appId) {
+  private TimelineV2Client getTimelineClient(ApplicationId appId) {
     return appToClientMap.get(appId);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
index ae9397a..e116122 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/TestNMTimelinePublisher.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
-import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
+import org.apache.hadoop.yarn.client.api.impl.TimelineV2ClientImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
@@ -50,7 +50,7 @@ public class TestNMTimelinePublisher {
   public void testContainerResourceUsage() {
     Context context = mock(Context.class);
     @SuppressWarnings("unchecked")
-    final DummyTimelineClient timelineClient = new DummyTimelineClient();
+    final DummyTimelineClient timelineClient = new DummyTimelineClient(null);
     when(context.getNodeId()).thenReturn(NodeId.newInstance("localhost", 0));
     when(context.getHttpPort()).thenReturn(0);
     NMTimelinePublisher publisher = new NMTimelinePublisher(context) {
@@ -137,7 +137,11 @@ public class TestNMTimelinePublisher {
     }
   }
 
-  protected static class DummyTimelineClient extends TimelineClientImpl {
+  protected static class DummyTimelineClient extends TimelineV2ClientImpl {
+    public DummyTimelineClient(ApplicationId appId) {
+      super(appId);
+    }
+
     private TimelineEntity[] lastPublishedEntities;
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
index 3ec222f..07058f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/TestTimelineServiceClientIntegration.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.QueueEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.UserEntity;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.CollectorNodemanagerProtocol;
@@ -96,8 +96,8 @@ public class TestTimelineServiceClientIntegration {
 
   @Test
   public void testPutEntities() throws Exception {
-    TimelineClient client =
-        TimelineClient.createTimelineClient(ApplicationId.newInstance(0, 1));
+    TimelineV2Client client =
+        TimelineV2Client.createTimelineClient(ApplicationId.newInstance(0, 1));
     try {
       // set the timeline service address manually
       client.setTimelineServiceAddress(
@@ -123,8 +123,8 @@ public class TestTimelineServiceClientIntegration {
   @Test
   public void testPutExtendedEntities() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(0, 1);
-    TimelineClient client =
-        TimelineClient.createTimelineClient(appId);
+    TimelineV2Client client =
+        TimelineV2Client.createTimelineClient(appId);
     try {
       // set the timeline service address manually
       client.setTimelineServiceAddress(


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


[09/50] [abbrv] hadoop git commit: YARN-4675. Reorganize TimelineClient and TimelineClientImpl into separate classes for ATSv1.x and ATSv2. Contributed by Naganarasimha G R.

Posted by va...@apache.org.
YARN-4675. Reorganize TimelineClient and TimelineClientImpl into separate classes for ATSv1.x and ATSv2. Contributed by Naganarasimha G R.

(cherry picked from commit 73235ab30361b41293846189f3c5fef321ae7cac)


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

Branch: refs/heads/YARN-5355_branch2
Commit: e7eed958f7d88cdcb5614d6f0d9f6d1cf1d4a43f
Parents: 673ab90
Author: Sangjin Lee <sj...@apache.org>
Authored: Thu Feb 16 18:43:31 2017 -0800
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:44 2017 +0530

----------------------------------------------------------------------
 .../jobhistory/JobHistoryEventHandler.java      |  57 +-
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |  14 +-
 .../v2/app/rm/RMContainerAllocator.java         |   4 +-
 .../jobhistory/TestJobHistoryEventHandler.java  |   8 +-
 .../distributedshell/ApplicationMaster.java     | 113 +--
 .../hadoop/yarn/client/api/AMRMClient.java      |  38 +-
 .../yarn/client/api/async/AMRMClientAsync.java  |  19 +-
 .../api/async/impl/AMRMClientAsyncImpl.java     |   5 +-
 .../yarn/client/api/impl/YarnClientImpl.java    |  15 +-
 .../hadoop/yarn/client/api/TimelineClient.java  |  94 +--
 .../yarn/client/api/TimelineV2Client.java       |  92 +++
 .../client/api/impl/TimelineClientImpl.java     | 825 ++-----------------
 .../yarn/client/api/impl/TimelineConnector.java | 442 ++++++++++
 .../client/api/impl/TimelineV2ClientImpl.java   | 459 +++++++++++
 .../client/api/impl/TestTimelineClient.java     |  39 +-
 .../api/impl/TestTimelineClientV2Impl.java      |   4 +-
 .../timelineservice/NMTimelinePublisher.java    |  22 +-
 .../TestNMTimelinePublisher.java                |  10 +-
 .../TestTimelineServiceClientIntegration.java   |  10 +-
 19 files changed, 1278 insertions(+), 992 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
index 1cf7182..5edd189 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
@@ -72,13 +72,12 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.util.TimelineServiceHelper;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.node.JsonNodeFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.sun.jersey.api.client.ClientHandlerException;
@@ -91,8 +90,6 @@ import com.sun.jersey.api.client.ClientHandlerException;
  */
 public class JobHistoryEventHandler extends AbstractService
     implements EventHandler<JobHistoryEvent> {
-  private static final JsonNodeFactory FACTORY =
-      new ObjectMapper().getNodeFactory();
 
   private final AppContext context;
   private final int startCount;
@@ -134,9 +131,10 @@ public class JobHistoryEventHandler extends AbstractService
   // should job completion be force when the AM shuts down?
   protected volatile boolean forceJobCompletion = false;
 
+  @VisibleForTesting
   protected TimelineClient timelineClient;
-
-  private boolean timelineServiceV2Enabled = false;
+  @VisibleForTesting
+  protected TimelineV2Client timelineV2Client;
 
   private static String MAPREDUCE_JOB_ENTITY_TYPE = "MAPREDUCE_JOB";
   private static String MAPREDUCE_TASK_ENTITY_TYPE = "MAPREDUCE_TASK";
@@ -269,12 +267,17 @@ public class JobHistoryEventHandler extends AbstractService
         MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA)) {
       LOG.info("Emitting job history data to the timeline service is enabled");
       if (YarnConfiguration.timelineServiceEnabled(conf)) {
-
-        timelineClient =
-            ((MRAppMaster.RunningAppContext)context).getTimelineClient();
-        timelineClient.init(conf);
-        timelineServiceV2Enabled =
-            YarnConfiguration.timelineServiceV2Enabled(conf);
+        boolean timelineServiceV2Enabled =
+            ((int) YarnConfiguration.getTimelineServiceVersion(conf) == 2);
+        if(timelineServiceV2Enabled) {
+          timelineV2Client =
+              ((MRAppMaster.RunningAppContext)context).getTimelineV2Client();
+          timelineV2Client.init(conf);
+        } else {
+          timelineClient =
+              ((MRAppMaster.RunningAppContext) context).getTimelineClient();
+          timelineClient.init(conf);
+        }
         LOG.info("Timeline service is enabled; version: " +
             YarnConfiguration.getTimelineServiceVersion(conf));
       } else {
@@ -325,6 +328,8 @@ public class JobHistoryEventHandler extends AbstractService
   protected void serviceStart() throws Exception {
     if (timelineClient != null) {
       timelineClient.start();
+    } else if (timelineV2Client != null) {
+      timelineV2Client.start();
     }
     eventHandlingThread = new Thread(new Runnable() {
       @Override
@@ -449,6 +454,8 @@ public class JobHistoryEventHandler extends AbstractService
     }
     if (timelineClient != null) {
       timelineClient.stop();
+    } else if (timelineV2Client != null) {
+      timelineV2Client.stop();
     }
     LOG.info("Stopped JobHistoryEventHandler. super.stop()");
     super.serviceStop();
@@ -606,14 +613,12 @@ public class JobHistoryEventHandler extends AbstractService
         }
         processEventForJobSummary(event.getHistoryEvent(), mi.getJobSummary(),
             event.getJobID());
-        if (timelineClient != null) {
-          if (timelineServiceV2Enabled) {
-            processEventForNewTimelineService(historyEvent, event.getJobID(),
-                event.getTimestamp());
-          } else {
-            processEventForTimelineServer(historyEvent, event.getJobID(),
-                event.getTimestamp());
-          }
+        if (timelineV2Client != null) {
+          processEventForNewTimelineService(historyEvent, event.getJobID(),
+              event.getTimestamp());
+        } else if (timelineClient != null) {
+          processEventForTimelineServer(historyEvent, event.getJobID(),
+              event.getTimestamp());
         }
         if (LOG.isDebugEnabled()) {
           LOG.debug("In HistoryEventHandler "
@@ -1166,8 +1171,8 @@ public class JobHistoryEventHandler extends AbstractService
         configSize += size;
         if (configSize > JobHistoryEventUtils.ATS_CONFIG_PUBLISH_SIZE_BYTES) {
           if (jobEntityForConfigs.getConfigs().size() > 0) {
-            timelineClient.putEntities(jobEntityForConfigs);
-            timelineClient.putEntities(appEntityForConfigs);
+            timelineV2Client.putEntities(jobEntityForConfigs);
+            timelineV2Client.putEntities(appEntityForConfigs);
             jobEntityForConfigs = createJobEntity(jobId);
             appEntityForConfigs = new ApplicationEntity();
             appEntityForConfigs.setId(appId);
@@ -1178,8 +1183,8 @@ public class JobHistoryEventHandler extends AbstractService
         appEntityForConfigs.addConfig(entry.getKey(), entry.getValue());
       }
       if (configSize > 0) {
-        timelineClient.putEntities(jobEntityForConfigs);
-        timelineClient.putEntities(appEntityForConfigs);
+        timelineV2Client.putEntities(jobEntityForConfigs);
+        timelineV2Client.putEntities(appEntityForConfigs);
       }
     } catch (IOException | YarnException e) {
       LOG.error("Exception while publishing configs on JOB_SUBMITTED Event " +
@@ -1315,9 +1320,9 @@ public class JobHistoryEventHandler extends AbstractService
     }
     try {
       if (appEntityWithJobMetrics == null) {
-        timelineClient.putEntitiesAsync(tEntity);
+        timelineV2Client.putEntitiesAsync(tEntity);
       } else {
-        timelineClient.putEntities(tEntity, appEntityWithJobMetrics);
+        timelineV2Client.putEntities(tEntity, appEntityWithJobMetrics);
       }
     } catch (IOException | YarnException e) {
       LOG.error("Failed to process Event " + event.getEventType()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
index 73f9280..b4192fe 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
@@ -38,6 +38,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import javax.crypto.KeyGenerator;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -139,6 +141,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -152,8 +155,6 @@ import org.apache.hadoop.yarn.util.SystemClock;
 
 import com.google.common.annotations.VisibleForTesting;
 
-import javax.crypto.KeyGenerator;
-
 /**
  * The Map-Reduce Application Master.
  * The state machine is encapsulated in the implementation of Job interface.
@@ -1048,6 +1049,7 @@ public class MRAppMaster extends CompositeService {
     private final ClusterInfo clusterInfo = new ClusterInfo();
     private final ClientToAMTokenSecretManager clientToAMTokenSecretManager;
     private TimelineClient timelineClient = null;
+    private TimelineV2Client timelineV2Client = null;
 
     private final TaskAttemptFinishingMonitor taskAttemptFinishingMonitor;
 
@@ -1063,7 +1065,7 @@ public class MRAppMaster extends CompositeService {
 
         if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
           // create new version TimelineClient
-          timelineClient = TimelineClient.createTimelineClient(
+          timelineV2Client = TimelineV2Client.createTimelineClient(
               appAttemptID.getApplicationId());
         } else {
           timelineClient = TimelineClient.createTimelineClient();
@@ -1159,10 +1161,14 @@ public class MRAppMaster extends CompositeService {
       return taskAttemptFinishingMonitor;
     }
 
-    // Get Timeline Collector's address (get sync from RM)
     public TimelineClient getTimelineClient() {
       return timelineClient;
     }
+
+    // Get Timeline Collector's address (get sync from RM)
+    public TimelineV2Client getTimelineV2Client() {
+      return timelineV2Client;
+    }
   }
 
   @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
index 01e3430..218e218 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
@@ -864,8 +864,8 @@ public class RMContainerAllocator extends RMContainerRequestor
     MRAppMaster.RunningAppContext appContext =
         (MRAppMaster.RunningAppContext)this.getContext();
     if (collectorAddr != null && !collectorAddr.isEmpty()
-        && appContext.getTimelineClient() != null) {
-      appContext.getTimelineClient().setTimelineServiceAddress(
+        && appContext.getTimelineV2Client() != null) {
+      appContext.getTimelineV2Client().setTimelineServiceAddress(
           response.getCollectorAddr());
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
index c42ce14..33bb8b0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
@@ -29,8 +29,8 @@ import static org.mockito.Mockito.when;
 
 import java.io.File;
 import java.io.FileOutputStream;
-import java.io.InputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.util.HashMap;
 
 import org.apache.commons.logging.Log;
@@ -69,6 +69,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
@@ -828,6 +829,9 @@ public class TestJobHistoryEventHandler {
     if (mockContext instanceof RunningAppContext) {
       when(((RunningAppContext)mockContext).getTimelineClient()).
           thenReturn(TimelineClient.createTimelineClient());
+      when(((RunningAppContext) mockContext).getTimelineV2Client())
+          .thenReturn(TimelineV2Client
+              .createTimelineClient(ApplicationId.newInstance(0, 1)));
     }
     return mockContext;
   }
@@ -936,6 +940,8 @@ class JHEvenHandlerForTest extends JobHistoryEventHandler {
   protected void serviceStart() {
     if (timelineClient != null) {
       timelineClient.start();
+    } else if (timelineV2Client != null) {
+      timelineV2Client.start();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
index f7a4195..706e5b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
@@ -97,6 +97,7 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
 import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl;
@@ -221,7 +222,9 @@ public class ApplicationMaster {
   // Tracking url to which app master publishes info for clients to monitor
   private String appMasterTrackingUrl = "";
 
-  private boolean timelineServiceV2 = false;
+  private boolean timelineServiceV2Enabled = false;
+
+  private boolean timelineServiceV1Enabled = false;
 
   // App Master configuration
   // No. of containers to run shell command on
@@ -295,6 +298,10 @@ public class ApplicationMaster {
   // Timeline Client
   @VisibleForTesting
   TimelineClient timelineClient;
+
+  // Timeline v2 Client
+  private TimelineV2Client timelineV2Client;
+
   static final String CONTAINER_ENTITY_GROUP_ID = "CONTAINERS";
   static final String APPID_TIMELINE_FILTER_NAME = "appId";
   static final String USER_TIMELINE_FILTER_NAME = "user";
@@ -569,9 +576,12 @@ public class ApplicationMaster {
         "container_retry_interval", "0"));
 
     if (YarnConfiguration.timelineServiceEnabled(conf)) {
-      timelineServiceV2 = YarnConfiguration.timelineServiceV2Enabled(conf);
+      timelineServiceV2Enabled =
+          ((int) YarnConfiguration.getTimelineServiceVersion(conf) == 2);
+      timelineServiceV1Enabled = !timelineServiceV2Enabled;
     } else {
       timelineClient = null;
+      timelineV2Client = null;
       LOG.warn("Timeline service is not enabled");
     }
 
@@ -634,18 +644,17 @@ public class ApplicationMaster {
     nmClientAsync.start();
 
     startTimelineClient(conf);
-    if (timelineServiceV2) {
+    if (timelineServiceV2Enabled) {
       // need to bind timelineClient
-      amRMClient.registerTimelineClient(timelineClient);
+      amRMClient.registerTimelineV2Client(timelineV2Client);
     }
-    if(timelineClient != null) {
-      if (timelineServiceV2) {
-        publishApplicationAttemptEventOnTimelineServiceV2(
-            DSEvent.DS_APP_ATTEMPT_START);
-      } else {
-        publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
-            DSEvent.DS_APP_ATTEMPT_START, domainId, appSubmitterUgi);
-      }
+
+    if (timelineServiceV2Enabled) {
+      publishApplicationAttemptEventOnTimelineServiceV2(
+          DSEvent.DS_APP_ATTEMPT_START);
+    } else if (timelineServiceV1Enabled) {
+      publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
+          DSEvent.DS_APP_ATTEMPT_START, domainId, appSubmitterUgi);
     }
 
     // Setup local RPC Server to accept status requests directly from clients
@@ -717,18 +726,21 @@ public class ApplicationMaster {
         public Void run() throws Exception {
           if (YarnConfiguration.timelineServiceEnabled(conf)) {
             // Creating the Timeline Client
-            if (timelineServiceV2) {
-              timelineClient = TimelineClient.createTimelineClient(
+            if (timelineServiceV2Enabled) {
+              timelineV2Client = TimelineV2Client.createTimelineClient(
                   appAttemptID.getApplicationId());
+              timelineV2Client.init(conf);
+              timelineV2Client.start();
               LOG.info("Timeline service V2 client is enabled");
             } else {
               timelineClient = TimelineClient.createTimelineClient();
+              timelineClient.init(conf);
+              timelineClient.start();
               LOG.info("Timeline service V1 client is enabled");
             }
-            timelineClient.init(conf);
-            timelineClient.start();
           } else {
             timelineClient = null;
+            timelineV2Client = null;
             LOG.warn("Timeline service is not enabled");
           }
           return null;
@@ -754,14 +766,12 @@ public class ApplicationMaster {
       } catch (InterruptedException ex) {}
     }
 
-    if (timelineClient != null) {
-      if (timelineServiceV2) {
-        publishApplicationAttemptEventOnTimelineServiceV2(
-            DSEvent.DS_APP_ATTEMPT_END);
-      } else {
-        publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
-            DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
-      }
+    if (timelineServiceV2Enabled) {
+      publishApplicationAttemptEventOnTimelineServiceV2(
+          DSEvent.DS_APP_ATTEMPT_END);
+    } else if (timelineServiceV1Enabled) {
+      publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
+          DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
     }
 
     // Join all launched threads
@@ -810,8 +820,10 @@ public class ApplicationMaster {
     amRMClient.stop();
 
     // Stop Timeline Client
-    if(timelineClient != null) {
+    if(timelineServiceV1Enabled) {
       timelineClient.stop();
+    } else if (timelineServiceV2Enabled) {
+      timelineV2Client.stop();
     }
 
     return success;
@@ -866,21 +878,19 @@ public class ApplicationMaster {
           LOG.info("Container completed successfully." + ", containerId="
               + containerStatus.getContainerId());
         }
-        if(timelineClient != null) {
-          if (timelineServiceV2) {
-            Long containerStartTime =
-                containerStartTimes.get(containerStatus.getContainerId());
-            if (containerStartTime == null) {
-              containerStartTime = SystemClock.getInstance().getTime();
-              containerStartTimes.put(containerStatus.getContainerId(),
-                  containerStartTime);
-            }
-            publishContainerEndEventOnTimelineServiceV2(containerStatus,
+        if (timelineServiceV2Enabled) {
+          Long containerStartTime =
+              containerStartTimes.get(containerStatus.getContainerId());
+          if (containerStartTime == null) {
+            containerStartTime = SystemClock.getInstance().getTime();
+            containerStartTimes.put(containerStatus.getContainerId(),
                 containerStartTime);
-          } else {
-            publishContainerEndEvent(
-                timelineClient, containerStatus, domainId, appSubmitterUgi);
           }
+          publishContainerEndEventOnTimelineServiceV2(containerStatus,
+              containerStartTime);
+        } else if (timelineServiceV1Enabled) {
+          publishContainerEndEvent(timelineClient, containerStatus, domainId,
+              appSubmitterUgi);
         }
       }
       
@@ -1004,18 +1014,15 @@ public class ApplicationMaster {
         applicationMaster.nmClientAsync.getContainerStatusAsync(
             containerId, container.getNodeId());
       }
-      if(applicationMaster.timelineClient != null) {
-        if (applicationMaster.timelineServiceV2) {
-          long startTime = SystemClock.getInstance().getTime();
-          applicationMaster.getContainerStartTimes().put(
-              containerId, startTime);
-          applicationMaster.publishContainerStartEventOnTimelineServiceV2(
-              container, startTime);
-        } else {
-          applicationMaster.publishContainerStartEvent(
-              applicationMaster.timelineClient, container,
-              applicationMaster.domainId, applicationMaster.appSubmitterUgi);
-        }
+      if (applicationMaster.timelineServiceV2Enabled) {
+        long startTime = SystemClock.getInstance().getTime();
+        applicationMaster.getContainerStartTimes().put(containerId, startTime);
+        applicationMaster.publishContainerStartEventOnTimelineServiceV2(
+            container, startTime);
+      } else if (applicationMaster.timelineServiceV1Enabled) {
+        applicationMaster.publishContainerStartEvent(
+            applicationMaster.timelineClient, container,
+            applicationMaster.domainId, applicationMaster.appSubmitterUgi);
       }
     }
 
@@ -1395,7 +1402,7 @@ public class ApplicationMaster {
       appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
         @Override
         public TimelinePutResponse run() throws Exception {
-          timelineClient.putEntities(entity);
+          timelineV2Client.putEntities(entity);
           return null;
         }
       });
@@ -1429,7 +1436,7 @@ public class ApplicationMaster {
       appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
         @Override
         public TimelinePutResponse run() throws Exception {
-          timelineClient.putEntities(entity);
+          timelineV2Client.putEntities(entity);
           return null;
         }
       });
@@ -1465,7 +1472,7 @@ public class ApplicationMaster {
       appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
         @Override
         public TimelinePutResponse run() throws Exception {
-          timelineClient.putEntitiesAsync(entity);
+          timelineV2Client.putEntitiesAsync(entity);
           return null;
         }
       });

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
index 07eff1f..0b9d06a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AMRMClient.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
@@ -39,6 +40,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
 import com.google.common.base.Preconditions;
@@ -51,7 +53,8 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
     AbstractService {
   private static final Log LOG = LogFactory.getLog(AMRMClient.class);
 
-  private TimelineClient timelineClient;
+  private TimelineV2Client timelineV2Client;
+  private boolean timelineServiceV2Enabled;
 
   /**
    * Create a new instance of AMRMClient.
@@ -76,6 +79,12 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
     nmTokenCache = NMTokenCache.getSingleton();
   }
 
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    timelineServiceV2Enabled = YarnConfiguration.timelineServiceV2Enabled(conf);
+  }
+
   /**
    * Object to represent a single container request for resources. Scheduler
    * documentation should be consulted for the specifics of how the parameters
@@ -653,19 +662,30 @@ public abstract class AMRMClient<T extends AMRMClient.ContainerRequest> extends
   }
 
   /**
-   * Register TimelineClient to AMRMClient.
-   * @param client the timeline client to register
+   * Register TimelineV2Client to AMRMClient. Writer's address for the timeline
+   * V2 client will be updated dynamically if registered.
+   *
+   * @param client the timeline v2 client to register
+   * @throws YarnException when this method is invoked even when ATS V2 is not
+   *           configured.
    */
-  public void registerTimelineClient(TimelineClient client) {
-    this.timelineClient = client;
+  public void registerTimelineV2Client(TimelineV2Client client)
+      throws YarnException {
+    if (timelineServiceV2Enabled) {
+      timelineV2Client = client;
+    } else {
+      LOG.error("Trying to register timeline v2 client when not configured.");
+      throw new YarnException(
+          "register timeline v2 client when not configured.");
+    }
   }
 
   /**
-   * Get registered timeline client.
-   * @return the registered timeline client
+   * Get registered timeline v2 client.
+   * @return the registered timeline v2 client
    */
-  public TimelineClient getRegisteredTimeineClient() {
-    return this.timelineClient;
+  public TimelineV2Client getRegisteredTimelineV2Client() {
+    return this.timelineV2Client;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
index 82208e4..5499dc5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/AMRMClientAsync.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.client.api.async;
 
-import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
@@ -29,8 +27,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -44,12 +42,14 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.client.api.async.impl.AMRMClientAsyncImpl;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
 
 /**
  * <code>AMRMClientAsync</code> handles communication with the ResourceManager
@@ -317,17 +317,20 @@ extends AbstractService {
   /**
    * Register TimelineClient to AMRMClient.
    * @param timelineClient
+   * @throws YarnException when this method is invoked even when ATS V2 is not
+   *           configured.
    */
-  public void registerTimelineClient(TimelineClient timelineClient) {
-    client.registerTimelineClient(timelineClient);
+  public void registerTimelineV2Client(TimelineV2Client timelineClient)
+      throws YarnException {
+    client.registerTimelineV2Client(timelineClient);
   }
 
   /**
    * Get registered timeline client.
    * @return the registered timeline client
    */
-  public TimelineClient getRegisteredTimeineClient() {
-    return client.getRegisteredTimeineClient();
+  public TimelineV2Client getRegisteredTimelineV2Client() {
+    return client.getRegisteredTimelineV2Client();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
index 0758efb..8b2557c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/async/impl/AMRMClientAsyncImpl.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.UpdatedContainer;
 import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl;
 import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
@@ -328,7 +328,8 @@ extends AMRMClientAsync<T> {
           AllocateResponse response = (AllocateResponse) object;
 
           String collectorAddress = response.getCollectorAddr();
-          TimelineClient timelineClient = client.getRegisteredTimeineClient();
+          TimelineV2Client timelineClient =
+              client.getRegisteredTimelineV2Client();
           if (timelineClient != null && collectorAddress != null
               && !collectorAddress.isEmpty()) {
             if (collectorAddr == null

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
index 2a2af8b..8865b52 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java
@@ -141,7 +141,7 @@ public class YarnClientImpl extends YarnClient {
   Text timelineService;
   @VisibleForTesting
   String timelineDTRenewer;
-  protected boolean timelineServiceEnabled;
+  private boolean timelineV1ServiceEnabled;
   protected boolean timelineServiceBestEffort;
 
   private static final String ROOT = "root";
@@ -167,9 +167,14 @@ public class YarnClientImpl extends YarnClient {
         YarnConfiguration.DEFAULT_YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS);
     }
 
+    float timelineServiceVersion =
+        conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION);
     if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
-      timelineServiceEnabled = true;
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)
+        && ((Float.compare(timelineServiceVersion, 1.0f) == 0)
+            || (Float.compare(timelineServiceVersion, 1.5f) == 0))) {
+      timelineV1ServiceEnabled = true;
       timelineDTRenewer = getTimelineDelegationTokenRenewer(conf);
       timelineService = TimelineUtils.buildTimelineTokenService(conf);
     }
@@ -178,7 +183,7 @@ public class YarnClientImpl extends YarnClient {
     // TimelineServer which means we are able to get history information
     // for applications/applicationAttempts/containers by using ahsClient
     // when the TimelineServer is running.
-    if (timelineServiceEnabled || conf.getBoolean(
+    if (timelineV1ServiceEnabled || conf.getBoolean(
         YarnConfiguration.APPLICATION_HISTORY_ENABLED,
         YarnConfiguration.DEFAULT_APPLICATION_HISTORY_ENABLED)) {
       historyServiceEnabled = true;
@@ -257,7 +262,7 @@ public class YarnClientImpl extends YarnClient {
 
     // Automatically add the timeline DT into the CLC
     // Only when the security and the timeline service are both enabled
-    if (isSecurityEnabled() && timelineServiceEnabled) {
+    if (isSecurityEnabled() && timelineV1ServiceEnabled) {
       addTimelineDelegationToken(appContext.getAMContainerSpec());
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
index cc76718..4835239 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineClient.java
@@ -21,14 +21,12 @@ package org.apache.hadoop.yarn.client.api;
 import java.io.Flushable;
 import java.io.IOException;
 
-import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
@@ -39,24 +37,22 @@ import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
 
 /**
  * A client library that can be used to post some information in terms of a
- * number of conceptual entities.
+ * number of conceptual entities. This client library needs to be used along
+ * with Timeline V.1.x server versions.
+ * Refer {@link TimelineV2Client} for ATS V2 interface.
  */
 @Public
 @Evolving
-public abstract class TimelineClient extends AbstractService implements
+public abstract class TimelineClient extends CompositeService implements
     Flushable {
 
   /**
-   * Create a timeline client. The current UGI when the user initialize the
-   * client will be used to do the put and the delegation token operations. The
-   * current user may use {@link UserGroupInformation#doAs} another user to
-   * construct and initialize a timeline client if the following operations are
-   * supposed to be conducted by that user.
-   */
-  private ApplicationId contextAppId;
-
-  /**
    * Creates an instance of the timeline v.1.x client.
+   * The current UGI when the user initialize the client will be used to do the
+   * put and the delegation token operations. The current user may use
+   * {@link UserGroupInformation#doAs} another user to construct and initialize
+   * a timeline client if the following operations are supposed to be conducted
+   * by that user.
    *
    * @return the created timeline client instance
    */
@@ -66,23 +62,8 @@ public abstract class TimelineClient extends AbstractService implements
     return client;
   }
 
-  /**
-   * Creates an instance of the timeline v.2 client.
-   *
-   * @param appId the application id with which the timeline client is
-   * associated
-   * @return the created timeline client instance
-   */
-  @Public
-  public static TimelineClient createTimelineClient(ApplicationId appId) {
-    TimelineClient client = new TimelineClientImpl(appId);
-    return client;
-  }
-
-  @Private
-  protected TimelineClient(String name, ApplicationId appId) {
+  protected TimelineClient(String name) {
     super(name);
-    setContextAppId(appId);
   }
 
   /**
@@ -207,57 +188,4 @@ public abstract class TimelineClient extends AbstractService implements
   public abstract void cancelDelegationToken(
       Token<TimelineDelegationTokenIdentifier> timelineDT)
           throws IOException, YarnException;
-
-  /**
-   * <p>
-   * Send the information of a number of conceptual entities to the timeline
-   * service v.2 collector. It is a blocking API. The method will not return
-   * until all the put entities have been persisted. If this method is invoked
-   * for a non-v.2 timeline client instance, a YarnException is thrown.
-   * </p>
-   *
-   * @param entities the collection of {@link
-   * org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity}
-   * @throws IOException
-   * @throws YarnException
-   */
-  @Public
-  public abstract void putEntities(
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
-          entities) throws IOException, YarnException;
-
-  /**
-   * <p>
-   * Send the information of a number of conceptual entities to the timeline
-   * service v.2 collector. It is an asynchronous API. The method will return
-   * once all the entities are received. If this method is invoked for a
-   * non-v.2 timeline client instance, a YarnException is thrown.
-   * </p>
-   *
-   * @param entities the collection of {@link
-   * org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity}
-   * @throws IOException
-   * @throws YarnException
-   */
-  @Public
-  public abstract void putEntitiesAsync(
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
-          entities) throws IOException, YarnException;
-
-  /**
-   * <p>
-   * Update the timeline service address where the request will be sent to.
-   * </p>
-   * @param address
-   *          the timeline service address
-   */
-  public abstract void setTimelineServiceAddress(String address);
-
-  protected ApplicationId getContextAppId() {
-    return contextAppId;
-  }
-
-  protected void setContextAppId(ApplicationId appId) {
-    this.contextAppId = appId;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java
new file mode 100644
index 0000000..32cf1e9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/TimelineV2Client.java
@@ -0,0 +1,92 @@
+/**
+ * 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.client.api;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.client.api.impl.TimelineV2ClientImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * A client library that can be used to post some information in terms of a
+ * number of conceptual entities. This client library needs to be used along
+ * with time line v.2 server version.
+ * Refer {@link TimelineClient} for ATS V1 interface.
+ */
+public abstract class TimelineV2Client extends CompositeService {
+  /**
+   * Creates an instance of the timeline v.2 client.
+   *
+   * @param appId the application id with which the timeline client is
+   *          associated
+   * @return the created timeline client instance
+   */
+  @Public
+  public static TimelineV2Client createTimelineClient(ApplicationId appId) {
+    TimelineV2Client client = new TimelineV2ClientImpl(appId);
+    return client;
+  }
+
+  protected TimelineV2Client(String name) {
+    super(name);
+  }
+
+  /**
+   * <p>
+   * Send the information of a number of conceptual entities to the timeline
+   * service v.2 collector. It is a blocking API. The method will not return
+   * until all the put entities have been persisted.
+   * </p>
+   *
+   * @param entities the collection of {@link TimelineEntity}
+   * @throws IOException  if there are I/O errors
+   * @throws YarnException if entities are incomplete/invalid
+   */
+  @Public
+  public abstract void putEntities(TimelineEntity... entities)
+      throws IOException, YarnException;
+
+  /**
+   * <p>
+   * Send the information of a number of conceptual entities to the timeline
+   * service v.2 collector. It is an asynchronous API. The method will return
+   * once all the entities are received.
+   * </p>
+   *
+   * @param entities the collection of {@link TimelineEntity}
+   * @throws IOException  if there are I/O errors
+   * @throws YarnException if entities are incomplete/invalid
+   */
+  @Public
+  public abstract void putEntitiesAsync(TimelineEntity... entities)
+      throws IOException, YarnException;
+
+  /**
+   * <p>
+   * Update the timeline service address where the request will be sent to.
+   * </p>
+   *
+   * @param address the timeline service address
+   */
+  public abstract void setTimelineServiceAddress(String address);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e7eed958/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
index cb9e43c..48d720a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineClientImpl.java
@@ -20,31 +20,9 @@ package org.apache.hadoop.yarn.client.api.impl;
 
 import java.io.File;
 import java.io.IOException;
-import java.lang.reflect.UndeclaredThrowableException;
-import java.net.ConnectException;
-import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
-import java.net.SocketException;
-import java.net.SocketTimeoutException;
 import java.net.URI;
-import java.net.URL;
-import java.net.URLConnection;
-import java.security.GeneralSecurityException;
 import java.security.PrivilegedExceptionAction;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-import javax.net.ssl.HostnameVerifier;
-import javax.net.ssl.HttpsURLConnection;
-import javax.net.ssl.SSLSocketFactory;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.MultivaluedMap;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
@@ -57,16 +35,9 @@ import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
-import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
-import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
-import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
-import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator;
-import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticator;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomains;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
@@ -81,30 +52,15 @@ import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
 import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientHandlerException;
-import com.sun.jersey.api.client.ClientRequest;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-import com.sun.jersey.api.client.filter.ClientFilter;
-import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
-import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
-import com.sun.jersey.core.util.MultivaluedMapImpl;
 
 @Private
 @Evolving
 public class TimelineClientImpl extends TimelineClient {
 
   private static final Log LOG = LogFactory.getLog(TimelineClientImpl.class);
-  private static final String RESOURCE_URI_STR = "/ws/v1/timeline/";
   private static final ObjectMapper MAPPER = new ObjectMapper();
   private static final String RESOURCE_URI_STR_V1 = "/ws/v1/timeline/";
-  private static final String RESOURCE_URI_STR_V2 = "/ws/v2/timeline/";
-  private static final Joiner JOINER = Joiner.on("");
-  public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
 
   private static Options opts;
   private static final String ENTITY_DATA_TYPE = "entity";
@@ -119,180 +75,38 @@ public class TimelineClientImpl extends TimelineClient {
     opts.addOption("help", false, "Print usage");
   }
 
-  private Client client;
-  private ConnectionConfigurator connConfigurator;
-  private DelegationTokenAuthenticator authenticator;
-  private DelegationTokenAuthenticatedURL.Token token;
-  private UserGroupInformation authUgi;
-  private String doAsUser;
-  private Configuration configuration;
-  private float timelineServiceVersion;
-  private TimelineWriter timelineWriter;
-  private SSLFactory sslFactory;
-
-  private volatile String timelineServiceAddress;
-
-  // Retry parameters for identifying new timeline service
-  // TODO consider to merge with connection retry
-  private int maxServiceRetries;
-  private long serviceRetryInterval;
-  private boolean timelineServiceV2 = false;
-
-  @Private
   @VisibleForTesting
-  TimelineClientConnectionRetry connectionRetry;
-
-  private TimelineEntityDispatcher entityDispatcher;
-
-  // Abstract class for an operation that should be retried by timeline client
-  @Private
+  protected DelegationTokenAuthenticatedURL.Token token;
   @VisibleForTesting
-  public static abstract class TimelineClientRetryOp {
-    // The operation that should be retried
-    public abstract Object run() throws IOException;
-    // The method to indicate if we should retry given the incoming exception
-    public abstract boolean shouldRetryOn(Exception e);
-  }
-
-  // Class to handle retry
-  // Outside this class, only visible to tests
-  @Private
+  protected UserGroupInformation authUgi;
   @VisibleForTesting
-  static class TimelineClientConnectionRetry {
-
-    // maxRetries < 0 means keep trying
-    @Private
-    @VisibleForTesting
-    public int maxRetries;
-
-    @Private
-    @VisibleForTesting
-    public long retryInterval;
-
-    // Indicates if retries happened last time. Only tests should read it.
-    // In unit tests, retryOn() calls should _not_ be concurrent.
-    private boolean retried = false;
-
-    @Private
-    @VisibleForTesting
-    boolean getRetired() {
-      return retried;
-    }
+  protected String doAsUser;
 
-    // Constructor with default retry settings
-    public TimelineClientConnectionRetry(Configuration conf) {
-      Preconditions.checkArgument(conf.getInt(
-          YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
-          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES) >= -1,
-          "%s property value should be greater than or equal to -1",
-          YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
-      Preconditions
-          .checkArgument(
-              conf.getLong(
-                  YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
-                  YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS) > 0,
-              "%s property value should be greater than zero",
-              YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
-      maxRetries = conf.getInt(
-        YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
-      retryInterval = conf.getLong(
-        YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
-    }
-
-    public Object retryOn(TimelineClientRetryOp op)
-        throws RuntimeException, IOException {
-      int leftRetries = maxRetries;
-      retried = false;
-
-      // keep trying
-      while (true) {
-        try {
-          // try perform the op, if fail, keep retrying
-          return op.run();
-        } catch (IOException | RuntimeException e) {
-          // break if there's no retries left
-          if (leftRetries == 0) {
-            break;
-          }
-          if (op.shouldRetryOn(e)) {
-            logException(e, leftRetries);
-          } else {
-            throw e;
-          }
-        }
-        if (leftRetries > 0) {
-          leftRetries--;
-        }
-        retried = true;
-        try {
-          // sleep for the given time interval
-          Thread.sleep(retryInterval);
-        } catch (InterruptedException ie) {
-          LOG.warn("Client retry sleep interrupted! ");
-        }
-      }
-      throw new RuntimeException("Failed to connect to timeline server. "
-          + "Connection retries limit exceeded. "
-          + "The posted timeline event may be missing");
-    };
-
-    private void logException(Exception e, int leftRetries) {
-      if (leftRetries > 0) {
-        LOG.info("Exception caught by TimelineClientConnectionRetry,"
-              + " will try " + leftRetries + " more time(s).\nMessage: "
-              + e.getMessage());
-      } else {
-        // note that maxRetries may be -1 at the very beginning
-        LOG.info("ConnectionException caught by TimelineClientConnectionRetry,"
-            + " will keep retrying.\nMessage: "
-            + e.getMessage());
-      }
-    }
-  }
+  private float timelineServiceVersion;
+  private TimelineWriter timelineWriter;
 
-  private class TimelineJerseyRetryFilter extends ClientFilter {
-    @Override
-    public ClientResponse handle(final ClientRequest cr)
-        throws ClientHandlerException {
-      // Set up the retry operation
-      TimelineClientRetryOp jerseyRetryOp = new TimelineClientRetryOp() {
-        @Override
-        public Object run() {
-          // Try pass the request, if fail, keep retrying
-          return getNext().handle(cr);
-        }
+  private String timelineServiceAddress;
 
-        @Override
-        public boolean shouldRetryOn(Exception e) {
-          // Only retry on connection exceptions
-          return (e instanceof ClientHandlerException)
-              && (e.getCause() instanceof ConnectException ||
-                  e.getCause() instanceof SocketTimeoutException ||
-                  e.getCause() instanceof SocketException);
-        }
-      };
-      try {
-        return (ClientResponse) connectionRetry.retryOn(jerseyRetryOp);
-      } catch (IOException e) {
-        throw new ClientHandlerException("Jersey retry failed!\nMessage: "
-              + e.getMessage());
-      }
-    }
-  }
+  @Private
+  @VisibleForTesting
+  TimelineConnector connector;
 
   public TimelineClientImpl() {
-    super(TimelineClientImpl.class.getName(), null);
-  }
-
-  public TimelineClientImpl(ApplicationId applicationId) {
-    super(TimelineClientImpl.class.getName(), applicationId);
-    this.timelineServiceV2 = true;
+    super(TimelineClientImpl.class.getName());
   }
 
   protected void serviceInit(Configuration conf) throws Exception {
-    this.configuration = conf;
+    timelineServiceVersion =
+        conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION);
+    LOG.info("Timeline service address: " + getTimelineServiceAddress());
+    if (!YarnConfiguration.timelineServiceEnabled(conf)
+        || !((Float.compare(this.timelineServiceVersion, 1.0f) == 0)
+            || (Float.compare(this.timelineServiceVersion, 1.5f) == 0))) {
+      throw new IOException("Timeline V1 client is not properly configured. "
+          + "Either timeline service is not enabled or version is not set to"
+          + " 1.x");
+    }
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     UserGroupInformation realUgi = ugi.getRealUser();
     if (realUgi != null) {
@@ -302,62 +116,34 @@ public class TimelineClientImpl extends TimelineClient {
       authUgi = ugi;
       doAsUser = null;
     }
-    ClientConfig cc = new DefaultClientConfig();
-    cc.getClasses().add(YarnJacksonJaxbJsonProvider.class);
-    connConfigurator = initConnConfigurator(conf);
-    if (UserGroupInformation.isSecurityEnabled()) {
-      authenticator = new KerberosDelegationTokenAuthenticator();
-    } else {
-      authenticator = new PseudoDelegationTokenAuthenticator();
-    }
-    authenticator.setConnectionConfigurator(connConfigurator);
     token = new DelegationTokenAuthenticatedURL.Token();
+    connector = createTimelineConnector();
 
-    connectionRetry = new TimelineClientConnectionRetry(conf);
-    client = new Client(new URLConnectionClientHandler(
-        new TimelineURLConnectionFactory()), cc);
-    TimelineJerseyRetryFilter retryFilter = new TimelineJerseyRetryFilter();
-    // TODO need to cleanup filter retry later.
-    if (!timelineServiceV2) {
-      client.addFilter(retryFilter);
-    }
-
-    // old version timeline service need to get address from configuration
-    // while new version need to auto discovery (with retry).
-    if (timelineServiceV2) {
-      maxServiceRetries = conf.getInt(
-          YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES,
-          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES);
-      serviceRetryInterval = conf.getLong(
-          YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
-          YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
-      entityDispatcher = new TimelineEntityDispatcher(conf);
+    if (YarnConfiguration.useHttps(conf)) {
+      timelineServiceAddress =
+          conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
+              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS);
     } else {
-      if (YarnConfiguration.useHttps(conf)) {
-        setTimelineServiceAddress(conf.get(
-            YarnConfiguration.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS));
-      } else {
-        setTimelineServiceAddress(conf.get(
-            YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS));
-      }
-      timelineServiceVersion =
-          conf.getFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION,
-              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_VERSION);
-      LOG.info("Timeline service address: " + getTimelineServiceAddress());
+      timelineServiceAddress =
+          conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS);
     }
     super.serviceInit(conf);
   }
 
+  @VisibleForTesting
+  protected TimelineConnector createTimelineConnector() {
+    TimelineConnector newConnector =
+        new TimelineConnector(true, authUgi, doAsUser, token);
+    addIfService(newConnector);
+    return newConnector;
+  }
+
   @Override
   protected void serviceStart() throws Exception {
-    if (timelineServiceV2) {
-      entityDispatcher.start();
-    } else {
-      timelineWriter = createTimelineWriter(configuration, authUgi, client,
-          constructResURI(getConfig(), timelineServiceAddress, false));
-    }
+    timelineWriter = createTimelineWriter(getConfig(), authUgi,
+        connector.getClient(), TimelineConnector.constructResURI(getConfig(),
+            timelineServiceAddress, RESOURCE_URI_STR_V1));
   }
 
   protected TimelineWriter createTimelineWriter(Configuration conf,
@@ -376,12 +162,6 @@ public class TimelineClientImpl extends TimelineClient {
     if (this.timelineWriter != null) {
       this.timelineWriter.close();
     }
-    if (this.sslFactory != null) {
-      this.sslFactory.destroy();
-    }
-    if (timelineServiceV2) {
-      entityDispatcher.stop();
-    }
     super.serviceStop();
   }
 
@@ -393,131 +173,17 @@ public class TimelineClientImpl extends TimelineClient {
   }
 
   @Override
-  public TimelinePutResponse putEntities(
-      TimelineEntity... entities) throws IOException, YarnException {
+  public TimelinePutResponse putEntities(TimelineEntity... entities)
+      throws IOException, YarnException {
     return timelineWriter.putEntities(entities);
   }
 
   @Override
-  public void putEntities(
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
-          entities) throws IOException, YarnException {
-    if (!timelineServiceV2) {
-      throw new YarnException("v.2 method is invoked on a v.1.x client");
-    }
-    entityDispatcher.dispatchEntities(true, entities);
-  }
-
-  @Override
-  public void putEntitiesAsync(
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity...
-          entities) throws IOException, YarnException {
-    if (!timelineServiceV2) {
-      throw new YarnException("v.2 method is invoked on a v.1.x client");
-    }
-    entityDispatcher.dispatchEntities(false, entities);
-  }
-
-  @Override
   public void putDomain(TimelineDomain domain) throws IOException,
       YarnException {
     timelineWriter.putDomain(domain);
   }
 
-  // Used for new timeline service only
-  @Private
-  protected void putObjects(String path, MultivaluedMap<String, String> params,
-      Object obj) throws IOException, YarnException {
-
-    int retries = verifyRestEndPointAvailable();
-
-    // timelineServiceAddress could be stale, add retry logic here.
-    boolean needRetry = true;
-    while (needRetry) {
-      try {
-        URI uri = constructResURI(getConfig(), timelineServiceAddress, true);
-        putObjects(uri, path, params, obj);
-        needRetry = false;
-      } catch (IOException e) {
-        // handle exception for timelineServiceAddress being updated.
-        checkRetryWithSleep(retries, e);
-        retries--;
-      }
-    }
-  }
-
-  private int verifyRestEndPointAvailable() throws YarnException {
-    // timelineServiceAddress could haven't be initialized yet
-    // or stale (only for new timeline service)
-    int retries = pollTimelineServiceAddress(this.maxServiceRetries);
-    if (timelineServiceAddress == null) {
-      String errMessage = "TimelineClient has reached to max retry times : "
-          + this.maxServiceRetries
-          + ", but failed to fetch timeline service address. Please verify"
-          + " Timeline Auxillary Service is configured in all the NMs";
-      LOG.error(errMessage);
-      throw new YarnException(errMessage);
-    }
-    return retries;
-  }
-
-  /**
-   * Check if reaching to maximum of retries.
-   * @param retries
-   * @param e
-   */
-  private void checkRetryWithSleep(int retries, IOException e)
-      throws YarnException, IOException {
-    if (retries > 0) {
-      try {
-        Thread.sleep(this.serviceRetryInterval);
-      } catch (InterruptedException ex) {
-        Thread.currentThread().interrupt();
-        throw new YarnException("Interrupted while retrying to connect to ATS");
-      }
-    } else {
-      StringBuilder msg =
-          new StringBuilder("TimelineClient has reached to max retry times : ");
-      msg.append(this.maxServiceRetries);
-      msg.append(" for service address: ");
-      msg.append(timelineServiceAddress);
-      LOG.error(msg.toString());
-      throw new IOException(msg.toString(), e);
-    }
-  }
-
-  protected void putObjects(
-      URI base, String path, MultivaluedMap<String, String> params, Object obj)
-          throws IOException, YarnException {
-    ClientResponse resp;
-    try {
-      resp = client.resource(base).path(path).queryParams(params)
-          .accept(MediaType.APPLICATION_JSON)
-          .type(MediaType.APPLICATION_JSON)
-          .put(ClientResponse.class, obj);
-    } catch (RuntimeException re) {
-      // runtime exception is expected if the client cannot connect the server
-      String msg =
-          "Failed to get the response from the timeline server.";
-      LOG.error(msg, re);
-      throw new IOException(re);
-    }
-    if (resp == null ||
-        resp.getClientResponseStatus() != ClientResponse.Status.OK) {
-      String msg = "Response from the timeline server is " +
-          ((resp == null) ? "null":
-          "not successful," + " HTTP error code: " + resp.getStatus()
-          + ", Server response:\n" + resp.getEntity(String.class));
-      LOG.error(msg);
-      throw new YarnException(msg);
-    }
-  }
-
-  @Override
-  public void setTimelineServiceAddress(String address) {
-    this.timelineServiceAddress = address;
-  }
-
   private String getTimelineServiceAddress() {
     return this.timelineServiceAddress;
   }
@@ -534,17 +200,17 @@ public class TimelineClientImpl extends TimelineClient {
           public Token<TimelineDelegationTokenIdentifier> run()
               throws Exception {
             DelegationTokenAuthenticatedURL authUrl =
-                new DelegationTokenAuthenticatedURL(authenticator,
-                    connConfigurator);
+                connector.getDelegationTokenAuthenticatedURL();
             // TODO we should add retry logic here if timelineServiceAddress is
             // not available immediately.
             return (Token) authUrl.getDelegationToken(
-                constructResURI(getConfig(),
-                    getTimelineServiceAddress(), false).toURL(),
+                TimelineConnector.constructResURI(getConfig(),
+                    getTimelineServiceAddress(), RESOURCE_URI_STR_V1).toURL(),
                 token, renewer, doAsUser);
           }
         };
-    return (Token<TimelineDelegationTokenIdentifier>) operateDelegationToken(getDTAction);
+    return (Token<TimelineDelegationTokenIdentifier>) connector
+        .operateDelegationToken(getDTAction);
   }
 
   @SuppressWarnings("unchecked")
@@ -570,26 +236,26 @@ public class TimelineClientImpl extends TimelineClient {
               token.setDelegationToken((Token) timelineDT);
             }
             DelegationTokenAuthenticatedURL authUrl =
-                new DelegationTokenAuthenticatedURL(authenticator,
-                    connConfigurator);
+                connector.getDelegationTokenAuthenticatedURL();
             // If the token service address is not available, fall back to use
             // the configured service address.
-            final URI serviceURI = isTokenServiceAddrEmpty ?
-                constructResURI(getConfig(), getTimelineServiceAddress(), false)
+            final URI serviceURI = isTokenServiceAddrEmpty
+                ? TimelineConnector.constructResURI(getConfig(),
+                    getTimelineServiceAddress(), RESOURCE_URI_STR_V1)
                 : new URI(scheme, null, address.getHostName(),
-                address.getPort(), RESOURCE_URI_STR_V1, null, null);
+                    address.getPort(), RESOURCE_URI_STR_V1, null, null);
             return authUrl
                 .renewDelegationToken(serviceURI.toURL(), token, doAsUser);
           }
         };
-    return (Long) operateDelegationToken(renewDTAction);
+    return (Long) connector.operateDelegationToken(renewDTAction);
   }
 
   @SuppressWarnings("unchecked")
   @Override
   public void cancelDelegationToken(
       final Token<TimelineDelegationTokenIdentifier> timelineDT)
-          throws IOException, YarnException {
+      throws IOException, YarnException {
     final boolean isTokenServiceAddrEmpty =
         timelineDT.getService().toString().isEmpty();
     final String scheme = isTokenServiceAddrEmpty ? null
@@ -609,134 +275,29 @@ public class TimelineClientImpl extends TimelineClient {
               token.setDelegationToken((Token) timelineDT);
             }
             DelegationTokenAuthenticatedURL authUrl =
-                new DelegationTokenAuthenticatedURL(authenticator,
-                    connConfigurator);
+                connector.getDelegationTokenAuthenticatedURL();
             // If the token service address is not available, fall back to use
             // the configured service address.
-            final URI serviceURI = isTokenServiceAddrEmpty ?
-                constructResURI(getConfig(), getTimelineServiceAddress(), false)
+            final URI serviceURI = isTokenServiceAddrEmpty
+                ? TimelineConnector.constructResURI(getConfig(),
+                    getTimelineServiceAddress(), RESOURCE_URI_STR_V1)
                 : new URI(scheme, null, address.getHostName(),
-                address.getPort(), RESOURCE_URI_STR_V1, null, null);
+                    address.getPort(), RESOURCE_URI_STR_V1, null, null);
             authUrl.cancelDelegationToken(serviceURI.toURL(), token, doAsUser);
             return null;
           }
         };
-    operateDelegationToken(cancelDTAction);
+    connector.operateDelegationToken(cancelDTAction);
   }
 
   @Override
   public String toString() {
     return super.toString() + " with timeline server "
-        + constructResURI(getConfig(), getTimelineServiceAddress(), false)
+        + TimelineConnector.constructResURI(getConfig(),
+            getTimelineServiceAddress(), RESOURCE_URI_STR_V1)
         + " and writer " + timelineWriter;
   }
 
-  private Object operateDelegationToken(
-      final PrivilegedExceptionAction<?> action)
-      throws IOException, YarnException {
-    // Set up the retry operation
-    TimelineClientRetryOp tokenRetryOp =
-        createTimelineClientRetryOpForOperateDelegationToken(action);
-
-    return connectionRetry.retryOn(tokenRetryOp);
-  }
-
-  /**
-   * Poll TimelineServiceAddress for maximum of retries times if it is null.
-   *
-   * @param retries
-   * @return the left retry times
-   * @throws IOException
-   */
-  private int pollTimelineServiceAddress(int retries) throws YarnException {
-    while (timelineServiceAddress == null && retries > 0) {
-      try {
-        Thread.sleep(this.serviceRetryInterval);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new YarnException("Interrupted while trying to connect ATS");
-      }
-      retries--;
-    }
-    return retries;
-  }
-
-  private class TimelineURLConnectionFactory
-      implements HttpURLConnectionFactory {
-
-    @Override
-    public HttpURLConnection getHttpURLConnection(final URL url) throws IOException {
-      authUgi.checkTGTAndReloginFromKeytab();
-      try {
-        return new DelegationTokenAuthenticatedURL(
-            authenticator, connConfigurator).openConnection(url, token,
-              doAsUser);
-      } catch (UndeclaredThrowableException e) {
-        throw new IOException(e.getCause());
-      } catch (AuthenticationException ae) {
-        throw new IOException(ae);
-      }
-    }
-
-  }
-
-  private ConnectionConfigurator initConnConfigurator(Configuration conf) {
-    try {
-      return initSslConnConfigurator(DEFAULT_SOCKET_TIMEOUT, conf);
-    } catch (Exception e) {
-      LOG.debug("Cannot load customized ssl related configuration. " +
-          "Fallback to system-generic settings.", e);
-      return DEFAULT_TIMEOUT_CONN_CONFIGURATOR;
-    }
-  }
-
-  private static final ConnectionConfigurator DEFAULT_TIMEOUT_CONN_CONFIGURATOR =
-      new ConnectionConfigurator() {
-    @Override
-    public HttpURLConnection configure(HttpURLConnection conn)
-        throws IOException {
-      setTimeouts(conn, DEFAULT_SOCKET_TIMEOUT);
-      return conn;
-    }
-  };
-
-  private ConnectionConfigurator initSslConnConfigurator(final int timeout,
-      Configuration conf) throws IOException, GeneralSecurityException {
-    final SSLSocketFactory sf;
-    final HostnameVerifier hv;
-
-    sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
-    sslFactory.init();
-    sf = sslFactory.createSSLSocketFactory();
-    hv = sslFactory.getHostnameVerifier();
-
-    return new ConnectionConfigurator() {
-      @Override
-      public HttpURLConnection configure(HttpURLConnection conn)
-          throws IOException {
-        if (conn instanceof HttpsURLConnection) {
-          HttpsURLConnection c = (HttpsURLConnection) conn;
-          c.setSSLSocketFactory(sf);
-          c.setHostnameVerifier(hv);
-        }
-        setTimeouts(conn, timeout);
-        return conn;
-      }
-    };
-  }
-
-  private static void setTimeouts(URLConnection connection, int socketTimeout) {
-    connection.setConnectTimeout(socketTimeout);
-    connection.setReadTimeout(socketTimeout);
-  }
-
-  private static URI constructResURI(
-      Configuration conf, String address, boolean v2) {
-    return URI.create(
-        JOINER.join(YarnConfiguration.useHttps(conf) ? "https://" : "http://",
-            address, v2 ? RESOURCE_URI_STR_V2 : RESOURCE_URI_STR_V1));
-  }
-
   public static void main(String[] argv) throws Exception {
     CommandLine cliParser = new GnuParser().parse(opts, argv);
     if (cliParser.hasOption("put")) {
@@ -872,266 +433,4 @@ public class TimelineClientImpl extends TimelineClient {
   public void setTimelineWriter(TimelineWriter writer) {
     this.timelineWriter = writer;
   }
-
-  @Private
-  @VisibleForTesting
-  public TimelineClientRetryOp
-      createTimelineClientRetryOpForOperateDelegationToken(
-          final PrivilegedExceptionAction<?> action) throws IOException {
-    return new TimelineClientRetryOpForOperateDelegationToken(
-        this.authUgi, action);
-  }
-
-  @Private
-  @VisibleForTesting
-  public class TimelineClientRetryOpForOperateDelegationToken
-      extends TimelineClientRetryOp {
-
-    private final UserGroupInformation authUgi;
-    private final PrivilegedExceptionAction<?> action;
-
-    public TimelineClientRetryOpForOperateDelegationToken(
-        UserGroupInformation authUgi, PrivilegedExceptionAction<?> action) {
-      this.authUgi = authUgi;
-      this.action = action;
-    }
-
-    @Override
-    public Object run() throws IOException {
-      // Try pass the request, if fail, keep retrying
-      authUgi.checkTGTAndReloginFromKeytab();
-      try {
-        return authUgi.doAs(action);
-      } catch (UndeclaredThrowableException e) {
-        throw new IOException(e.getCause());
-      } catch (InterruptedException e) {
-        throw new IOException(e);
-      }
-    }
-
-    @Override
-    public boolean shouldRetryOn(Exception e) {
-      // retry on connection exceptions
-      // and SocketTimeoutException
-      return (e instanceof ConnectException
-          || e instanceof SocketTimeoutException);
-    }
-  }
-
-  private final class EntitiesHolder extends FutureTask<Void> {
-    private final
-        org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
-            entities;
-    private final boolean isSync;
-
-    EntitiesHolder(
-        final
-            org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
-                entities,
-        final boolean isSync) {
-      super(new Callable<Void>() {
-        // publishEntities()
-        public Void call() throws Exception {
-          MultivaluedMap<String, String> params = new MultivaluedMapImpl();
-          params.add("appid", getContextAppId().toString());
-          params.add("async", Boolean.toString(!isSync));
-          putObjects("entities", params, entities);
-          return null;
-        }
-      });
-      this.entities = entities;
-      this.isSync = isSync;
-    }
-
-    public boolean isSync() {
-      return isSync;
-    }
-
-    public org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
-        getEntities() {
-      return entities;
-    }
-  }
-
-  /**
-   * This class is responsible for collecting the timeline entities and
-   * publishing them in async.
-   */
-  private class TimelineEntityDispatcher {
-    /**
-     * Time period for which the timelineclient will wait for draining after
-     * stop.
-     */
-    private static final long DRAIN_TIME_PERIOD = 2000L;
-
-    private int numberOfAsyncsToMerge;
-    private final BlockingQueue<EntitiesHolder> timelineEntityQueue;
-    private ExecutorService executor;
-
-    TimelineEntityDispatcher(Configuration conf) {
-      timelineEntityQueue = new LinkedBlockingQueue<EntitiesHolder>();
-      numberOfAsyncsToMerge =
-          conf.getInt(YarnConfiguration.NUMBER_OF_ASYNC_ENTITIES_TO_MERGE,
-              YarnConfiguration.DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE);
-    }
-
-    Runnable createRunnable() {
-      return new Runnable() {
-        @Override
-        public void run() {
-          try {
-            EntitiesHolder entitiesHolder;
-            while (!Thread.currentThread().isInterrupted()) {
-              // Merge all the async calls and make one push, but if its sync
-              // call push immediately
-              try {
-                entitiesHolder = timelineEntityQueue.take();
-              } catch (InterruptedException ie) {
-                LOG.info("Timeline dispatcher thread was interrupted ");
-                Thread.currentThread().interrupt();
-                return;
-              }
-              if (entitiesHolder != null) {
-                publishWithoutBlockingOnQueue(entitiesHolder);
-              }
-            }
-          } finally {
-            if (!timelineEntityQueue.isEmpty()) {
-              LOG.info("Yet to publish " + timelineEntityQueue.size()
-                  + " timelineEntities, draining them now. ");
-            }
-            // Try to drain the remaining entities to be published @ the max for
-            // 2 seconds
-            long timeTillweDrain =
-                System.currentTimeMillis() + DRAIN_TIME_PERIOD;
-            while (!timelineEntityQueue.isEmpty()) {
-              publishWithoutBlockingOnQueue(timelineEntityQueue.poll());
-              if (System.currentTimeMillis() > timeTillweDrain) {
-                // time elapsed stop publishing further....
-                if (!timelineEntityQueue.isEmpty()) {
-                  LOG.warn("Time to drain elapsed! Remaining "
-                      + timelineEntityQueue.size() + "timelineEntities will not"
-                      + " be published");
-                  // if some entities were not drained then we need interrupt
-                  // the threads which had put sync EntityHolders to the queue.
-                  EntitiesHolder nextEntityInTheQueue = null;
-                  while ((nextEntityInTheQueue =
-                      timelineEntityQueue.poll()) != null) {
-                    nextEntityInTheQueue.cancel(true);
-                  }
-                }
-                break;
-              }
-            }
-          }
-        }
-
-        /**
-         * Publishes the given EntitiesHolder and return immediately if sync
-         * call, else tries to fetch the EntitiesHolder from the queue in non
-         * blocking fashion and collate the Entities if possible before
-         * publishing through REST.
-         *
-         * @param entitiesHolder
-         */
-        private void publishWithoutBlockingOnQueue(
-            EntitiesHolder entitiesHolder) {
-          if (entitiesHolder.isSync()) {
-            entitiesHolder.run();
-            return;
-          }
-          int count = 1;
-          while (true) {
-            // loop till we find a sync put Entities or there is nothing
-            // to take
-            EntitiesHolder nextEntityInTheQueue = timelineEntityQueue.poll();
-            if (nextEntityInTheQueue == null) {
-              // Nothing in the queue just publish and get back to the
-              // blocked wait state
-              entitiesHolder.run();
-              break;
-            } else if (nextEntityInTheQueue.isSync()) {
-              // flush all the prev async entities first
-              entitiesHolder.run();
-              // and then flush the sync entity
-              nextEntityInTheQueue.run();
-              break;
-            } else {
-              // append all async entities together and then flush
-              entitiesHolder.getEntities().addEntities(
-                  nextEntityInTheQueue.getEntities().getEntities());
-              count++;
-              if (count == numberOfAsyncsToMerge) {
-                // Flush the entities if the number of the async
-                // putEntites merged reaches the desired limit. To avoid
-                // collecting multiple entities and delaying for a long
-                // time.
-                entitiesHolder.run();
-                break;
-              }
-            }
-          }
-        }
-      };
-    }
-
-    public void dispatchEntities(boolean sync,
-        org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity[]
-            entitiesTobePublished) throws YarnException {
-      if (executor.isShutdown()) {
-        throw new YarnException("Timeline client is in the process of stopping,"
-            + " not accepting any more TimelineEntities");
-      }
-
-      // wrap all TimelineEntity into TimelineEntities object
-      org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
-          entities =
-              new org.apache.hadoop.yarn.api.records.timelineservice.
-                  TimelineEntities();
-      for (org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
-               entity : entitiesTobePublished) {
-        entities.addEntity(entity);
-      }
-
-      // created a holder and place it in queue
-      EntitiesHolder entitiesHolder = new EntitiesHolder(entities, sync);
-      try {
-        timelineEntityQueue.put(entitiesHolder);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new YarnException(
-            "Failed while adding entity to the queue for publishing", e);
-      }
-
-      if (sync) {
-        // In sync call we need to wait till its published and if any error then
-        // throw it back
-        try {
-          entitiesHolder.get();
-        } catch (ExecutionException e) {
-          throw new YarnException("Failed while publishing entity",
-              e.getCause());
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new YarnException("Interrupted while publishing entity", e);
-        }
-      }
-    }
-
-    public void start() {
-      executor = Executors.newSingleThreadExecutor();
-      executor.execute(createRunnable());
-    }
-
-    public void stop() {
-      LOG.info("Stopping TimelineClient.");
-      executor.shutdownNow();
-      try {
-        executor.awaitTermination(DRAIN_TIME_PERIOD, TimeUnit.MILLISECONDS);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        e.printStackTrace();
-      }
-    }
-  }
 }


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


[24/50] [abbrv] hadoop git commit: Addendum for YARN-5865. Retrospect updateApplicationPriority api to handle state store exception in align with YARN-5611

Posted by va...@apache.org.
Addendum for YARN-5865. Retrospect updateApplicationPriority api to handle state store exception in align with YARN-5611


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

Branch: refs/heads/YARN-5355_branch2
Commit: 3f3e926ba97532d78778b4abf61654a6e4fd48f1
Parents: 39b383a
Author: Varun Saxena <va...@apache.org>
Authored: Wed Apr 26 15:43:32 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 15:43:32 2017 +0530

----------------------------------------------------------------------
 .../server/resourcemanager/metrics/TimelineServiceV1Publisher.java | 2 +-
 .../server/resourcemanager/metrics/TimelineServiceV2Publisher.java | 2 +-
 .../resourcemanager/metrics/TestSystemMetricsPublisherForV2.java   | 1 +
 3 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f3e926b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
index 89a8616..4c371a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV1Publisher.java
@@ -88,7 +88,7 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
         ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO,
         app.getApplicationSubmissionContext().getUnmanagedAM());
     entityInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO,
-        app.getApplicationSubmissionContext().getPriority().getPriority());
+        app.getApplicationPriority().getPriority());
     entityInfo.put(ApplicationMetricsConstants.AM_NODE_LABEL_EXPRESSION,
         app.getAmNodeLabelExpression());
     entityInfo.put(ApplicationMetricsConstants.APP_NODE_LABEL_EXPRESSION,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f3e926b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
index 495c36c..f3b495b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
@@ -118,7 +118,7 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
         ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO,
         app.getApplicationSubmissionContext().getUnmanagedAM());
     entityInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO,
-        app.getApplicationSubmissionContext().getPriority().getPriority());
+        app.getApplicationPriority().getPriority());
     entity.getConfigs().put(
         ApplicationMetricsConstants.AM_NODE_LABEL_EXPRESSION,
         app.getAmNodeLabelExpression());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3f3e926b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
index 0f7d9d6..c34307b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisherForV2.java
@@ -372,6 +372,7 @@ public class TestSystemMetricsPublisherForV2 {
         .thenReturn(Collections.singletonList("java -Xmx1024m"));
     when(appSubmissionContext.getAMContainerSpec())
         .thenReturn(containerLaunchContext);
+    when(app.getApplicationPriority()).thenReturn(Priority.newInstance(10));
 
     when(app.getApplicationSubmissionContext())
         .thenReturn(appSubmissionContext);


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


[44/50] [abbrv] hadoop git commit: YARN-6734. Ensure sub-application user is extracted & sent to timeline service (Rohith Sharma K S via Varun Saxena)

Posted by va...@apache.org.
YARN-6734. Ensure sub-application user is extracted & sent to timeline service (Rohith Sharma K S via Varun Saxena)

(cherry picked from commit 0443928d771e3e21825b4f487e8c0865ea641970)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 69d2c1e46eb414ddbb84b1981ad89d0179ee3a69
Parents: aeecf69
Author: Varun Saxena <va...@apache.org>
Authored: Fri Jul 28 22:02:19 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Fri Jul 28 23:02:34 2017 +0530

----------------------------------------------------------------------
 ...stTimelineReaderWebServicesHBaseStorage.java |  26 ++-
 .../storage/DataGeneratorForTest.java           |  49 ++--
 .../storage/TestHBaseTimelineStorageApps.java   |  22 +-
 .../TestHBaseTimelineStorageEntities.java       | 128 +++++++++-
 .../flow/TestHBaseStorageFlowActivity.java      |  33 ++-
 .../storage/flow/TestHBaseStorageFlowRun.java   |  84 +++++--
 .../flow/TestHBaseStorageFlowRunCompaction.java |  14 +-
 .../storage/HBaseTimelineWriterImpl.java        | 232 +++++++++++--------
 .../SubApplicationRowKeyPrefix.java             |  20 --
 .../collector/TimelineCollector.java            |  11 +-
 .../storage/FileSystemTimelineWriterImpl.java   |  15 +-
 .../timelineservice/storage/TimelineWriter.java |  28 +--
 .../collector/TestTimelineCollector.java        |  12 +-
 .../TestFileSystemTimelineWriterImpl.java       |   8 +-
 14 files changed, 465 insertions(+), 217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/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 b36eb9d..4f6ba03 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
@@ -36,6 +36,7 @@ import java.util.Set;
 import javax.ws.rs.core.MediaType;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
@@ -47,6 +48,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
@@ -334,16 +336,21 @@ public class TestTimelineReaderWebServicesHBaseStorage
 
     HBaseTimelineWriterImpl hbi = null;
     Configuration c1 = getHBaseTestingUtility().getConfiguration();
+    UserGroupInformation remoteUser =
+        UserGroupInformation.createRemoteUser(user);
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
-      hbi.write(cluster, user, flow, flowVersion, runid, entity.getId(), te);
-      hbi.write(cluster, user, flow, flowVersion, runid, entity1.getId(), te1);
-      hbi.write(cluster, user, flow, flowVersion, runid1, entity4.getId(), te4);
-      hbi.write(cluster, user, flow2,
-          flowVersion2, runid2, entity3.getId(), te3);
-      hbi.write(cluster, user, flow, flowVersion, runid,
-          "application_1111111111_1111", userEntities);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, entity.getId()), te, remoteUser);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, entity1.getId()), te1, remoteUser);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid1, entity4.getId()), te4, remoteUser);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow2, flowVersion2,
+          runid2, entity3.getId()), te3, remoteUser);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, "application_1111111111_1111"), userEntities, remoteUser);
       writeApplicationEntities(hbi, ts);
       hbi.flush();
     } finally {
@@ -375,8 +382,9 @@ public class TestTimelineReaderWebServicesHBaseStorage
 
         appEntity.addEvent(finished);
         te.addEntity(appEntity);
-        hbi.write("cluster1", "user1", "flow1", "CF7022C10F1354", i,
-            appEntity.getId(), te);
+        hbi.write(new TimelineCollectorContext("cluster1", "user1", "flow1",
+            "CF7022C10F1354", i, appEntity.getId()), te,
+            UserGroupInformation.createRemoteUser("user1"));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/DataGeneratorForTest.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/storage/DataGeneratorForTest.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/storage/DataGeneratorForTest.java
index 926d8bb..cf6a854 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/storage/DataGeneratorForTest.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/storage/DataGeneratorForTest.java
@@ -26,6 +26,7 @@ import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
@@ -34,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 
 /**
  * Utility class that creates the schema and generates test data.
@@ -155,17 +157,20 @@ public final class DataGeneratorForTest {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(util.getConfiguration());
       hbi.start();
-      String cluster = "cluster1";
-      String user = "user1";
-      String flow = "some_flow_name";
-      String flowVersion = "AB7822C10F1111";
-      long runid = 1002345678919L;
-      String appName = "application_1111111111_2222";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
-      appName = "application_1111111111_3333";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
-      appName = "application_1111111111_4444";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te2);
+      UserGroupInformation remoteUser =
+          UserGroupInformation.createRemoteUser("user1");
+      hbi.write(
+          new TimelineCollectorContext("cluster1", "user1", "some_flow_name",
+              "AB7822C10F1111", 1002345678919L, "application_1111111111_2222"),
+          te, remoteUser);
+      hbi.write(
+          new TimelineCollectorContext("cluster1", "user1", "some_flow_name",
+              "AB7822C10F1111", 1002345678919L, "application_1111111111_3333"),
+          te1, remoteUser);
+      hbi.write(
+          new TimelineCollectorContext("cluster1", "user1", "some_flow_name",
+              "AB7822C10F1111", 1002345678919L, "application_1111111111_4444"),
+          te2, remoteUser);
       hbi.stop();
     } finally {
       if (hbi != null) {
@@ -433,15 +438,19 @@ public final class DataGeneratorForTest {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(util.getConfiguration());
       hbi.start();
-      String cluster = "cluster1";
-      String user = "user1";
-      String flow = "some_flow_name";
-      String flowVersion = "AB7822C10F1111";
-      long runid = 1002345678919L;
-      hbi.write(cluster, user, flow, flowVersion, runid, appName1, te);
-      hbi.write(cluster, user, flow, flowVersion, runid, appName2, te);
-      hbi.write(cluster, user, flow, flowVersion, runid, appName1, appTe1);
-      hbi.write(cluster, user, flow, flowVersion, runid, appName2, appTe2);
+
+      UserGroupInformation user =
+          UserGroupInformation.createRemoteUser("user1");
+      TimelineCollectorContext context =
+          new TimelineCollectorContext("cluster1", "user1", "some_flow_name",
+              "AB7822C10F1111", 1002345678919L, appName1);
+      hbi.write(context, te, user);
+      hbi.write(context, appTe1, user);
+
+      context = new TimelineCollectorContext("cluster1", "user1",
+          "some_flow_name", "AB7822C10F1111", 1002345678919L, appName2);
+      hbi.write(context, te, user);
+      hbi.write(context, appTe2, user);
       hbi.stop();
     } finally {
       if (hbi != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.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/storage/TestHBaseTimelineStorageApps.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/storage/TestHBaseTimelineStorageApps.java
index d6b0370..65c7034 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/storage/TestHBaseTimelineStorageApps.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/storage/TestHBaseTimelineStorageApps.java
@@ -41,6 +41,7 @@ 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.util.Bytes;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
@@ -51,6 +52,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 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;
@@ -161,7 +163,8 @@ public class TestHBaseTimelineStorageApps {
       String flow = null;
       String flowVersion = "AB7822C10F1111";
       long runid = 1002345678919L;
-      hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appId), te, UserGroupInformation.createRemoteUser(user));
       hbi.stop();
 
       // retrieve the row
@@ -279,7 +282,8 @@ public class TestHBaseTimelineStorageApps {
       String flow = "s!ome_f\tlow  _n am!e";
       String flowVersion = "AB7822C10F1111";
       long runid = 1002345678919L;
-      hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appId), te, UserGroupInformation.createRemoteUser(user));
 
       // Write entity again, this time without created time.
       entity = new ApplicationEntity();
@@ -291,7 +295,8 @@ public class TestHBaseTimelineStorageApps {
       entity.addInfo(infoMap1);
       te = new TimelineEntities();
       te.addEntity(entity);
-      hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appId), te, UserGroupInformation.createRemoteUser(user));
       hbi.stop();
 
       infoMap.putAll(infoMap1);
@@ -512,7 +517,9 @@ public class TestHBaseTimelineStorageApps {
       String flowVersion = "1111F01C2287BA";
       long runid = 1009876543218L;
       String appName = "application_123465899910_1001";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), entities,
+          UserGroupInformation.createRemoteUser(user));
       hbi.stop();
 
       // retrieve the row
@@ -627,14 +634,17 @@ public class TestHBaseTimelineStorageApps {
       hbi.init(c1);
       hbi.start();
       // Writing application entity.
+      TimelineCollectorContext context = new TimelineCollectorContext("c1",
+          "u1", "f1", "v1", 1002345678919L, appId);
+      UserGroupInformation user = UserGroupInformation.createRemoteUser("u1");
       try {
-        hbi.write("c1", "u1", "f1", "v1", 1002345678919L, appId, teApp);
+        hbi.write(context, teApp, user);
         Assert.fail("Expected an exception as metric values are non integral");
       } catch (IOException e) {}
 
       // Writing generic entity.
       try {
-        hbi.write("c1", "u1", "f1", "v1", 1002345678919L, appId, teEntity);
+        hbi.write(context, teEntity, user);
         Assert.fail("Expected an exception as metric values are non integral");
       } catch (IOException e) {}
       hbi.stop();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.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/storage/TestHBaseTimelineStorageEntities.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/storage/TestHBaseTimelineStorageEntities.java
index 7ac5b36..422316b 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/storage/TestHBaseTimelineStorageEntities.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/storage/TestHBaseTimelineStorageEntities.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 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.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
@@ -48,6 +49,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 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;
@@ -71,6 +73,11 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumn;
+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.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -196,12 +203,15 @@ public class TestHBaseTimelineStorageEntities {
       hbi.start();
       String cluster = "cluster_test_write_entity";
       String user = "user1";
+      String subAppUser = "subAppUser1";
       String flow = "some_flow_name";
       String flowVersion = "AB7822C10F1111";
       long runid = 1002345678919L;
       String appName = ApplicationId.newInstance(System.currentTimeMillis() +
           9000000L, 1).toString();
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te,
+          UserGroupInformation.createRemoteUser(subAppUser));
       hbi.stop();
 
       // scan the table and see that entity exists
@@ -352,6 +362,11 @@ public class TestHBaseTimelineStorageEntities {
         assertEquals(metricValues.get(ts - 20000),
             metric.getValues().get(ts - 20000));
       }
+
+      // verify for sub application table entities.
+      verifySubApplicationTableEntities(cluster, user, flow, flowVersion, runid,
+          appName, subAppUser, c1, entity, id, type, infoMap, isRelatedTo,
+          relatesTo, conf, metricValues, metrics, cTime, m1);
     } finally {
       if (hbi != null) {
         hbi.stop();
@@ -360,6 +375,98 @@ public class TestHBaseTimelineStorageEntities {
     }
   }
 
+  private void verifySubApplicationTableEntities(String cluster, String user,
+      String flow, String flowVersion, Long runid, String appName,
+      String subAppUser, Configuration c1, TimelineEntity entity, String id,
+      String type, Map<String, Object> infoMap,
+      Map<String, Set<String>> isRelatedTo, Map<String, Set<String>> relatesTo,
+      Map<String, String> conf, Map<Long, Number> metricValues,
+      Set<TimelineMetric> metrics, Long cTime, TimelineMetric m1)
+      throws IOException {
+    Scan s = new Scan();
+    // read from SubApplicationTable
+    byte[] startRow = new SubApplicationRowKeyPrefix(cluster, subAppUser, null,
+        null, null, null).getRowKeyPrefix();
+    s.setStartRow(startRow);
+    s.setMaxVersions(Integer.MAX_VALUE);
+    Connection conn = ConnectionFactory.createConnection(c1);
+    ResultScanner scanner =
+        new SubApplicationTable().getResultScanner(c1, conn, s);
+
+    int rowCount = 0;
+    int colCount = 0;
+    KeyConverter<String> stringKeyConverter = new StringKeyConverter();
+    for (Result result : scanner) {
+      if (result != null && !result.isEmpty()) {
+        rowCount++;
+        colCount += result.size();
+        byte[] row1 = result.getRow();
+        assertTrue(verifyRowKeyForSubApplication(row1, subAppUser, cluster,
+            user, entity));
+
+        // check info column family
+        String id1 = SubApplicationColumn.ID.readResult(result).toString();
+        assertEquals(id, id1);
+
+        String type1 = SubApplicationColumn.TYPE.readResult(result).toString();
+        assertEquals(type, type1);
+
+        Long cTime1 =
+            (Long) SubApplicationColumn.CREATED_TIME.readResult(result);
+        assertEquals(cTime1, cTime);
+
+        Map<String, Object> infoColumns = SubApplicationColumnPrefix.INFO
+            .readResults(result, new StringKeyConverter());
+        assertEquals(infoMap, infoColumns);
+
+        // Remember isRelatedTo is of type Map<String, Set<String>>
+        for (Map.Entry<String, Set<String>> isRelatedToEntry : isRelatedTo
+            .entrySet()) {
+          Object isRelatedToValue = SubApplicationColumnPrefix.IS_RELATED_TO
+              .readResult(result, isRelatedToEntry.getKey());
+          String compoundValue = isRelatedToValue.toString();
+          // id7?id9?id6
+          Set<String> isRelatedToValues =
+              new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
+          assertEquals(isRelatedTo.get(isRelatedToEntry.getKey()).size(),
+              isRelatedToValues.size());
+          for (String v : isRelatedToEntry.getValue()) {
+            assertTrue(isRelatedToValues.contains(v));
+          }
+        }
+
+        // RelatesTo
+        for (Map.Entry<String, Set<String>> relatesToEntry : relatesTo
+            .entrySet()) {
+          String compoundValue = SubApplicationColumnPrefix.RELATES_TO
+              .readResult(result, relatesToEntry.getKey()).toString();
+          // id3?id4?id5
+          Set<String> relatesToValues =
+              new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
+          assertEquals(relatesTo.get(relatesToEntry.getKey()).size(),
+              relatesToValues.size());
+          for (String v : relatesToEntry.getValue()) {
+            assertTrue(relatesToValues.contains(v));
+          }
+        }
+
+        // Configuration
+        Map<String, Object> configColumns = SubApplicationColumnPrefix.CONFIG
+            .readResults(result, stringKeyConverter);
+        assertEquals(conf, configColumns);
+
+        NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
+            SubApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result,
+                stringKeyConverter);
+
+        NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
+        matchMetrics(metricValues, metricMap);
+      }
+    }
+    assertEquals(1, rowCount);
+    assertEquals(16, colCount);
+  }
+
   private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user,
       String flow, Long runid, String appName, TimelineEntity te) {
 
@@ -407,7 +514,9 @@ public class TestHBaseTimelineStorageEntities {
       byte[] startRow =
           new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
               .getRowKeyPrefix();
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), entities,
+          UserGroupInformation.createRemoteUser(user));
       hbi.stop();
       // scan the table and see that entity exists
       Scan s = new Scan();
@@ -510,7 +619,9 @@ public class TestHBaseTimelineStorageEntities {
       String flowVersion = "1111F01C2287BA";
       long runid = 1009876543218L;
       String appName = "application_123465899910_2001";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), entities,
+          UserGroupInformation.createRemoteUser(user));
       hbi.stop();
 
       // read the timeline entity using the reader this time
@@ -1758,4 +1869,15 @@ public class TestHBaseTimelineStorageEntities {
   public static void tearDownAfterClass() throws Exception {
     util.shutdownMiniCluster();
   }
+
+  private boolean verifyRowKeyForSubApplication(byte[] rowKey, String suAppUser,
+      String cluster, String user, TimelineEntity te) {
+    SubApplicationRowKey key = SubApplicationRowKey.parseRowKey(rowKey);
+    assertEquals(suAppUser, key.getSubAppUserId());
+    assertEquals(cluster, key.getClusterId());
+    assertEquals(te.getType(), key.getEntityType());
+    assertEquals(te.getId(), key.getEntityId());
+    assertEquals(user, key.getUserId());
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.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/storage/flow/TestHBaseStorageFlowActivity.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/storage/flow/TestHBaseStorageFlowActivity.java
index 0923105..4bf221e 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/storage/flow/TestHBaseStorageFlowActivity.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/storage/flow/TestHBaseStorageFlowActivity.java
@@ -38,12 +38,14 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 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;
@@ -117,13 +119,18 @@ public class TestHBaseStorageFlowActivity {
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+
+      UserGroupInformation remoteUser =
+          UserGroupInformation.createRemoteUser(user);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, remoteUser);
 
       // write another entity with the right min start time
       te = new TimelineEntities();
       te.addEntity(entityMinStartTime);
       appName = "application_100000000000_3333";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, remoteUser);
 
       // writer another entity for max end time
       TimelineEntity entityMaxEndTime = TestFlowDataGenerator
@@ -131,7 +138,8 @@ public class TestHBaseStorageFlowActivity {
       te = new TimelineEntities();
       te.addEntity(entityMaxEndTime);
       appName = "application_100000000000_4444";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, remoteUser);
 
       // writer another entity with greater start time
       TimelineEntity entityGreaterStartTime = TestFlowDataGenerator
@@ -139,7 +147,8 @@ public class TestHBaseStorageFlowActivity {
       te = new TimelineEntities();
       te.addEntity(entityGreaterStartTime);
       appName = "application_1000000000000000_2222";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, remoteUser);
 
       // flush everything to hbase
       hbi.flush();
@@ -227,7 +236,8 @@ public class TestHBaseStorageFlowActivity {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
       String appName = "application_1111999999_1234";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, UserGroupInformation.createRemoteUser(user));
       hbi.flush();
     } finally {
       if (hbi != null) {
@@ -340,20 +350,27 @@ public class TestHBaseStorageFlowActivity {
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
+
+      UserGroupInformation remoteUser =
+          UserGroupInformation.createRemoteUser(user);
+
       String appName = "application_11888888888_1111";
-      hbi.write(cluster, user, flow, flowVersion1, runid1, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion1,
+          runid1, appName), te, remoteUser);
 
       // write an application with to this flow but a different runid/ version
       te = new TimelineEntities();
       te.addEntity(entityApp1);
       appName = "application_11888888888_2222";
-      hbi.write(cluster, user, flow, flowVersion2, runid2, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion2,
+          runid2, appName), te, remoteUser);
 
       // write an application with to this flow but a different runid/ version
       te = new TimelineEntities();
       te.addEntity(entityApp1);
       appName = "application_11888888888_3333";
-      hbi.write(cluster, user, flow, flowVersion3, runid3, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion3,
+          runid3, appName), te, remoteUser);
 
       hbi.flush();
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.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/storage/flow/TestHBaseStorageFlowRun.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/storage/flow/TestHBaseStorageFlowRun.java
index acfdc4d..1ad02e1 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/storage/flow/TestHBaseStorageFlowRun.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/storage/flow/TestHBaseStorageFlowRun.java
@@ -43,11 +43,13 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 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;
@@ -181,13 +183,18 @@ public class TestHBaseStorageFlowRun {
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+
+      UserGroupInformation remoteUser =
+          UserGroupInformation.createRemoteUser(user);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, remoteUser);
 
       // write another entity with the right min start time
       te = new TimelineEntities();
       te.addEntity(entityMinStartTime);
       appName = "application_100000000000_3333";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, remoteUser);
 
       // writer another entity for max end time
       TimelineEntity entityMaxEndTime = TestFlowDataGenerator
@@ -195,7 +202,8 @@ public class TestHBaseStorageFlowRun {
       te = new TimelineEntities();
       te.addEntity(entityMaxEndTime);
       appName = "application_100000000000_4444";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, remoteUser);
 
       // writer another entity with greater start time
       TimelineEntity entityGreaterStartTime = TestFlowDataGenerator
@@ -203,7 +211,8 @@ public class TestHBaseStorageFlowRun {
       te = new TimelineEntities();
       te.addEntity(entityGreaterStartTime);
       appName = "application_1000000000000000_2222";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, remoteUser);
 
       // flush everything to hbase
       hbi.flush();
@@ -287,15 +296,19 @@ public class TestHBaseStorageFlowRun {
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
+      UserGroupInformation remoteUser =
+          UserGroupInformation.createRemoteUser(user);
       String appName = "application_11111111111111_1111";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, remoteUser);
       // write another application with same metric to this flow
       te = new TimelineEntities();
       TimelineEntity entityApp2 = TestFlowDataGenerator
           .getEntityMetricsApp2(System.currentTimeMillis());
       te.addEntity(entityApp2);
       appName = "application_11111111111111_2222";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, remoteUser);
       hbi.flush();
     } finally {
       if (hbi != null) {
@@ -556,15 +569,22 @@ public class TestHBaseStorageFlowRun {
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
+      UserGroupInformation remoteUser =
+          UserGroupInformation.createRemoteUser(user);
+
       String appName = "application_11111111111111_1111";
-      hbi.write(cluster, user, flow, flowVersion, 1002345678919L, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          1002345678919L, appName), te,
+          remoteUser);
       // write another application with same metric to this flow
       te = new TimelineEntities();
       TimelineEntity entityApp2 = TestFlowDataGenerator
           .getEntityMetricsApp2(System.currentTimeMillis());
       te.addEntity(entityApp2);
       appName = "application_11111111111111_2222";
-      hbi.write(cluster, user, flow, flowVersion, 1002345678918L, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          1002345678918L, appName), te,
+          remoteUser);
       hbi.flush();
     } finally {
       if (hbi != null) {
@@ -643,15 +663,20 @@ public class TestHBaseStorageFlowRun {
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
+      UserGroupInformation remoteUser =
+          UserGroupInformation.createRemoteUser(user);
+
       String appName = "application_11111111111111_1111";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, remoteUser);
       // write another application with same metric to this flow
       te = new TimelineEntities();
       TimelineEntity entityApp2 = TestFlowDataGenerator
           .getEntityMetricsApp2(System.currentTimeMillis());
       te.addEntity(entityApp2);
       appName = "application_11111111111111_2222";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+          runid, appName), te, remoteUser);
       hbi.flush();
     } finally {
       if (hbi != null) {
@@ -737,6 +762,8 @@ public class TestHBaseStorageFlowRun {
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
+      UserGroupInformation remoteUser =
+          UserGroupInformation.createRemoteUser(user);
 
       for (int i = start; i < count; i++) {
         String appName = "application_1060350000000_" + appIdSuffix;
@@ -746,7 +773,8 @@ public class TestHBaseStorageFlowRun {
         te1.addEntity(entityApp1);
         entityApp2 = TestFlowDataGenerator.getMaxFlushEntity(insertTs);
         te1.addEntity(entityApp2);
-        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+        hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+            runid, appName), te1, remoteUser);
         Thread.sleep(1);
 
         appName = "application_1001199480000_7" + appIdSuffix;
@@ -758,7 +786,9 @@ public class TestHBaseStorageFlowRun {
         entityApp2 = TestFlowDataGenerator.getMaxFlushEntity(insertTs);
         te1.addEntity(entityApp2);
 
-        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+        hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+            runid, appName), te1,
+            remoteUser);
         if (i % 1000 == 0) {
           hbi.flush();
           checkMinMaxFlush(c1, minTS, startTs, count, cluster, user, flow,
@@ -826,16 +856,23 @@ public class TestHBaseStorageFlowRun {
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
-      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678919L,
-          "application_11111111111111_1111", te);
+      UserGroupInformation remoteUser =
+          UserGroupInformation.createRemoteUser(user);
+
+      hbi.write(
+          new TimelineCollectorContext(cluster, user, flow, "CF7022C10F1354",
+              1002345678919L, "application_11111111111111_1111"),
+          te, remoteUser);
       // write another application with same metric to this flow
       te = new TimelineEntities();
       TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2(
           System.currentTimeMillis());
       entityApp2.setCreatedTime(1425016502000L);
       te.addEntity(entityApp2);
-      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678918L,
-          "application_11111111111111_2222", te);
+      hbi.write(
+          new TimelineCollectorContext(cluster, user, flow, "CF7022C10F1354",
+              1002345678918L, "application_11111111111111_2222"),
+          te, remoteUser);
       hbi.flush();
     } finally {
       if (hbi != null) {
@@ -911,15 +948,22 @@ public class TestHBaseStorageFlowRun {
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
-      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678919L,
-          "application_11111111111111_1111", te);
+      UserGroupInformation remoteUser =
+          UserGroupInformation.createRemoteUser(user);
+
+      hbi.write(
+          new TimelineCollectorContext(cluster, user, flow, "CF7022C10F1354",
+              1002345678919L, "application_11111111111111_1111"),
+          te, remoteUser);
       // write another application with same metric to this flow
       te = new TimelineEntities();
       TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2(
           System.currentTimeMillis());
       te.addEntity(entityApp2);
-      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678918L,
-          "application_11111111111111_2222", te);
+      hbi.write(
+          new TimelineCollectorContext(cluster, user, flow, "CF7022C10F1354",
+              1002345678918L, "application_11111111111111_2222"),
+          te, remoteUser);
       hbi.flush();
     } finally {
       if (hbi != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.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/storage/flow/TestHBaseStorageFlowRunCompaction.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/storage/flow/TestHBaseStorageFlowRunCompaction.java
index fa9d029..0ef8260 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/storage/flow/TestHBaseStorageFlowRunCompaction.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/storage/flow/TestHBaseStorageFlowRunCompaction.java
@@ -48,8 +48,10 @@ import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 import org.apache.hadoop.yarn.server.timelineservice.storage.DataGeneratorForTest;
 import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
@@ -280,9 +282,12 @@ public class TestHBaseStorageFlowRunCompaction {
     Configuration c1 = util.getConfiguration();
     TimelineEntities te1 = null;
     TimelineEntity entityApp1 = null;
+    UserGroupInformation remoteUser =
+        UserGroupInformation.createRemoteUser(user);
     try {
       hbi = new HBaseTimelineWriterImpl();
       hbi.init(c1);
+
       // now insert count * ( 100 + 100) metrics
       // each call to getEntityMetricsApp1 brings back 100 values
       // of metric1 and 100 of metric2
@@ -292,14 +297,16 @@ public class TestHBaseStorageFlowRunCompaction {
         te1 = new TimelineEntities();
         entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1(insertTs, c1);
         te1.addEntity(entityApp1);
-        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+        hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+            runid, appName), te1, remoteUser);
 
         appName = "application_2048000000000_7" + appIdSuffix;
         insertTs++;
         te1 = new TimelineEntities();
         entityApp1 = TestFlowDataGenerator.getEntityMetricsApp2(insertTs);
         te1.addEntity(entityApp1);
-        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+        hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+            runid, appName), te1, remoteUser);
       }
     } finally {
       String appName = "application_10240000000000_" + appIdSuffix;
@@ -308,7 +315,8 @@ public class TestHBaseStorageFlowRunCompaction {
           insertTs + 1, c1);
       te1.addEntity(entityApp1);
       if (hbi != null) {
-        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+        hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
+            runid, appName), te1, remoteUser);
         hbi.flush();
         hbi.close();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.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/HBaseTimelineWriterImpl.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/HBaseTimelineWriterImpl.java
index dfd63bf..b05bfd8 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/HBaseTimelineWriterImpl.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/HBaseTimelineWriterImpl.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.AbstractService;
 import  org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
@@ -36,6 +37,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
@@ -65,6 +67,10 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.subapplication.SubApplicationColumn;
+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.SubApplicationTable;
 
 /**
  * This implements a hbase based backend for storing the timeline entity
@@ -85,6 +91,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   private TypedBufferedMutator<ApplicationTable> applicationTable;
   private TypedBufferedMutator<FlowActivityTable> flowActivityTable;
   private TypedBufferedMutator<FlowRunTable> flowRunTable;
+  private TypedBufferedMutator<SubApplicationTable> subApplicationTable;
 
   /**
    * Used to convert strings key components to and from storage format.
@@ -97,6 +104,10 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
    */
   private final KeyConverter<Long> longKeyConverter = new LongKeyConverter();
 
+  private enum Tables {
+    APPLICATION_TABLE, ENTITY_TABLE, SUBAPPLICATION_TABLE
+  };
+
   public HBaseTimelineWriterImpl() {
     super(HBaseTimelineWriterImpl.class.getName());
   }
@@ -116,17 +127,28 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     flowRunTable = new FlowRunTable().getTableMutator(hbaseConf, conn);
     flowActivityTable =
         new FlowActivityTable().getTableMutator(hbaseConf, conn);
+    subApplicationTable =
+        new SubApplicationTable().getTableMutator(hbaseConf, conn);
   }
 
   /**
    * Stores the entire information in TimelineEntities to the timeline store.
    */
   @Override
-  public TimelineWriteResponse write(String clusterId, String userId,
-      String flowName, String flowVersion, long flowRunId, String appId,
-      TimelineEntities data) throws IOException {
+  public TimelineWriteResponse write(TimelineCollectorContext context,
+      TimelineEntities data, UserGroupInformation callerUgi)
+      throws IOException {
 
     TimelineWriteResponse putStatus = new TimelineWriteResponse();
+
+    String clusterId = context.getClusterId();
+    String userId = context.getUserId();
+    String flowName = context.getFlowName();
+    String flowVersion = context.getFlowVersion();
+    long flowRunId = context.getFlowRunId();
+    String appId = context.getAppId();
+    String subApplicationUser = callerUgi.getShortUserName();
+
     // defensive coding to avoid NPE during row key construction
     if ((flowName == null) || (appId == null) || (clusterId == null)
         || (userId == null)) {
@@ -152,18 +174,22 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
             new ApplicationRowKey(clusterId, userId, flowName, flowRunId,
                 appId);
         rowKey = applicationRowKey.getRowKey();
+        store(rowKey, te, flowVersion, Tables.APPLICATION_TABLE);
       } else {
         EntityRowKey entityRowKey =
             new EntityRowKey(clusterId, userId, flowName, flowRunId, appId,
                 te.getType(), te.getIdPrefix(), te.getId());
         rowKey = entityRowKey.getRowKey();
+        store(rowKey, te, flowVersion, Tables.ENTITY_TABLE);
       }
 
-      storeInfo(rowKey, te, flowVersion, isApplication);
-      storeEvents(rowKey, te.getEvents(), isApplication);
-      storeConfig(rowKey, te.getConfigs(), isApplication);
-      storeMetrics(rowKey, te.getMetrics(), isApplication);
-      storeRelations(rowKey, te, isApplication);
+      if (!isApplication && !userId.equals(subApplicationUser)) {
+        SubApplicationRowKey subApplicationRowKey =
+            new SubApplicationRowKey(subApplicationUser, clusterId,
+                te.getType(), te.getIdPrefix(), te.getId(), userId);
+        rowKey = subApplicationRowKey.getRowKey();
+        store(rowKey, te, flowVersion, Tables.SUBAPPLICATION_TABLE);
+      }
 
       if (isApplication) {
         TimelineEvent event =
@@ -304,72 +330,108 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     }
   }
 
-  private void storeRelations(byte[] rowKey, TimelineEntity te,
-      boolean isApplication) throws IOException {
-    if (isApplication) {
-      storeRelations(rowKey, te.getIsRelatedToEntities(),
-          ApplicationColumnPrefix.IS_RELATED_TO, applicationTable);
-      storeRelations(rowKey, te.getRelatesToEntities(),
-          ApplicationColumnPrefix.RELATES_TO, applicationTable);
-    } else {
-      storeRelations(rowKey, te.getIsRelatedToEntities(),
-          EntityColumnPrefix.IS_RELATED_TO, entityTable);
-      storeRelations(rowKey, te.getRelatesToEntities(),
-          EntityColumnPrefix.RELATES_TO, entityTable);
-    }
-  }
-
   /**
    * Stores the Relations from the {@linkplain TimelineEntity} object.
    */
   private <T> void storeRelations(byte[] rowKey,
-      Map<String, Set<String>> connectedEntities,
-      ColumnPrefix<T> columnPrefix, TypedBufferedMutator<T> table)
-          throws IOException {
-    for (Map.Entry<String, Set<String>> connectedEntity : connectedEntities
-        .entrySet()) {
-      // id3?id4?id5
-      String compoundValue =
-          Separator.VALUES.joinEncoded(connectedEntity.getValue());
-      columnPrefix.store(rowKey, table,
-          stringKeyConverter.encode(connectedEntity.getKey()), null,
-          compoundValue);
+      Map<String, Set<String>> connectedEntities, ColumnPrefix<T> columnPrefix,
+      TypedBufferedMutator<T> table) throws IOException {
+    if (connectedEntities != null) {
+      for (Map.Entry<String, Set<String>> connectedEntity : connectedEntities
+          .entrySet()) {
+        // id3?id4?id5
+        String compoundValue =
+            Separator.VALUES.joinEncoded(connectedEntity.getValue());
+        columnPrefix.store(rowKey, table,
+            stringKeyConverter.encode(connectedEntity.getKey()), null,
+            compoundValue);
+      }
     }
   }
 
   /**
    * Stores information from the {@linkplain TimelineEntity} object.
    */
-  private void storeInfo(byte[] rowKey, TimelineEntity te, String flowVersion,
-      boolean isApplication) throws IOException {
-
-    if (isApplication) {
+  private void store(byte[] rowKey, TimelineEntity te,
+      String flowVersion,
+      Tables table) throws IOException {
+    switch (table) {
+    case APPLICATION_TABLE:
       ApplicationColumn.ID.store(rowKey, applicationTable, null, te.getId());
       ApplicationColumn.CREATED_TIME.store(rowKey, applicationTable, null,
           te.getCreatedTime());
       ApplicationColumn.FLOW_VERSION.store(rowKey, applicationTable, null,
           flowVersion);
-      Map<String, Object> info = te.getInfo();
-      if (info != null) {
-        for (Map.Entry<String, Object> entry : info.entrySet()) {
-          ApplicationColumnPrefix.INFO.store(rowKey, applicationTable,
-              stringKeyConverter.encode(entry.getKey()), null,
-              entry.getValue());
-        }
-      }
-    } else {
+      storeInfo(rowKey, te.getInfo(), flowVersion, ApplicationColumnPrefix.INFO,
+          applicationTable);
+      storeMetrics(rowKey, te.getMetrics(), ApplicationColumnPrefix.METRIC,
+          applicationTable);
+      storeEvents(rowKey, te.getEvents(), ApplicationColumnPrefix.EVENT,
+          applicationTable);
+      storeConfig(rowKey, te.getConfigs(), ApplicationColumnPrefix.CONFIG,
+          applicationTable);
+      storeRelations(rowKey, te.getIsRelatedToEntities(),
+          ApplicationColumnPrefix.IS_RELATED_TO, applicationTable);
+      storeRelations(rowKey, te.getRelatesToEntities(),
+          ApplicationColumnPrefix.RELATES_TO, applicationTable);
+      break;
+    case ENTITY_TABLE:
       EntityColumn.ID.store(rowKey, entityTable, null, te.getId());
       EntityColumn.TYPE.store(rowKey, entityTable, null, te.getType());
       EntityColumn.CREATED_TIME.store(rowKey, entityTable, null,
           te.getCreatedTime());
       EntityColumn.FLOW_VERSION.store(rowKey, entityTable, null, flowVersion);
-      Map<String, Object> info = te.getInfo();
-      if (info != null) {
-        for (Map.Entry<String, Object> entry : info.entrySet()) {
-          EntityColumnPrefix.INFO.store(rowKey, entityTable,
-              stringKeyConverter.encode(entry.getKey()), null,
-              entry.getValue());
-        }
+      storeInfo(rowKey, te.getInfo(), flowVersion, EntityColumnPrefix.INFO,
+          entityTable);
+      storeMetrics(rowKey, te.getMetrics(), EntityColumnPrefix.METRIC,
+          entityTable);
+      storeEvents(rowKey, te.getEvents(), EntityColumnPrefix.EVENT,
+          entityTable);
+      storeConfig(rowKey, te.getConfigs(), EntityColumnPrefix.CONFIG,
+          entityTable);
+      storeRelations(rowKey, te.getIsRelatedToEntities(),
+          EntityColumnPrefix.IS_RELATED_TO, entityTable);
+      storeRelations(rowKey, te.getRelatesToEntities(),
+          EntityColumnPrefix.RELATES_TO, entityTable);
+      break;
+    case SUBAPPLICATION_TABLE:
+      SubApplicationColumn.ID.store(rowKey, subApplicationTable, null,
+          te.getId());
+      SubApplicationColumn.TYPE.store(rowKey, subApplicationTable, null,
+          te.getType());
+      SubApplicationColumn.CREATED_TIME.store(rowKey, subApplicationTable, null,
+          te.getCreatedTime());
+      SubApplicationColumn.FLOW_VERSION.store(rowKey, subApplicationTable, null,
+          flowVersion);
+      storeInfo(rowKey, te.getInfo(), flowVersion,
+          SubApplicationColumnPrefix.INFO, subApplicationTable);
+      storeMetrics(rowKey, te.getMetrics(), SubApplicationColumnPrefix.METRIC,
+          subApplicationTable);
+      storeEvents(rowKey, te.getEvents(), SubApplicationColumnPrefix.EVENT,
+          subApplicationTable);
+      storeConfig(rowKey, te.getConfigs(), SubApplicationColumnPrefix.CONFIG,
+          subApplicationTable);
+      storeRelations(rowKey, te.getIsRelatedToEntities(),
+          SubApplicationColumnPrefix.IS_RELATED_TO, subApplicationTable);
+      storeRelations(rowKey, te.getRelatesToEntities(),
+          SubApplicationColumnPrefix.RELATES_TO, subApplicationTable);
+      break;
+    default:
+      LOG.info("Invalid table name provided.");
+      break;
+    }
+  }
+
+  /**
+   * stores the info information from {@linkplain TimelineEntity}.
+   */
+  private <T> void storeInfo(byte[] rowKey, Map<String, Object> info,
+      String flowVersion, ColumnPrefix<T> columnPrefix,
+      TypedBufferedMutator<T> table) throws IOException {
+    if (info != null) {
+      for (Map.Entry<String, Object> entry : info.entrySet()) {
+        columnPrefix.store(rowKey, table,
+            stringKeyConverter.encode(entry.getKey()), null, entry.getValue());
       }
     }
   }
@@ -377,19 +439,13 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   /**
    * stores the config information from {@linkplain TimelineEntity}.
    */
-  private void storeConfig(byte[] rowKey, Map<String, String> config,
-      boolean isApplication) throws IOException {
-    if (config == null) {
-      return;
-    }
-    for (Map.Entry<String, String> entry : config.entrySet()) {
-      byte[] configKey = stringKeyConverter.encode(entry.getKey());
-      if (isApplication) {
-        ApplicationColumnPrefix.CONFIG.store(rowKey, applicationTable,
-            configKey, null, entry.getValue());
-      } else {
-        EntityColumnPrefix.CONFIG.store(rowKey, entityTable, configKey,
-            null, entry.getValue());
+  private <T> void storeConfig(byte[] rowKey, Map<String, String> config,
+      ColumnPrefix<T> columnPrefix, TypedBufferedMutator<T> table)
+      throws IOException {
+    if (config != null) {
+      for (Map.Entry<String, String> entry : config.entrySet()) {
+        byte[] configKey = stringKeyConverter.encode(entry.getKey());
+        columnPrefix.store(rowKey, table, configKey, null, entry.getValue());
       }
     }
   }
@@ -398,8 +454,9 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
    * stores the {@linkplain TimelineMetric} information from the
    * {@linkplain TimelineEvent} object.
    */
-  private void storeMetrics(byte[] rowKey, Set<TimelineMetric> metrics,
-      boolean isApplication) throws IOException {
+  private <T> void storeMetrics(byte[] rowKey, Set<TimelineMetric> metrics,
+      ColumnPrefix<T> columnPrefix, TypedBufferedMutator<T> table)
+      throws IOException {
     if (metrics != null) {
       for (TimelineMetric metric : metrics) {
         byte[] metricColumnQualifier =
@@ -407,13 +464,8 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
         Map<Long, Number> timeseries = metric.getValues();
         for (Map.Entry<Long, Number> timeseriesEntry : timeseries.entrySet()) {
           Long timestamp = timeseriesEntry.getKey();
-          if (isApplication) {
-            ApplicationColumnPrefix.METRIC.store(rowKey, applicationTable,
-                metricColumnQualifier, timestamp, timeseriesEntry.getValue());
-          } else {
-            EntityColumnPrefix.METRIC.store(rowKey, entityTable,
-                metricColumnQualifier, timestamp, timeseriesEntry.getValue());
-          }
+          columnPrefix.store(rowKey, table, metricColumnQualifier, timestamp,
+              timeseriesEntry.getValue());
         }
       }
     }
@@ -422,8 +474,9 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   /**
    * Stores the events from the {@linkplain TimelineEvent} object.
    */
-  private void storeEvents(byte[] rowKey, Set<TimelineEvent> events,
-      boolean isApplication) throws IOException {
+  private <T> void storeEvents(byte[] rowKey, Set<TimelineEvent> events,
+      ColumnPrefix<T> columnPrefix, TypedBufferedMutator<T> table)
+      throws IOException {
     if (events != null) {
       for (TimelineEvent event : events) {
         if (event != null) {
@@ -441,26 +494,16 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
               byte[] columnQualifierBytes =
                   new EventColumnName(eventId, eventTimestamp, null)
                       .getColumnQualifier();
-              if (isApplication) {
-                ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
-                    columnQualifierBytes, null, Separator.EMPTY_BYTES);
-              } else {
-                EntityColumnPrefix.EVENT.store(rowKey, entityTable,
-                    columnQualifierBytes, null, Separator.EMPTY_BYTES);
-              }
+              columnPrefix.store(rowKey, table, columnQualifierBytes, null,
+                  Separator.EMPTY_BYTES);
             } else {
               for (Map.Entry<String, Object> info : eventInfo.entrySet()) {
                 // eventId=infoKey
                 byte[] columnQualifierBytes =
                     new EventColumnName(eventId, eventTimestamp, info.getKey())
                         .getColumnQualifier();
-                if (isApplication) {
-                  ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
-                      columnQualifierBytes, null, info.getValue());
-                } else {
-                  EntityColumnPrefix.EVENT.store(rowKey, entityTable,
-                      columnQualifierBytes, null, info.getValue());
-                }
+                columnPrefix.store(rowKey, table, columnQualifierBytes, null,
+                    info.getValue());
               } // for info: eventInfo
             }
           }
@@ -500,6 +543,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     applicationTable.flush();
     flowRunTable.flush();
     flowActivityTable.flush();
+    subApplicationTable.flush();
   }
 
   /**
@@ -532,11 +576,13 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       // The close API performs flushing and releases any resources held
       flowActivityTable.close();
     }
+    if (subApplicationTable != null) {
+      subApplicationTable.close();
+    }
     if (conn != null) {
       LOG.info("closing the hbase Connection");
       conn.close();
     }
     super.serviceStop();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/subapplication/SubApplicationRowKeyPrefix.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/subapplication/SubApplicationRowKeyPrefix.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/subapplication/SubApplicationRowKeyPrefix.java
index e42c6cd..0c04959 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/subapplication/SubApplicationRowKeyPrefix.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/subapplication/SubApplicationRowKeyPrefix.java
@@ -56,26 +56,6 @@ public class SubApplicationRowKeyPrefix extends SubApplicationRowKey
         userId);
   }
 
-  /**
-   * Creates a prefix which generates the following rowKeyPrefixes for the sub
-   * application table:
-   * {@code subAppUserId!clusterId!entityType!entityPrefix!entityId!userId}.
-   *
-   * subAppUserId is usually the doAsUser.
-   * userId is the yarn user that the AM runs as.
-   *
-   * @param clusterId
-   *          identifying the cluster
-   * @param subAppUserId
-   *          identifying the sub app user
-   * @param userId
-   *          identifying the user who runs the AM
-   */
-  public SubApplicationRowKeyPrefix(String clusterId, String subAppUserId,
-      String userId) {
-    this(subAppUserId, clusterId, null, null, null, userId);
-  }
-
   /*
    * (non-Javadoc)
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.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/collector/TimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
index 5416b26..7cc4d3d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
@@ -138,7 +138,7 @@ public abstract class TimelineCollector extends CompositeService {
     // flush the writer buffer concurrently and swallow any exception
     // caused by the timeline enitites that are being put here.
     synchronized (writer) {
-      response = writeTimelineEntities(entities);
+      response = writeTimelineEntities(entities, callerUgi);
       flushBufferedTimelineEntities();
     }
 
@@ -146,15 +146,14 @@ public abstract class TimelineCollector extends CompositeService {
   }
 
   private TimelineWriteResponse writeTimelineEntities(
-      TimelineEntities entities) throws IOException {
+      TimelineEntities entities, UserGroupInformation callerUgi)
+      throws IOException {
     // Update application metrics for aggregation
     updateAggregateStatus(entities, aggregationGroups,
         getEntityTypesSkipAggregation());
 
     final TimelineCollectorContext context = getTimelineEntityContext();
-    return writer.write(context.getClusterId(), context.getUserId(),
-        context.getFlowName(), context.getFlowVersion(),
-        context.getFlowRunId(), context.getAppId(), entities);
+    return writer.write(context, entities, callerUgi);
   }
 
   /**
@@ -186,7 +185,7 @@ public abstract class TimelineCollector extends CompositeService {
           callerUgi + ")");
     }
 
-    writeTimelineEntities(entities);
+    writeTimelineEntities(entities, callerUgi);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.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/storage/FileSystemTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java
index 1f527f2..ee41970 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java
@@ -28,12 +28,14 @@ import java.io.PrintWriter;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse.TimelineWriteError;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -68,10 +70,17 @@ public class FileSystemTimelineWriterImpl extends AbstractService
   }
 
   @Override
-  public TimelineWriteResponse write(String clusterId, String userId,
-      String flowName, String flowVersion, long flowRunId, String appId,
-      TimelineEntities entities) throws IOException {
+  public TimelineWriteResponse write(TimelineCollectorContext context,
+      TimelineEntities entities, UserGroupInformation callerUgi)
+      throws IOException {
     TimelineWriteResponse response = new TimelineWriteResponse();
+    String clusterId = context.getClusterId();
+    String userId = context.getUserId();
+    String flowName = context.getFlowName();
+    String flowVersion = context.getFlowVersion();
+    long flowRunId = context.getFlowRunId();
+    String appId = context.getAppId();
+
     for (TimelineEntity entity : entities.getEntities()) {
       write(clusterId, userId, flowName, flowVersion, flowRunId, appId, entity,
           response);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.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/storage/TimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.java
index 663a18a..12bc1cb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.java
@@ -21,10 +21,12 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 
 /**
  * This interface is for storing application timeline information.
@@ -34,25 +36,19 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
 public interface TimelineWriter extends Service {
 
   /**
-   * Stores the entire information in {@link TimelineEntities} to the
-   * timeline store. Any errors occurring for individual write request objects
-   * will be reported in the response.
+   * Stores the entire information in {@link TimelineEntities} to the timeline
+   * store. Any errors occurring for individual write request objects will be
+   * reported in the response.
    *
-   * @param clusterId context cluster ID
-   * @param userId context user ID
-   * @param flowName context flow name
-   * @param flowVersion context flow version
-   * @param flowRunId run id for the flow.
-   * @param appId context app ID.
-   * @param data
-   *          a {@link TimelineEntities} object.
+   * @param context a {@link TimelineCollectorContext}
+   * @param data a {@link TimelineEntities} object.
+   * @param callerUgi {@link UserGroupInformation}.
    * @return a {@link TimelineWriteResponse} object.
-   * @throws IOException if there is any exception encountered while storing
-   *     or writing entities to the backend storage.
+   * @throws IOException if there is any exception encountered while storing or
+   *           writing entities to the back end storage.
    */
-  TimelineWriteResponse write(String clusterId, String userId,
-      String flowName, String flowVersion, long flowRunId, String appId,
-      TimelineEntities data) throws IOException;
+  TimelineWriteResponse write(TimelineCollectorContext context,
+      TimelineEntities data, UserGroupInformation callerUgi) throws IOException;
 
   /**
    * Aggregates the entity information to the timeline store based on which

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.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/collector/TestTimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
index 0f17553..ec45428 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
@@ -41,8 +41,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
@@ -156,9 +154,8 @@ public class TestTimelineCollector {
     collector.putEntities(
         entities, UserGroupInformation.createRemoteUser("test-user"));
 
-    verify(writer, times(1)).write(
-        anyString(), anyString(), anyString(), anyString(), anyLong(),
-        anyString(), any(TimelineEntities.class));
+    verify(writer, times(1)).write(any(TimelineCollectorContext.class),
+        any(TimelineEntities.class), any(UserGroupInformation.class));
     verify(writer, times(1)).flush();
   }
 
@@ -175,9 +172,8 @@ public class TestTimelineCollector {
     collector.putEntitiesAsync(
         entities, UserGroupInformation.createRemoteUser("test-user"));
 
-    verify(writer, times(1)).write(
-        anyString(), anyString(), anyString(), anyString(), anyLong(),
-        anyString(), any(TimelineEntities.class));
+    verify(writer, times(1)).write(any(TimelineCollectorContext.class),
+        any(TimelineEntities.class), any(UserGroupInformation.class));
     verify(writer, never()).flush();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/69d2c1e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.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/storage/TestFileSystemTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java
index 4f12c57..bb9f54f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java
@@ -30,11 +30,13 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.junit.Rule;
 import org.junit.Test;
@@ -89,8 +91,10 @@ public class TestFileSystemTimelineWriterImpl {
           outputRoot);
       fsi.init(conf);
       fsi.start();
-      fsi.write("cluster_id", "user_id", "flow_name", "flow_version", 12345678L,
-          "app_id", te);
+      fsi.write(
+          new TimelineCollectorContext("cluster_id", "user_id", "flow_name",
+              "flow_version", 12345678L, "app_id"),
+          te, UserGroupInformation.createRemoteUser("user_id"));
 
       String fileName = fsi.getOutputRoot() + File.separator + "entities" +
           File.separator + "cluster_id" + File.separator + "user_id" +


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


[05/50] [abbrv] hadoop git commit: YARN-6027 addendum. Fixed the broken build for YARN-5355-branch-2. Contributed by Sangjin Lee.

Posted by va...@apache.org.
YARN-6027 addendum. Fixed the broken build for YARN-5355-branch-2. Contributed by Sangjin Lee.


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

Branch: refs/heads/YARN-5355_branch2
Commit: 6c993a39cbcb1775e47028c9ba0d1291a18c5766
Parents: 1862a03
Author: Sangjin Lee <sj...@apache.org>
Authored: Wed Mar 1 15:29:17 2017 -0800
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Apr 26 00:48:44 2017 +0530

----------------------------------------------------------------------
 .../AbstractTimelineReaderHBaseTestBase.java    | 22 +++++---------------
 1 file changed, 5 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c993a39/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/AbstractTimelineReaderHBaseTestBase.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/AbstractTimelineReaderHBaseTestBase.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/AbstractTimelineReaderHBaseTestBase.java
index 7853c94..ccfdfd0 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/AbstractTimelineReaderHBaseTestBase.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/AbstractTimelineReaderHBaseTestBase.java
@@ -86,19 +86,7 @@ public abstract class AbstractTimelineReaderHBaseTestBase {
           "org.apache.hadoop.yarn.server.timelineservice.storage."
               + "HBaseTimelineReaderImpl");
       config.setInt("hfile.format.version", 3);
-      server = new TimelineReaderServer() {
-        @Override
-        protected void setupOptions(Configuration conf) {
-          // The parent code tries to use HttpServer2 from this version of
-          // Hadoop, but the tests are loading in HttpServer2 from
-          // ${hbase-compatible-hadoop.version}. This version uses Jetty 9
-          // while ${hbase-compatible-hadoop.version} uses Jetty 6, and there
-          // are many differences, including classnames and packages.
-          // We do nothing here, so that we don't cause a NoSuchMethodError.
-          // Once ${hbase-compatible-hadoop.version} is changed to Hadoop 3,
-          // we should be able to remove this @Override.
-        }
-      };
+      server = new TimelineReaderServer();
       server.init(config);
       server.start();
       serverPort = server.getWebServerPort();
@@ -119,11 +107,11 @@ public abstract class AbstractTimelineReaderHBaseTestBase {
     ClientResponse resp =
         client.resource(uri).accept(MediaType.APPLICATION_JSON)
             .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    if (resp == null || resp.getStatusInfo()
-        .getStatusCode() != ClientResponse.Status.OK.getStatusCode()) {
+    if (resp == null ||
+        resp.getClientResponseStatus() != ClientResponse.Status.OK) {
       String msg = "";
       if (resp != null) {
-        msg = String.valueOf(resp.getStatusInfo().getStatusCode());
+        msg = String.valueOf(resp.getClientResponseStatus());
       }
       throw new IOException(
           "Incorrect response from timeline reader. " + "Status=" + msg);
@@ -137,7 +125,7 @@ public abstract class AbstractTimelineReaderHBaseTestBase {
             .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     assertNotNull(resp);
     assertTrue("Response from server should have been " + status,
-        resp.getStatusInfo().getStatusCode() == status.getStatusCode());
+        resp.getClientResponseStatus() == status);
     System.out.println("Response is: " + resp.getEntity(String.class));
   }
 


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


[41/50] [abbrv] hadoop git commit: YARN-6102. RMActiveService context to be updated with new RMContext on failover. Contributed by Rohith Sharma K S.

Posted by va...@apache.org.
YARN-6102. RMActiveService context to be updated with new RMContext on failover. Contributed by Rohith Sharma K S.

(cherry picked from commit a657472b42c58f87fd3165e0a746d83b72182a24)

Conflicts:
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java


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

Branch: refs/heads/YARN-5355_branch2
Commit: c602f05b82a4f44448ec11efe42d2d90d3b2cb59
Parents: bb4f440
Author: Sunil G <su...@apache.org>
Authored: Mon Jul 24 20:57:25 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Fri Jul 28 22:48:33 2017 +0530

----------------------------------------------------------------------
 ...ActiveStandbyElectorBasedElectorService.java |  12 +-
 .../server/resourcemanager/AdminService.java    |  71 ++---
 .../CuratorBasedElectorService.java             |  10 +-
 .../resourcemanager/RMActiveServiceContext.java |  15 +
 .../server/resourcemanager/RMContextImpl.java   | 294 ++++++++++---------
 .../resourcemanager/RMServiceContext.java       | 151 ++++++++++
 .../server/resourcemanager/ResourceManager.java |  28 +-
 .../yarn/server/resourcemanager/MockRM.java     |   2 +-
 .../resourcemanager/TestRMEmbeddedElector.java  |   8 +-
 .../yarn/server/resourcemanager/TestRMHA.java   |  16 +-
 10 files changed, 406 insertions(+), 201 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c602f05b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java
index 751eedd..7e41399 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ActiveStandbyElectorBasedElectorService.java
@@ -57,7 +57,7 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
       new HAServiceProtocol.StateChangeRequestInfo(
           HAServiceProtocol.RequestSource.REQUEST_BY_ZKFC);
 
-  private RMContext rmContext;
+  private ResourceManager rm;
 
   private byte[] localActiveNodeInfo;
   private ActiveStandbyElector elector;
@@ -66,9 +66,9 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
   @VisibleForTesting
   final Object zkDisconnectLock = new Object();
 
-  ActiveStandbyElectorBasedElectorService(RMContext rmContext) {
+  ActiveStandbyElectorBasedElectorService(ResourceManager rm) {
     super(ActiveStandbyElectorBasedElectorService.class.getName());
-    this.rmContext = rmContext;
+    this.rm = rm;
   }
 
   @Override
@@ -139,7 +139,7 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
     cancelDisconnectTimer();
 
     try {
-      rmContext.getRMAdminService().transitionToActive(req);
+      rm.getRMContext().getRMAdminService().transitionToActive(req);
     } catch (Exception e) {
       throw new ServiceFailedException("RM could not transition to Active", e);
     }
@@ -150,7 +150,7 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
     cancelDisconnectTimer();
 
     try {
-      rmContext.getRMAdminService().transitionToStandby(req);
+      rm.getRMContext().getRMAdminService().transitionToStandby(req);
     } catch (Exception e) {
       LOG.error("RM could not transition to Standby", e);
     }
@@ -204,7 +204,7 @@ public class ActiveStandbyElectorBasedElectorService extends AbstractService
   @SuppressWarnings(value = "unchecked")
   @Override
   public void notifyFatalError(String errorMessage) {
-    rmContext.getDispatcher().getEventHandler().handle(
+    rm.getRMContext().getDispatcher().getEventHandler().handle(
         new RMFatalEvent(RMFatalEventType.EMBEDDED_ELECTOR_FAILED,
             errorMessage));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c602f05b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index 74c87a2..afea100 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -102,7 +102,6 @@ public class AdminService extends CompositeService implements
 
   private static final Log LOG = LogFactory.getLog(AdminService.class);
 
-  private final RMContext rmContext;
   private final ResourceManager rm;
   private String rmId;
 
@@ -123,16 +122,16 @@ public class AdminService extends CompositeService implements
   @VisibleForTesting
   boolean isCentralizedNodeLabelConfiguration = true;
 
-  public AdminService(ResourceManager rm, RMContext rmContext) {
+  public AdminService(ResourceManager rm) {
     super(AdminService.class.getName());
     this.rm = rm;
-    this.rmContext = rmContext;
   }
 
   @Override
   public void serviceInit(Configuration conf) throws Exception {
     autoFailoverEnabled =
-        rmContext.isHAEnabled() && HAUtil.isAutomaticFailoverEnabled(conf);
+        rm.getRMContext().isHAEnabled()
+            && HAUtil.isAutomaticFailoverEnabled(conf);
 
     masterServiceBindAddress = conf.getSocketAddr(
         YarnConfiguration.RM_BIND_HOST,
@@ -189,7 +188,7 @@ public class AdminService extends CompositeService implements
           RMPolicyProvider.getInstance());
     }
 
-    if (rmContext.isHAEnabled()) {
+    if (rm.getRMContext().isHAEnabled()) {
       RPC.setProtocolEngine(conf, HAServiceProtocolPB.class,
           ProtobufRpcEngine.class);
 
@@ -265,7 +264,7 @@ public class AdminService extends CompositeService implements
   }
 
   private synchronized boolean isRMActive() {
-    return HAServiceState.ACTIVE == rmContext.getHAServiceState();
+    return HAServiceState.ACTIVE == rm.getRMContext().getHAServiceState();
   }
 
   private void throwStandbyException() throws StandbyException {
@@ -305,7 +304,7 @@ public class AdminService extends CompositeService implements
       refreshAll();
     } catch (Exception e) {
       LOG.error("RefreshAll failed so firing fatal event", e);
-      rmContext
+      rm.getRMContext()
           .getDispatcher()
           .getEventHandler()
           .handle(
@@ -364,7 +363,7 @@ public class AdminService extends CompositeService implements
   @Override
   public synchronized HAServiceStatus getServiceStatus() throws IOException {
     checkAccess("getServiceState");
-    HAServiceState haState = rmContext.getHAServiceState();
+    HAServiceState haState = rm.getRMContext().getHAServiceState();
     HAServiceStatus ret = new HAServiceStatus(haState);
     if (isRMActive() || haState == HAServiceProtocol.HAServiceState.STANDBY) {
       ret.setReadyToBecomeActive();
@@ -396,11 +395,12 @@ public class AdminService extends CompositeService implements
   }
 
   private void refreshQueues() throws IOException, YarnException {
-    rmContext.getScheduler().reinitialize(getConfig(), this.rmContext);
+    rm.getRMContext().getScheduler().reinitialize(getConfig(),
+        this.rm.getRMContext());
     // refresh the reservation system
-    ReservationSystem rSystem = rmContext.getReservationSystem();
+    ReservationSystem rSystem = rm.getRMContext().getReservationSystem();
     if (rSystem != null) {
-      rSystem.reinitialize(getConfig(), rmContext);
+      rSystem.reinitialize(getConfig(), rm.getRMContext());
     }
   }
 
@@ -419,14 +419,14 @@ public class AdminService extends CompositeService implements
               YarnConfiguration.YARN_SITE_CONFIGURATION_FILE);
       switch (request.getDecommissionType()) {
       case NORMAL:
-        rmContext.getNodesListManager().refreshNodes(conf);
+        rm.getRMContext().getNodesListManager().refreshNodes(conf);
         break;
       case GRACEFUL:
-        rmContext.getNodesListManager().refreshNodesGracefully(
+        rm.getRMContext().getNodesListManager().refreshNodesGracefully(
             conf, request.getDecommissionTimeout());
         break;
       case FORCEFUL:
-        rmContext.getNodesListManager().refreshNodesForcefully();
+        rm.getRMContext().getNodesListManager().refreshNodesForcefully();
         break;
       }
       RMAuditLogger.logSuccess(user.getShortUserName(), operation,
@@ -441,7 +441,7 @@ public class AdminService extends CompositeService implements
     Configuration conf =
         getConfiguration(new Configuration(false),
             YarnConfiguration.YARN_SITE_CONFIGURATION_FILE);
-    rmContext.getNodesListManager().refreshNodes(conf);
+    rm.getRMContext().getNodesListManager().refreshNodes(conf);
   }
 
   @Override
@@ -560,10 +560,11 @@ public class AdminService extends CompositeService implements
     Configuration conf =
         getConfiguration(new Configuration(false),
             YarnConfiguration.HADOOP_POLICY_CONFIGURATION_FILE);
-    rmContext.getClientRMService().refreshServiceAcls(conf, policyProvider);
-    rmContext.getApplicationMasterService().refreshServiceAcls(
+    rm.getRMContext().getClientRMService().refreshServiceAcls(conf,
+        policyProvider);
+    rm.getRMContext().getApplicationMasterService().refreshServiceAcls(
         conf, policyProvider);
-    rmContext.getResourceTrackerService().refreshServiceAcls(
+    rm.getRMContext().getResourceTrackerService().refreshServiceAcls(
         conf, policyProvider);
   }
 
@@ -602,7 +603,7 @@ public class AdminService extends CompositeService implements
     // if any invalid nodes, throw exception instead of partially updating
     // valid nodes.
     for (NodeId nodeId : nodeIds) {
-      RMNode node = this.rmContext.getRMNodes().get(nodeId);
+      RMNode node = this.rm.getRMContext().getRMNodes().get(nodeId);
       if (node == null) {
         LOG.error("Resource update get failed on all nodes due to change "
             + "resource on an unrecognized node: " + nodeId);
@@ -620,14 +621,14 @@ public class AdminService extends CompositeService implements
     for (Map.Entry<NodeId, ResourceOption> entry : nodeResourceMap.entrySet()) {
       ResourceOption newResourceOption = entry.getValue();
       NodeId nodeId = entry.getKey();
-      RMNode node = this.rmContext.getRMNodes().get(nodeId);
+      RMNode node = this.rm.getRMContext().getRMNodes().get(nodeId);
 
       if (node == null) {
         LOG.warn("Resource update get failed on an unrecognized node: " + nodeId);
         allSuccess = false;
       } else {
         // update resource to RMNode
-        this.rmContext.getDispatcher().getEventHandler()
+        this.rm.getRMContext().getDispatcher().getEventHandler()
           .handle(new RMNodeResourceUpdateEvent(nodeId, newResourceOption));
         LOG.info("Update resource on node(" + node.getNodeID()
             + ") with resource(" + newResourceOption.toString() + ")");
@@ -662,7 +663,8 @@ public class AdminService extends CompositeService implements
       DynamicResourceConfiguration newConf;
 
       InputStream drInputStream =
-          this.rmContext.getConfigurationProvider().getConfigurationInputStream(
+          this.rm.getRMContext().getConfigurationProvider()
+              .getConfigurationInputStream(
               configuration, YarnConfiguration.DR_CONFIGURATION_FILE);
 
       if (drInputStream != null) {
@@ -680,7 +682,7 @@ public class AdminService extends CompositeService implements
         updateNodeResource(updateRequest);
       }
       // refresh dynamic resource in ResourceTrackerService
-      this.rmContext.getResourceTrackerService().
+      this.rm.getRMContext().getResourceTrackerService().
           updateDynamicResourceConfiguration(newConf);
       RMAuditLogger.logSuccess(user.getShortUserName(), operation,
               "AdminService");
@@ -693,7 +695,8 @@ public class AdminService extends CompositeService implements
   private synchronized Configuration getConfiguration(Configuration conf,
       String... confFileNames) throws YarnException, IOException {
     for (String confFileName : confFileNames) {
-      InputStream confFileInputStream = this.rmContext.getConfigurationProvider()
+      InputStream confFileInputStream =
+          this.rm.getRMContext().getConfigurationProvider()
           .getConfigurationInputStream(conf, confFileName);
       if (confFileInputStream != null) {
         conf.addResource(confFileInputStream);
@@ -747,7 +750,7 @@ public class AdminService extends CompositeService implements
     AddToClusterNodeLabelsResponse response =
         recordFactory.newRecordInstance(AddToClusterNodeLabelsResponse.class);
     try {
-      rmContext.getNodeLabelManager()
+      rm.getRMContext().getNodeLabelManager()
           .addToCluserNodeLabels(request.getNodeLabels());
       RMAuditLogger.logSuccess(user.getShortUserName(), operation,
           "AdminService");
@@ -770,7 +773,8 @@ public class AdminService extends CompositeService implements
     RemoveFromClusterNodeLabelsResponse response =
         recordFactory.newRecordInstance(RemoveFromClusterNodeLabelsResponse.class);
     try {
-      rmContext.getNodeLabelManager().removeFromClusterNodeLabels(request.getNodeLabels());
+      rm.getRMContext().getNodeLabelManager()
+          .removeFromClusterNodeLabels(request.getNodeLabels());
       RMAuditLogger
           .logSuccess(user.getShortUserName(), operation, "AdminService");
       return response;
@@ -806,19 +810,20 @@ public class AdminService extends CompositeService implements
         boolean isKnown = false;
         // both active and inactive nodes are recognized as known nodes
         if (requestedNode.getPort() != 0) {
-          if (rmContext.getRMNodes().containsKey(requestedNode)
-              || rmContext.getInactiveRMNodes().containsKey(requestedNode)) {
+          if (rm.getRMContext().getRMNodes().containsKey(requestedNode) || rm
+              .getRMContext().getInactiveRMNodes().containsKey(requestedNode)) {
             isKnown = true;
           }
         } else {
-          for (NodeId knownNode : rmContext.getRMNodes().keySet()) {
+          for (NodeId knownNode : rm.getRMContext().getRMNodes().keySet()) {
             if (knownNode.getHost().equals(requestedNode.getHost())) {
               isKnown = true;
               break;
             }
           }
           if (!isKnown) {
-            for (NodeId knownNode : rmContext.getInactiveRMNodes().keySet()) {
+            for (NodeId knownNode : rm.getRMContext().getInactiveRMNodes()
+                .keySet()) {
               if (knownNode.getHost().equals(requestedNode.getHost())) {
                 isKnown = true;
                 break;
@@ -842,7 +847,7 @@ public class AdminService extends CompositeService implements
       }
     }
     try {
-      rmContext.getNodeLabelManager().replaceLabelsOnNode(
+      rm.getRMContext().getNodeLabelManager().replaceLabelsOnNode(
           request.getNodeToLabels());
       RMAuditLogger
           .logSuccess(user.getShortUserName(), operation, "AdminService");
@@ -879,7 +884,7 @@ public class AdminService extends CompositeService implements
 
     checkRMStatus(user.getShortUserName(), operation, msg);
 
-    Set<NodeId> decommissioningNodes = rmContext.getNodesListManager()
+    Set<NodeId> decommissioningNodes = rm.getRMContext().getNodesListManager()
         .checkForDecommissioningNodes();
     RMAuditLogger.logSuccess(user.getShortUserName(), operation,
             "AdminService");
@@ -915,6 +920,6 @@ public class AdminService extends CompositeService implements
         getConfiguration(new Configuration(false),
             YarnConfiguration.YARN_SITE_CONFIGURATION_FILE);
 
-    rmContext.getScheduler().setClusterMaxPriority(conf);
+    rm.getRMContext().getScheduler().setClusterMaxPriority(conf);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c602f05b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/CuratorBasedElectorService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/CuratorBasedElectorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/CuratorBasedElectorService.java
index bcdf48b..d7485f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/CuratorBasedElectorService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/CuratorBasedElectorService.java
@@ -45,14 +45,12 @@ public class CuratorBasedElectorService extends AbstractService
       LogFactory.getLog(CuratorBasedElectorService.class);
   private LeaderLatch leaderLatch;
   private CuratorFramework curator;
-  private RMContext rmContext;
   private String latchPath;
   private String rmId;
   private ResourceManager rm;
 
-  public CuratorBasedElectorService(RMContext rmContext, ResourceManager rm) {
+  public CuratorBasedElectorService(ResourceManager rm) {
     super(CuratorBasedElectorService.class.getName());
-    this.rmContext = rmContext;
     this.rm = rm;
   }
 
@@ -102,7 +100,8 @@ public class CuratorBasedElectorService extends AbstractService
   public void isLeader() {
     LOG.info(rmId + "is elected leader, transitioning to active");
     try {
-      rmContext.getRMAdminService().transitionToActive(
+      rm.getRMContext().getRMAdminService()
+          .transitionToActive(
           new HAServiceProtocol.StateChangeRequestInfo(
               HAServiceProtocol.RequestSource.REQUEST_BY_ZKFC));
     } catch (Exception e) {
@@ -123,7 +122,8 @@ public class CuratorBasedElectorService extends AbstractService
   public void notLeader() {
     LOG.info(rmId + " relinquish leadership");
     try {
-      rmContext.getRMAdminService().transitionToStandby(
+      rm.getRMContext().getRMAdminService()
+          .transitionToStandby(
           new HAServiceProtocol.StateChangeRequestInfo(
               HAServiceProtocol.RequestSource.REQUEST_BY_ZKFC));
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c602f05b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
index 0e305a9..4844eba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.monitor.RMAppLifetime
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.distributed.QueueLimitCalculator;
 import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
@@ -107,6 +108,7 @@ public class RMActiveServiceContext {
   private PlacementManager queuePlacementManager = null;
 
   private RMAppLifetimeMonitor rmAppLifetimeMonitor;
+  private QueueLimitCalculator queueLimitCalculator;
 
   public RMActiveServiceContext() {
     queuePlacementManager = new PlacementManager();
@@ -483,4 +485,17 @@ public class RMActiveServiceContext {
   public RMAppLifetimeMonitor getRMAppLifetimeMonitor() {
     return this.rmAppLifetimeMonitor;
   }
+
+  @Private
+  @Unstable
+  public QueueLimitCalculator getNodeManagerQueueLimitCalculator() {
+    return this.queueLimitCalculator;
+  }
+
+  @Private
+  @Unstable
+  public void setContainerQueueLimitCalculator(
+      QueueLimitCalculator limitCalculator) {
+    this.queueLimitCalculator = limitCalculator;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c602f05b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index fb160c4..ab3672e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -57,37 +57,39 @@ import org.apache.hadoop.yarn.util.Clock;
 
 import com.google.common.annotations.VisibleForTesting;
 
+/**
+ * RMContextImpl class holds two services context.
+ * <ul>
+ * <li>serviceContext : These services called as <b>Always On</b> services.
+ * Services that need to run always irrespective of the HA state of the RM.</li>
+ * <li>activeServiceCotext : Active services context. Services that need to run
+ * only on the Active RM.</li>
+ * </ul>
+ * <p>
+ * <b>Note:</b> If any new service to be added to context, add it to a right
+ * context as per above description.
+ */
 public class RMContextImpl implements RMContext {
 
-  private Dispatcher rmDispatcher;
-
-  private boolean isHAEnabled;
-
-  private HAServiceState haServiceState =
-      HAServiceProtocol.HAServiceState.INITIALIZING;
-
-  private AdminService adminService;
-
-  private ConfigurationProvider configurationProvider;
+  /**
+   * RM service contexts which runs through out RM life span. These are created
+   * once during start of RM.
+   */
+  private RMServiceContext serviceContext;
 
+  /**
+   * RM Active service context. This will be recreated for every transition from
+   * ACTIVE->STANDBY.
+   */
   private RMActiveServiceContext activeServiceContext;
 
-  private Configuration yarnConfiguration;
-
-  private RMApplicationHistoryWriter rmApplicationHistoryWriter;
-  private SystemMetricsPublisher systemMetricsPublisher;
-  private EmbeddedElector elector;
-
-  private QueueLimitCalculator queueLimitCalculator;
-
-  private final Object haServiceStateLock = new Object();
-
-  private ResourceManager resourceManager;
   /**
    * Default constructor. To be used in conjunction with setter methods for
    * individual fields.
    */
   public RMContextImpl() {
+    this.serviceContext = new RMServiceContext();
+    this.activeServiceContext = new RMActiveServiceContext();
   }
 
   @VisibleForTesting
@@ -138,19 +140,143 @@ public class RMContextImpl implements RMContext {
       clientToAMTokenSecretManager, null);
   }
 
+  /**
+   * RM service contexts which runs through out JVM life span. These are created
+   * once during start of RM.
+   * @return serviceContext of RM
+   */
+  @Private
+  @Unstable
+  public RMServiceContext getServiceContext() {
+    return serviceContext;
+  }
+
+  /**
+   * <b>Note:</b> setting service context clears all services embedded with it.
+   * @param context rm service context
+   */
+  @Private
+  @Unstable
+  public void setServiceContext(RMServiceContext context) {
+    this.serviceContext = context;
+  }
+
   @Override
-  public Dispatcher getDispatcher() {
-    return this.rmDispatcher;
+  public ResourceManager getResourceManager() {
+    return serviceContext.getResourceManager();
+  }
+
+  public void setResourceManager(ResourceManager rm) {
+    serviceContext.setResourceManager(rm);
+  }
+
+  @Override
+  public EmbeddedElector getLeaderElectorService() {
+    return serviceContext.getLeaderElectorService();
   }
 
   @Override
   public void setLeaderElectorService(EmbeddedElector elector) {
-    this.elector = elector;
+    serviceContext.setLeaderElectorService(elector);
   }
 
   @Override
-  public EmbeddedElector getLeaderElectorService() {
-    return this.elector;
+  public Dispatcher getDispatcher() {
+    return serviceContext.getDispatcher();
+  }
+
+  void setDispatcher(Dispatcher dispatcher) {
+    serviceContext.setDispatcher(dispatcher);
+  }
+
+  @Override
+  public AdminService getRMAdminService() {
+    return serviceContext.getRMAdminService();
+  }
+
+  void setRMAdminService(AdminService adminService) {
+    serviceContext.setRMAdminService(adminService);
+  }
+
+  @Override
+  public boolean isHAEnabled() {
+    return serviceContext.isHAEnabled();
+  }
+
+  void setHAEnabled(boolean isHAEnabled) {
+    serviceContext.setHAEnabled(isHAEnabled);
+  }
+
+  @Override
+  public HAServiceState getHAServiceState() {
+    return serviceContext.getHAServiceState();
+  }
+
+  void setHAServiceState(HAServiceState serviceState) {
+    serviceContext.setHAServiceState(serviceState);
+  }
+
+  @Override
+  public RMApplicationHistoryWriter getRMApplicationHistoryWriter() {
+    return serviceContext.getRMApplicationHistoryWriter();
+  }
+
+  @Override
+  public void setRMApplicationHistoryWriter(
+      RMApplicationHistoryWriter rmApplicationHistoryWriter) {
+    serviceContext.setRMApplicationHistoryWriter(rmApplicationHistoryWriter);
+  }
+
+  @Override
+  public SystemMetricsPublisher getSystemMetricsPublisher() {
+    return serviceContext.getSystemMetricsPublisher();
+  }
+
+  @Override
+  public void setSystemMetricsPublisher(
+      SystemMetricsPublisher metricsPublisher) {
+    serviceContext.setSystemMetricsPublisher(metricsPublisher);
+  }
+
+  @Override
+  public ConfigurationProvider getConfigurationProvider() {
+    return serviceContext.getConfigurationProvider();
+  }
+
+  public void setConfigurationProvider(
+      ConfigurationProvider configurationProvider) {
+    serviceContext.setConfigurationProvider(configurationProvider);
+  }
+
+  @Override
+  public Configuration getYarnConfiguration() {
+    return serviceContext.getYarnConfiguration();
+  }
+
+  public void setYarnConfiguration(Configuration yarnConfiguration) {
+    serviceContext.setYarnConfiguration(yarnConfiguration);
+  }
+
+  public String getHAZookeeperConnectionState() {
+    return serviceContext.getHAZookeeperConnectionState();
+  }
+
+  // ==========================================================================
+  /**
+   * RM Active service context. This will be recreated for every transition from
+   * ACTIVE to STANDBY.
+   * @return activeServiceContext of active services
+   */
+  @Private
+  @Unstable
+  public RMActiveServiceContext getActiveServiceContext() {
+    return activeServiceContext;
+  }
+
+  @Private
+  @Unstable
+  void setActiveServiceContext(RMActiveServiceContext activeServiceContext) {
+    this.activeServiceContext = activeServiceContext;
   }
 
   @Override
@@ -228,11 +354,6 @@ public class RMContextImpl implements RMContext {
     return activeServiceContext.getClientToAMTokenSecretManager();
   }
 
-  @Override
-  public AdminService getRMAdminService() {
-    return this.adminService;
-  }
-
   @VisibleForTesting
   public void setStateStore(RMStateStore store) {
     activeServiceContext.setStateStore(store);
@@ -253,24 +374,6 @@ public class RMContextImpl implements RMContext {
     return activeServiceContext.getResourceTrackerService();
   }
 
-  void setHAEnabled(boolean isHAEnabled) {
-    this.isHAEnabled = isHAEnabled;
-  }
-
-  void setHAServiceState(HAServiceState serviceState) {
-    synchronized (haServiceStateLock) {
-      this.haServiceState = serviceState;
-    }
-  }
-
-  void setDispatcher(Dispatcher dispatcher) {
-    this.rmDispatcher = dispatcher;
-  }
-
-  void setRMAdminService(AdminService adminService) {
-    this.adminService = adminService;
-  }
-
   @Override
   public void setClientRMService(ClientRMService clientRMService) {
     activeServiceContext.setClientRMService(clientRMService);
@@ -348,18 +451,6 @@ public class RMContextImpl implements RMContext {
     activeServiceContext.setResourceTrackerService(resourceTrackerService);
   }
 
-  @Override
-  public boolean isHAEnabled() {
-    return isHAEnabled;
-  }
-
-  @Override
-  public HAServiceState getHAServiceState() {
-    synchronized (haServiceStateLock) {
-      return haServiceState;
-    }
-  }
-
   public void setWorkPreservingRecoveryEnabled(boolean enabled) {
     activeServiceContext.setWorkPreservingRecoveryEnabled(enabled);
   }
@@ -370,11 +461,6 @@ public class RMContextImpl implements RMContext {
   }
 
   @Override
-  public RMApplicationHistoryWriter getRMApplicationHistoryWriter() {
-    return this.rmApplicationHistoryWriter;
-  }
-
-  @Override
   public void setRMTimelineCollectorManager(
       RMTimelineCollectorManager timelineCollectorManager) {
     activeServiceContext.setRMTimelineCollectorManager(
@@ -382,39 +468,6 @@ public class RMContextImpl implements RMContext {
   }
 
   @Override
-  public RMTimelineCollectorManager getRMTimelineCollectorManager() {
-    return activeServiceContext.getRMTimelineCollectorManager();
-  }
-
-  @Override
-  public void setSystemMetricsPublisher(
-      SystemMetricsPublisher metricsPublisher) {
-    this.systemMetricsPublisher = metricsPublisher;
-  }
-
-  @Override
-  public SystemMetricsPublisher getSystemMetricsPublisher() {
-    return this.systemMetricsPublisher;
-  }
-
-  @Override
-  public void setRMApplicationHistoryWriter(
-      RMApplicationHistoryWriter rmApplicationHistoryWriter) {
-    this.rmApplicationHistoryWriter = rmApplicationHistoryWriter;
-
-  }
-
-  @Override
-  public ConfigurationProvider getConfigurationProvider() {
-    return this.configurationProvider;
-  }
-
-  public void setConfigurationProvider(
-      ConfigurationProvider configurationProvider) {
-    this.configurationProvider = configurationProvider;
-  }
-
-  @Override
   public long getEpoch() {
     return activeServiceContext.getEpoch();
   }
@@ -463,27 +516,6 @@ public class RMContextImpl implements RMContext {
     return activeServiceContext.getSystemCredentialsForApps();
   }
 
-  @Private
-  @Unstable
-  public RMActiveServiceContext getActiveServiceContext() {
-    return activeServiceContext;
-  }
-
-  @Private
-  @Unstable
-  void setActiveServiceContext(RMActiveServiceContext activeServiceContext) {
-    this.activeServiceContext = activeServiceContext;
-  }
-
-  @Override
-  public Configuration getYarnConfiguration() {
-    return this.yarnConfiguration;
-  }
-
-  public void setYarnConfiguration(Configuration yarnConfiguration) {
-    this.yarnConfiguration=yarnConfiguration;
-  }
-
   @Override
   public PlacementManager getQueuePlacementManager() {
     return this.activeServiceContext.getQueuePlacementManager();
@@ -496,12 +528,12 @@ public class RMContextImpl implements RMContext {
 
   @Override
   public QueueLimitCalculator getNodeManagerQueueLimitCalculator() {
-    return this.queueLimitCalculator;
+    return activeServiceContext.getNodeManagerQueueLimitCalculator();
   }
 
   public void setContainerQueueLimitCalculator(
       QueueLimitCalculator limitCalculator) {
-    this.queueLimitCalculator = limitCalculator;
+    activeServiceContext.setContainerQueueLimitCalculator(limitCalculator);
   }
 
   @Override
@@ -515,21 +547,5 @@ public class RMContextImpl implements RMContext {
     return this.activeServiceContext.getRMAppLifetimeMonitor();
   }
 
-  public String getHAZookeeperConnectionState() {
-    if (elector == null) {
-      return "Could not find leader elector. Verify both HA and automatic " +
-          "failover are enabled.";
-    } else {
-      return elector.getZookeeperConnectionState();
-    }
-  }
-
-  @Override
-  public ResourceManager getResourceManager() {
-    return resourceManager;
-  }
-
-  public void setResourceManager(ResourceManager rm) {
-    this.resourceManager = rm;
-  }
+  // Note: Read java doc before adding any services over here.
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c602f05b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java
new file mode 100644
index 0000000..fe34d63
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java
@@ -0,0 +1,151 @@
+/**
+ * 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.resourcemanager;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.yarn.conf.ConfigurationProvider;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
+import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
+
+/**
+ * RMServiceContext class maintains "Always On" services. Services that need to
+ * run always irrespective of the HA state of the RM. This is created during
+ * initialization of RMContextImpl.
+ * <p>
+ * <b>Note:</b> If any services to be added in this class, make sure service
+ * will be running always irrespective of the HA state of the RM
+ */
+@Private
+@Unstable
+public class RMServiceContext {
+
+  private Dispatcher rmDispatcher;
+  private boolean isHAEnabled;
+  private HAServiceState haServiceState =
+      HAServiceProtocol.HAServiceState.INITIALIZING;
+  private AdminService adminService;
+  private ConfigurationProvider configurationProvider;
+  private Configuration yarnConfiguration;
+  private RMApplicationHistoryWriter rmApplicationHistoryWriter;
+  private SystemMetricsPublisher systemMetricsPublisher;
+  private EmbeddedElector elector;
+  private final Object haServiceStateLock = new Object();
+  private ResourceManager resourceManager;
+
+  public ResourceManager getResourceManager() {
+    return resourceManager;
+  }
+
+  public void setResourceManager(ResourceManager rm) {
+    this.resourceManager = rm;
+  }
+
+  public ConfigurationProvider getConfigurationProvider() {
+    return this.configurationProvider;
+  }
+
+  public void setConfigurationProvider(
+      ConfigurationProvider configurationProvider) {
+    this.configurationProvider = configurationProvider;
+  }
+
+  public Dispatcher getDispatcher() {
+    return this.rmDispatcher;
+  }
+
+  void setDispatcher(Dispatcher dispatcher) {
+    this.rmDispatcher = dispatcher;
+  }
+
+  public EmbeddedElector getLeaderElectorService() {
+    return this.elector;
+  }
+
+  public void setLeaderElectorService(EmbeddedElector embeddedElector) {
+    this.elector = embeddedElector;
+  }
+
+  public AdminService getRMAdminService() {
+    return this.adminService;
+  }
+
+  void setRMAdminService(AdminService service) {
+    this.adminService = service;
+  }
+
+  void setHAEnabled(boolean rmHAEnabled) {
+    this.isHAEnabled = rmHAEnabled;
+  }
+
+  public boolean isHAEnabled() {
+    return isHAEnabled;
+  }
+
+  public HAServiceState getHAServiceState() {
+    synchronized (haServiceStateLock) {
+      return haServiceState;
+    }
+  }
+
+  void setHAServiceState(HAServiceState serviceState) {
+    synchronized (haServiceStateLock) {
+      this.haServiceState = serviceState;
+    }
+  }
+
+  public RMApplicationHistoryWriter getRMApplicationHistoryWriter() {
+    return this.rmApplicationHistoryWriter;
+  }
+
+  public void setRMApplicationHistoryWriter(
+      RMApplicationHistoryWriter applicationHistoryWriter) {
+    this.rmApplicationHistoryWriter = applicationHistoryWriter;
+  }
+
+  public void setSystemMetricsPublisher(
+      SystemMetricsPublisher metricsPublisher) {
+    this.systemMetricsPublisher = metricsPublisher;
+  }
+
+  public SystemMetricsPublisher getSystemMetricsPublisher() {
+    return this.systemMetricsPublisher;
+  }
+
+  public Configuration getYarnConfiguration() {
+    return this.yarnConfiguration;
+  }
+
+  public void setYarnConfiguration(Configuration yarnConfiguration) {
+    this.yarnConfiguration = yarnConfiguration;
+  }
+
+  public String getHAZookeeperConnectionState() {
+    if (elector == null) {
+      return "Could not find leader elector. Verify both HA and automatic "
+          + "failover are enabled.";
+    } else {
+      return elector.getZookeeperConnectionState();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c602f05b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 1adef33..d8de137 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -347,9 +347,9 @@ public class ResourceManager extends CompositeService implements Recoverable {
             YarnConfiguration.DEFAULT_CURATOR_LEADER_ELECTOR_ENABLED);
     if (curatorEnabled) {
       this.curator = createAndStartCurator(conf);
-      elector = new CuratorBasedElectorService(rmContext, this);
+      elector = new CuratorBasedElectorService(this);
     } else {
-      elector = new ActiveStandbyElectorBasedElectorService(rmContext);
+      elector = new ActiveStandbyElectorBasedElectorService(this);
     }
     return elector;
   }
@@ -562,7 +562,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
     private ApplicationMasterLauncher applicationMasterLauncher;
     private ContainerAllocationExpirer containerAllocationExpirer;
     private ResourceManager rm;
-    private RMActiveServiceContext activeServiceContext;
     private boolean fromActive = false;
     private StandByTransitionRunnable standByTransitionRunnable;
 
@@ -575,9 +574,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
     protected void serviceInit(Configuration configuration) throws Exception {
       standByTransitionRunnable = new StandByTransitionRunnable();
 
-      activeServiceContext = new RMActiveServiceContext();
-      rmContext.setActiveServiceContext(activeServiceContext);
-
       conf.setBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY, true);
       rmSecretManagerService = createRMSecretManagerService();
       addService(rmSecretManagerService);
@@ -1135,7 +1131,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
     ClusterMetrics.destroy();
     QueueMetrics.clearQueueMetrics();
     if (initialize) {
-      resetDispatcher();
+      resetRMContext();
       createAndInitActiveServices(true);
     }
   }
@@ -1280,7 +1276,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
   }
 
   protected AdminService createAdminService() {
-    return new AdminService(this, rmContext);
+    return new AdminService(this);
   }
 
   protected RMSecretManagerService createRMSecretManagerService() {
@@ -1403,16 +1399,24 @@ public class ResourceManager extends CompositeService implements Recoverable {
     return dispatcher;
   }
 
-  private void resetDispatcher() {
+  private void resetRMContext() {
+    RMContextImpl rmContextImpl = new RMContextImpl();
+    // transfer service context to new RM service Context
+    rmContextImpl.setServiceContext(rmContext.getServiceContext());
+
+    // reset dispatcher
     Dispatcher dispatcher = setupDispatcher();
-    ((Service)dispatcher).init(this.conf);
-    ((Service)dispatcher).start();
-    removeService((Service)rmDispatcher);
+    ((Service) dispatcher).init(this.conf);
+    ((Service) dispatcher).start();
+    removeService((Service) rmDispatcher);
     // Need to stop previous rmDispatcher before assigning new dispatcher
     // otherwise causes "AsyncDispatcher event handler" thread leak
     ((Service) rmDispatcher).stop();
     rmDispatcher = dispatcher;
     addIfService(rmDispatcher);
+    rmContextImpl.setDispatcher(dispatcher);
+
+    rmContext = rmContextImpl;
     rmContext.setDispatcher(rmDispatcher);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c602f05b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index aca2fc5..b0ad977 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -1042,7 +1042,7 @@ public class MockRM extends ResourceManager {
 
   @Override
   protected AdminService createAdminService() {
-    return new AdminService(this, getRMContext()) {
+    return new AdminService(this) {
       @Override
       protected void startServer() {
         // override to not start rpc handler

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c602f05b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
index c4fcc5d..47d18f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
@@ -123,13 +123,15 @@ public class TestRMEmbeddedElector extends ClientBaseWithFixes {
       throws IOException, InterruptedException {
     AdminService as = mock(AdminService.class);
     RMContext rc = mock(RMContext.class);
+    ResourceManager rm = mock(ResourceManager.class);
     Configuration myConf = new Configuration(conf);
 
     myConf.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, 50);
+    when(rm.getRMContext()).thenReturn(rc);
     when(rc.getRMAdminService()).thenReturn(as);
 
-    ActiveStandbyElectorBasedElectorService
-        ees = new ActiveStandbyElectorBasedElectorService(rc);
+    ActiveStandbyElectorBasedElectorService ees =
+        new ActiveStandbyElectorBasedElectorService(rm);
     ees.init(myConf);
 
     ees.enterNeutralMode();
@@ -291,7 +293,7 @@ public class TestRMEmbeddedElector extends ClientBaseWithFixes {
 
     @Override
     protected EmbeddedElector createEmbeddedElector() {
-      return new ActiveStandbyElectorBasedElectorService(getRMContext()) {
+      return new ActiveStandbyElectorBasedElectorService(this) {
         @Override
         public void becomeActive() throws
             ServiceFailedException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c602f05b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
index 0efda9e..a558dd5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMHA.java
@@ -70,6 +70,7 @@ public class TestRMHA {
   private Log LOG = LogFactory.getLog(TestRMHA.class);
   private Configuration configuration;
   private MockRM rm = null;
+  private MockNM nm = null;
   private RMApp app = null;
   private RMAppAttempt attempt = null;
   private static final String STATE_ERR =
@@ -134,7 +135,7 @@ public class TestRMHA {
 
     try {
       rm.getNewAppId();
-      rm.registerNode("127.0.0.1:1", 2048);
+      nm = rm.registerNode("127.0.0.1:1", 2048);
       app = rm.submitApp(1024);
       attempt = app.getCurrentAppAttempt();
       rm.waitForState(attempt.getAppAttemptId(), RMAppAttemptState.SCHEDULED);
@@ -549,6 +550,17 @@ public class TestRMHA {
     verifyClusterMetrics(1, 1, 1, 1, 2048, 1);
     assertEquals(1, rm.getRMContext().getRMNodes().size());
     assertEquals(1, rm.getRMContext().getRMApps().size());
+    Assert.assertNotNull("Node not registered", nm);
+
+    rm.adminService.transitionToStandby(requestInfo);
+    checkMonitorHealth();
+    checkStandbyRMFunctionality();
+    // race condition causes to register/node heartbeat node even after service
+    // is stopping/stopped. New RMContext is being created on every transition
+    // to standby, so metrics should be 0 which indicates new context reference
+    // has taken.
+    nm.registerNode();
+    verifyClusterMetrics(0, 0, 0, 0, 0, 0);
 
     // 3. Create new RM
     rm = new MockRM(conf, memStore) {
@@ -590,7 +602,7 @@ public class TestRMHA {
     rm = new MockRM(configuration) {
       @Override
       protected AdminService createAdminService() {
-        return new AdminService(this, getRMContext()) {
+        return new AdminService(this) {
           int counter = 0;
           @Override
           protected void setConfig(Configuration conf) {


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


[50/50] [abbrv] hadoop git commit: Addendum for YARN-6102.

Posted by va...@apache.org.
Addendum for YARN-6102.


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

Branch: refs/heads/YARN-5355_branch2
Commit: 6d72ddab0a3b564e2396ec5101463b5794f1e148
Parents: 4811d48
Author: Varun Saxena <va...@apache.org>
Authored: Wed Aug 2 00:02:18 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Wed Aug 2 00:02:18 2017 +0530

----------------------------------------------------------------------
 .../resourcemanager/RMActiveServiceContext.java | 21 +++---------------
 .../server/resourcemanager/RMContextImpl.java   | 23 ++++++++++----------
 .../resourcemanager/RMServiceContext.java       | 11 ++++++++++
 3 files changed, 25 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d72ddab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
index 4844eba..9dc5945 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
@@ -49,14 +49,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRen
 import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
-import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
 
 /**
- * The RMActiveServiceContext is the class that maintains all the
- * RMActiveService contexts.This is expected to be used only by ResourceManager
- * and RMContext.
+ * The RMActiveServiceContext is the class that maintains <b>Active</b> service
+ * context. Services that need to run only on the Active RM. This is expected to
+ * be used only by RMContext.
  */
 @Private
 @Unstable
@@ -95,7 +94,6 @@ public class RMActiveServiceContext {
   private NodesListManager nodesListManager;
   private ResourceTrackerService resourceTrackerService;
   private ApplicationMasterService applicationMasterService;
-  private RMTimelineCollectorManager timelineCollectorManager;
 
   private RMNodeLabelsManager nodeLabelManager;
   private RMDelegatedNodeLabelsUpdater rmDelegatedNodeLabelsUpdater;
@@ -376,19 +374,6 @@ public class RMActiveServiceContext {
 
   @Private
   @Unstable
-  public RMTimelineCollectorManager getRMTimelineCollectorManager() {
-    return timelineCollectorManager;
-  }
-
-  @Private
-  @Unstable
-  public void setRMTimelineCollectorManager(
-      RMTimelineCollectorManager collectorManager) {
-    this.timelineCollectorManager = collectorManager;
-  }
-
-  @Private
-  @Unstable
   public long getEpoch() {
     return this.epoch;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d72ddab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index c27e1bf..a960dc9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -238,6 +238,17 @@ public class RMContextImpl implements RMContext {
   }
 
   @Override
+  public RMTimelineCollectorManager getRMTimelineCollectorManager() {
+    return serviceContext.getRMTimelineCollectorManager();
+  }
+
+  @Override
+  public void setRMTimelineCollectorManager(
+      RMTimelineCollectorManager timelineCollectorManager) {
+    serviceContext.setRMTimelineCollectorManager(timelineCollectorManager);
+  }
+
+  @Override
   public ConfigurationProvider getConfigurationProvider() {
     return serviceContext.getConfigurationProvider();
   }
@@ -460,18 +471,6 @@ public class RMContextImpl implements RMContext {
   }
 
   @Override
-  public void setRMTimelineCollectorManager(
-      RMTimelineCollectorManager timelineCollectorManager) {
-    activeServiceContext.setRMTimelineCollectorManager(
-        timelineCollectorManager);
-  }
-
-  @Override
-  public RMTimelineCollectorManager getRMTimelineCollectorManager() {
-    return activeServiceContext.getRMTimelineCollectorManager();
-  }
-
-  @Override
   public long getEpoch() {
     return activeServiceContext.getEpoch();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6d72ddab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java
index fe34d63..45c6166 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServiceContext.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.conf.ConfigurationProvider;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
+import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
 
 /**
  * RMServiceContext class maintains "Always On" services. Services that need to
@@ -52,6 +53,7 @@ public class RMServiceContext {
   private EmbeddedElector elector;
   private final Object haServiceStateLock = new Object();
   private ResourceManager resourceManager;
+  private RMTimelineCollectorManager timelineCollectorManager;
 
   public ResourceManager getResourceManager() {
     return resourceManager;
@@ -140,6 +142,15 @@ public class RMServiceContext {
     this.yarnConfiguration = yarnConfiguration;
   }
 
+  public RMTimelineCollectorManager getRMTimelineCollectorManager() {
+    return timelineCollectorManager;
+  }
+
+  public void setRMTimelineCollectorManager(
+      RMTimelineCollectorManager collectorManager) {
+    this.timelineCollectorManager = collectorManager;
+  }
+
   public String getHAZookeeperConnectionState() {
     if (elector == null) {
       return "Could not find leader elector. Verify both HA and automatic "


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