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 gt...@apache.org on 2016/01/20 10:13:18 UTC

[01/50] [abbrv] hadoop git commit: YARN-3908. Fixed bugs in HBaseTimelineWriterImpl. Contributed by Vrushali C and Sangjin Lee. [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/feature-YARN-2928 2dce9faa7 -> cab8a1d1e (forced update)


YARN-3908. Fixed bugs in HBaseTimelineWriterImpl. Contributed by Vrushali C and Sangjin Lee.

(cherry picked from commit df0ec473a84871b0effd7ca6faac776210d7df09)


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

Branch: refs/heads/feature-YARN-2928
Commit: 5f00adf3df820325511e45595e62e5024877579d
Parents: 7f5b3b9
Author: Zhijie Shen <zj...@apache.org>
Authored: Mon Jul 27 15:50:28 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:36:53 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../records/timelineservice/TimelineEvent.java  |  4 +-
 .../storage/HBaseTimelineWriterImpl.java        | 18 ++++++-
 .../storage/common/ColumnHelper.java            | 21 ++++----
 .../storage/common/ColumnPrefix.java            |  7 +--
 .../storage/common/Separator.java               |  7 +++
 .../storage/entity/EntityColumnPrefix.java      | 15 ++++--
 .../storage/entity/EntityTable.java             |  6 ++-
 .../storage/TestHBaseTimelineWriterImpl.java    | 56 ++++++++++++++++++--
 9 files changed, 111 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f00adf3/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index fc381d6..79c061f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -115,6 +115,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3792. Test case failures in TestDistributedShell and some issue fixes
     related to ATSV2 (Naganarasimha G R via sjlee)
 
+    YARN-3908. Fixed bugs in HBaseTimelineWriterImpl. (Vrushali C and Sangjin
+    Lee via zjshen)
+
 Trunk - Unreleased
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f00adf3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEvent.java
index 1dbf7e5..a563658 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEvent.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEvent.java
@@ -33,6 +33,8 @@ import java.util.Map;
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 public class TimelineEvent implements Comparable<TimelineEvent> {
+  public static final long INVALID_TIMESTAMP = 0L;
+
   private String id;
   private HashMap<String, Object> info = new HashMap<>();
   private long timestamp;
@@ -83,7 +85,7 @@ public class TimelineEvent implements Comparable<TimelineEvent> {
   }
 
   public boolean isValid() {
-    return (id != null && timestamp != 0L);
+    return (id != null && timestamp != INVALID_TIMESTAMP);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f00adf3/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 876ad6a..cd2e76e 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
@@ -141,6 +141,13 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     EntityColumn.MODIFIED_TIME.store(rowKey, entityTable, null,
         te.getModifiedTime());
     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, entry.getKey(),
+            null, entry.getValue());
+      }
+    }
   }
 
   /**
@@ -186,6 +193,13 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
         if (event != null) {
           String eventId = event.getId();
           if (eventId != null) {
+            long eventTimestamp = event.getTimestamp();
+            // if the timestamp is not set, use the current timestamp
+            if (eventTimestamp == TimelineEvent.INVALID_TIMESTAMP) {
+              LOG.warn("timestamp is not set for event " + eventId +
+                  "! Using the current timestamp");
+              eventTimestamp = System.currentTimeMillis();
+            }
             Map<String, Object> eventInfo = event.getInfo();
             if (eventInfo != null) {
               for (Map.Entry<String, Object> info : eventInfo.entrySet()) {
@@ -198,8 +212,8 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
                 // convert back to string to avoid additional API on store.
                 String compoundColumnQualifier =
                     Bytes.toString(compoundColumnQualifierBytes);
-                EntityColumnPrefix.METRIC.store(rowKey, entityTable,
-                    compoundColumnQualifier, null, info.getValue());
+                EntityColumnPrefix.EVENT.store(rowKey, entityTable,
+                    compoundColumnQualifier, eventTimestamp, info.getValue());
               } // for info: eventInfo
             }
           }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f00adf3/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 6a204dc..a902924 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
@@ -113,19 +113,22 @@ public class ColumnHelper<T> {
   }
 
   /**
-   * @param result from which to reads timeseries data
+   * @param result from which to reads data with timestamps
    * @param columnPrefixBytes optional prefix to limit columns. If null all
    *          columns are returned.
+   * @param <V> the type of the values. The values will be cast into that type.
    * @return the cell values at each respective time in for form
    *         {idA={timestamp1->value1}, idA={timestamp2->value2},
    *         idB={timestamp3->value3}, idC={timestamp1->value4}}
    * @throws IOException
    */
-  public NavigableMap<String, NavigableMap<Long, Number>> readTimeseriesResults(
-      Result result, byte[] columnPrefixBytes) throws IOException {
+  @SuppressWarnings("unchecked")
+  public <V> NavigableMap<String, NavigableMap<Long, V>>
+      readResultsWithTimestamps(Result result, byte[] columnPrefixBytes)
+          throws IOException {
 
-    NavigableMap<String, NavigableMap<Long, Number>> results =
-        new TreeMap<String, NavigableMap<Long, Number>>();
+    NavigableMap<String, NavigableMap<Long, V>> results =
+        new TreeMap<String, NavigableMap<Long, V>>();
 
     if (result != null) {
       NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> resultMap =
@@ -157,13 +160,13 @@ public class ColumnHelper<T> {
 
           // If this column has the prefix we want
           if (columnName != null) {
-            NavigableMap<Long, Number> cellResults =
-                new TreeMap<Long, Number>();
+            NavigableMap<Long, V> cellResults =
+                new TreeMap<Long, V>();
             NavigableMap<Long, byte[]> cells = entry.getValue();
             if (cells != null) {
               for (Entry<Long, byte[]> cell : cells.entrySet()) {
-                Number value =
-                    (Number) GenericObjectMapper.read(cell.getValue());
+                V value =
+                    (V) GenericObjectMapper.read(cell.getValue());
                 cellResults.put(cell.getKey(), value);
               }
             }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f00adf3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.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/ColumnPrefix.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/ColumnPrefix.java
index 2eedea0..671c824 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/ColumnPrefix.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/ColumnPrefix.java
@@ -72,12 +72,13 @@ public interface ColumnPrefix<T> {
   public Map<String, Object> readResults(Result result) throws IOException;
 
   /**
-   * @param result from which to reads timeseries data
+   * @param result from which to reads data with timestamps
+   * @param <V> the type of the values. The values will be cast into that type.
    * @return the cell values at each respective time in for form
    *         {idA={timestamp1->value1}, idA={timestamp2->value2},
    *         idB={timestamp3->value3}, idC={timestamp1->value4}}
    * @throws IOException
    */
-  public NavigableMap<String, NavigableMap<Long, Number>> readTimeseriesResults(
-      Result result) throws IOException;
+  public <V> NavigableMap<String, NavigableMap<Long, V>>
+      readResultsWithTimestamps(Result result) throws IOException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f00adf3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.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/Separator.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/Separator.java
index ee57890..3319419 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/Separator.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/Separator.java
@@ -90,6 +90,13 @@ public enum Separator {
   }
 
   /**
+   * @return the original value of the separator
+   */
+  public String getValue() {
+    return value;
+  }
+
+  /**
    * Used to make token safe to be used with this separator without collisions.
    *
    * @param token

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f00adf3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
index 4459868..8b7bc3e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
@@ -45,6 +45,11 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
   RELATES_TO(EntityColumnFamily.INFO, "r"),
 
   /**
+   * To store TimelineEntity info values.
+   */
+  INFO(EntityColumnFamily.INFO, "i"),
+
+  /**
    * Lifecycle events for an entity
    */
   EVENT(EntityColumnFamily.INFO, "e"),
@@ -92,7 +97,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
   /**
    * @return the column name value
    */
-  private String getColumnPrefix() {
+  public String getColumnPrefix() {
     return columnPrefix;
   }
 
@@ -150,11 +155,11 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    *
    * @see
    * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
-   * #readTimeseriesResults(org.apache.hadoop.hbase.client.Result)
+   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result)
    */
-  public NavigableMap<String, NavigableMap<Long, Number>> readTimeseriesResults(
-      Result result) throws IOException {
-    return column.readTimeseriesResults(result, columnPrefixBytes);
+  public <T> NavigableMap<String, NavigableMap<Long, T>>
+      readResultsWithTimestamps(Result result) throws IOException {
+    return column.readResultsWithTimestamps(result, columnPrefixBytes);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f00adf3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.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/entity/EntityTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
index 61f7c4c..2ae7d39 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
@@ -54,7 +54,10 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineEnti
  * |            | modified_time:          |              |              |
  * |            | 1392995081012           | metricId2:   |              |
  * |            |                         | metricValue1 |              |
- * |            | r!relatesToKey:         | @timestamp2  |              |
+ * |            | i!infoKey:              | @timestamp2  |              |
+ * |            | infoValue               |              |              |
+ * |            |                         |              |              |
+ * |            | r!relatesToKey:         |              |              |
  * |            | id3?id4?id5             |              |              |
  * |            |                         |              |              |
  * |            | s!isRelatedToKey        |              |              |
@@ -62,6 +65,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineEnti
  * |            |                         |              |              |
  * |            | e!eventId?eventInfoKey: |              |              |
  * |            | eventInfoValue          |              |              |
+ * |            | @timestamp              |              |              |
  * |            |                         |              |              |
  * |            | flowVersion:            |              |              |
  * |            | versionValue            |              |              |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f00adf3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.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/TestHBaseTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
index 6abf240..31cb5d2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
@@ -43,8 +43,10 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 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.timeline.GenericObjectMapper;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
@@ -84,6 +86,12 @@ public class TestHBaseTimelineWriterImpl {
     entity.setCreatedTime(cTime);
     entity.setModifiedTime(mTime);
 
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<String, Object>();
+    infoMap.put("infoMapKey1", "infoMapValue1");
+    infoMap.put("infoMapKey2", 10);
+    entity.addInfo(infoMap);
+
     // add the isRelatedToEntity info
     String key = "task";
     String value = "is_related_to_entity_id_here";
@@ -177,6 +185,14 @@ public class TestHBaseTimelineWriterImpl {
           Long mTime1 = val.longValue();
           assertEquals(mTime1, mTime);
 
+          Map<String, Object> infoColumns =
+              EntityColumnPrefix.INFO.readResults(result);
+          assertEquals(infoMap.size(), infoColumns.size());
+          for (String infoItem : infoMap.keySet()) {
+            assertEquals(infoMap.get(infoItem),
+                infoColumns.get(infoItem));
+          }
+
           // Remember isRelatedTo is of type Map<String, Set<String>>
           for (String isRelatedToKey : isRelatedTo.keySet()) {
             Object isRelatedToValue =
@@ -219,7 +235,7 @@ public class TestHBaseTimelineWriterImpl {
           }
 
           NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
-              EntityColumnPrefix.METRIC.readTimeseriesResults(result);
+              EntityColumnPrefix.METRIC.readResultsWithTimestamps(result);
 
           NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
           // We got metrics back
@@ -237,7 +253,7 @@ public class TestHBaseTimelineWriterImpl {
         }
       }
       assertEquals(1, rowCount);
-      assertEquals(15, colCount);
+      assertEquals(17, colCount);
 
     } finally {
       hbi.stop();
@@ -267,13 +283,18 @@ public class TestHBaseTimelineWriterImpl {
 
   private void testAdditionalEntity() throws IOException {
     TimelineEvent event = new TimelineEvent();
-    event.setId("foo_event_id");
-    event.setTimestamp(System.currentTimeMillis());
-    event.addInfo("foo_event", "test");
+    String eventId = "foo_event_id";
+    event.setId(eventId);
+    Long expTs = 1436512802000L;
+    event.setTimestamp(expTs);
+    String expKey = "foo_event";
+    Object expVal = "test";
+    event.addInfo(expKey, expVal);
 
     final TimelineEntity entity = new TimelineEntity();
     entity.setId("attempt_1329348432655_0001_m_000008_18");
     entity.setType("FOO_ATTEMPT");
+    entity.addEvent(event);
 
     TimelineEntities entities = new TimelineEntities();
     entities.addEntity(entity);
@@ -304,6 +325,31 @@ public class TestHBaseTimelineWriterImpl {
       for (Result result : scanner) {
         if (result != null && !result.isEmpty()) {
           rowCount++;
+
+          // check the row key
+          byte[] row1 = result.getRow();
+          assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
+              entity));
+
+          // check the events
+          NavigableMap<String, NavigableMap<Long, Object>> eventsResult =
+              EntityColumnPrefix.EVENT.readResultsWithTimestamps(result);
+          // there should be only one event
+          assertEquals(1, eventsResult.size());
+          // key name for the event
+          String valueKey = eventId + Separator.VALUES.getValue() + expKey;
+          for (Map.Entry<String, NavigableMap<Long, Object>> e :
+              eventsResult.entrySet()) {
+            // the value key must match
+            assertEquals(valueKey, e.getKey());
+            NavigableMap<Long, Object> value = e.getValue();
+            // there should be only one timestamp and value
+            assertEquals(1, value.size());
+            for (Map.Entry<Long, Object> e2: value.entrySet()) {
+              assertEquals(expTs, e2.getKey());
+              assertEquals(expVal, e2.getValue());
+            }
+          }
         }
       }
       assertEquals(1, rowCount);


[33/50] [abbrv] hadoop git commit: YARN-4221. Store user in app to flow table (Varun Saxena via sjlee)

Posted by gt...@apache.org.
YARN-4221. Store user in app to flow table (Varun Saxena via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: af14edb2e520c8f73837288a5254013bef69a503
Parents: 1c54f76
Author: Sangjin Lee <sj...@apache.org>
Authored: Fri Oct 23 22:07:00 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:59:39 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../reader/TimelineReaderWebServices.java       | 103 +++++++++----------
 .../storage/ApplicationEntityReader.java        |   5 +-
 .../storage/GenericEntityReader.java            |  11 +-
 .../storage/HBaseTimelineWriterImpl.java        |   8 +-
 .../storage/apptoflow/AppToFlowColumn.java      |   8 +-
 .../storage/apptoflow/AppToFlowTable.java       |   3 +
 ...stTimelineReaderWebServicesHBaseStorage.java |  98 +++++++++++++-----
 8 files changed, 147 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/af14edb2/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 5a91c8a..fdd0d83 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -138,6 +138,8 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4129. Refactor the SystemMetricPublisher in RM to better support
     newer events (Naganarasimha G R via sjlee)
 
+    YARN-4221. Store user in app to flow table (Varun Saxena via sjlee)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af14edb2/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 d82a402..d3ff8b7 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
@@ -265,11 +265,6 @@ public class TimelineReaderWebServices {
     return str == null ? null : str.trim();
   }
 
-  private static String parseUser(UserGroupInformation callerUGI, String user) {
-    return (callerUGI != null && (user == null || user.isEmpty()) ?
-        callerUGI.getUserName().trim() : parseStr(user));
-  }
-
   private static UserGroupInformation getUser(HttpServletRequest req) {
     String remoteUser = req.getRemoteUser();
     UserGroupInformation callerUGI = null;
@@ -389,7 +384,7 @@ public class TimelineReaderWebServices {
     Set<TimelineEntity> entities = null;
     try {
       entities = timelineReaderManager.getEntities(
-          parseUser(callerUGI, userId), parseStr(clusterId), parseStr(flowId),
+          parseStr(userId), parseStr(clusterId), parseStr(flowId),
           parseLongStr(flowRunId), parseStr(appId), parseStr(entityType),
           parseLongStr(limit), parseLongStr(createdTimeStart),
           parseLongStr(createdTimeEnd), parseLongStr(modifiedTimeStart),
@@ -463,7 +458,7 @@ public class TimelineReaderWebServices {
     TimelineEntity entity = null;
     try {
       entity = timelineReaderManager.getEntity(
-          parseUser(callerUGI, userId), parseStr(clusterId), parseStr(flowId),
+          parseStr(userId), parseStr(clusterId), parseStr(flowId),
           parseLongStr(flowRunId), parseStr(appId), parseStr(entityType),
           parseStr(entityId), parseFieldsStr(fields, COMMA_DELIMITER));
     } catch (Exception e) {
@@ -482,35 +477,35 @@ public class TimelineReaderWebServices {
   }
 
   /**
-   * Return a single flow run for the given cluster, flow id and run id.
+   * Return a single flow run for the given user, flow id and run id.
    * Cluster ID is not provided by client so default cluster ID has to be taken.
    */
   @GET
-  @Path("/flowrun/{flowid}/{flowrunid}/")
+  @Path("/flowrun/{userid}/{flowid}/{flowrunid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public TimelineEntity getFlowRun(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
+      @PathParam("userid") String userId,
       @PathParam("flowid") String flowId,
       @PathParam("flowrunid") String flowRunId,
-      @QueryParam("userid") String userId,
       @QueryParam("fields") String fields) {
-    return getFlowRun(req, res, null, flowId, flowRunId, userId, fields);
+    return getFlowRun(req, res, userId, null, flowId, flowRunId, fields);
   }
 
   /**
-   * Return a single flow run for the given cluster, flow id and run id.
+   * Return a single flow run for the given user, cluster, flow id and run id.
    */
   @GET
-  @Path("/flowrun/{clusterid}/{flowid}/{flowrunid}/")
+  @Path("/flowrun/{userid}/{clusterid}/{flowid}/{flowrunid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public TimelineEntity getFlowRun(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
+      @PathParam("userid") String userId,
       @PathParam("clusterid") String clusterId,
       @PathParam("flowid") String flowId,
       @PathParam("flowrunid") String flowRunId,
-      @QueryParam("userid") String userId,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -522,9 +517,8 @@ public class TimelineReaderWebServices {
     TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
     TimelineEntity entity = null;
     try {
-      entity = timelineReaderManager.getEntity(
-          parseUser(callerUGI, userId), parseStr(clusterId),
-          parseStr(flowId), parseLongStr(flowRunId), null,
+      entity = timelineReaderManager.getEntity(parseStr(userId),
+          parseStr(clusterId), parseStr(flowId), parseLongStr(flowRunId), null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null,
           parseFieldsStr(fields, COMMA_DELIMITER));
     } catch (Exception e) {
@@ -543,37 +537,37 @@ public class TimelineReaderWebServices {
   }
 
   /**
-   * Return a set of flows runs for the given flow id.
+   * Return a set of flows runs for the given user and flow id.
    * Cluster ID is not provided by client so default cluster ID has to be taken.
    */
   @GET
-  @Path("/flowruns/{flowid}/")
+  @Path("/flowruns/{userid}/{flowid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getFlowRuns(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
+      @PathParam("userid") String userId,
       @PathParam("flowid") String flowId,
-      @QueryParam("userid") String userId,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
       @QueryParam("fields") String fields) {
-    return getFlowRuns(req, res, null, flowId, userId, limit, createdTimeStart,
+    return getFlowRuns(req, res, userId, null, flowId, limit, createdTimeStart,
         createdTimeEnd, fields);
   }
 
   /**
-   * Return a set of flow runs for the given cluster and flow id.
+   * Return a set of flow runs for the given user, cluster and flow id.
    */
   @GET
-  @Path("/flowruns/{clusterid}/{flowid}/")
+  @Path("/flowruns/{userid}/{clusterid}/{flowid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getFlowRuns(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
+      @PathParam("userid") String userId,
       @PathParam("clusterid") String clusterId,
       @PathParam("flowid") String flowId,
-      @QueryParam("userid") String userId,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
@@ -589,11 +583,11 @@ public class TimelineReaderWebServices {
     Set<TimelineEntity> entities = null;
     try {
       entities = timelineReaderManager.getEntities(
-          parseUser(callerUGI, userId), parseStr(clusterId), parseStr(flowId),
-          null, null, TimelineEntityType.YARN_FLOW_RUN.toString(),
-          parseLongStr(limit), parseLongStr(createdTimeStart),
-          parseLongStr(createdTimeEnd), null, null, null, null, null, null,
-          null, null, parseFieldsStr(fields, COMMA_DELIMITER));
+          parseStr(userId), parseStr(clusterId), parseStr(flowId), null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), parseLongStr(limit),
+          parseLongStr(createdTimeStart), parseLongStr(createdTimeEnd), null,
+          null, null, null, null, null, null, null,
+          parseFieldsStr(fields, COMMA_DELIMITER));
     } catch (Exception e) {
       handleException(e, url, startTime, "createdTime start/end or limit");
     }
@@ -730,10 +724,9 @@ public class TimelineReaderWebServices {
     TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
     TimelineEntity entity = null;
     try {
-      entity = timelineReaderManager.getEntity(
-          parseUser(callerUGI, userId), parseStr(clusterId),
-          parseStr(flowId), parseLongStr(flowRunId), parseStr(appId),
-          TimelineEntityType.YARN_APPLICATION.toString(), null,
+      entity = timelineReaderManager.getEntity(parseStr(userId),
+          parseStr(clusterId), parseStr(flowId), parseLongStr(flowRunId),
+          parseStr(appId), TimelineEntityType.YARN_APPLICATION.toString(), null,
           parseFieldsStr(fields, COMMA_DELIMITER));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
@@ -750,20 +743,20 @@ public class TimelineReaderWebServices {
   }
 
   /**
-   * Return a list of apps for given flow id and flow run id. Cluster ID is not
-   * provided by client so default cluster ID has to be taken. If number of
-   * matching apps are more than the limit, most recent apps till the limit is
-   * reached, will be returned.
+   * Return a list of apps for given user, flow id and flow run id. Cluster ID
+   * is not provided by client so default cluster ID has to be taken. If number
+   * of matching apps are more than the limit, most recent apps till the limit
+   * is reached, will be returned.
    */
   @GET
-  @Path("/flowrunapps/{flowid}/{flowrunid}/")
+  @Path("/flowrunapps/{userid}/{flowid}/{flowrunid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getFlowRunApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
+      @PathParam("userid") String userId,
       @PathParam("flowid") String flowId,
       @PathParam("flowrunid") String flowRunId,
-      @QueryParam("userid") String userId,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
@@ -784,20 +777,20 @@ public class TimelineReaderWebServices {
   }
 
   /**
-   * Return a list of apps for a given cluster id, flow id and flow run id. If
-   * number of matching apps are more than the limit, most recent apps till the
-   * limit is reached, will be returned.
+   * Return a list of apps for a given user, cluster id, flow id and flow run
+   * id. If number of matching apps are more than the limit, most recent apps
+   * till the limit is reached, will be returned.
    */
   @GET
-  @Path("/flowrunapps/{clusterid}/{flowid}/{flowrunid}/")
+  @Path("/flowrunapps/{userid}/{clusterid}/{flowid}/{flowrunid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getFlowRunApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
+      @PathParam("userid") String userId,
       @PathParam("clusterid") String clusterId,
       @PathParam("flowid") String flowId,
       @PathParam("flowrunid") String flowRunId,
-      @QueryParam("userid") String userId,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
@@ -818,19 +811,19 @@ public class TimelineReaderWebServices {
   }
 
   /**
-   * Return a list of apps for given flow id. Cluster ID is not provided by
-   * client so default cluster ID has to be taken. If number of matching apps
-   * are more than the limit, most recent apps till the limit is reached, will
-   * be returned.
+   * Return a list of apps for given user and flow id. Cluster ID is not
+   * provided by client so default cluster ID has to be taken. If number of
+   * matching apps are more than the limit, most recent apps till the limit is
+   * reached, will be returned.
    */
   @GET
-  @Path("/flowapps/{flowid}/")
+  @Path("/flowapps/{userid}/{flowid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getFlowApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
+      @PathParam("userid") String userId,
       @PathParam("flowid") String flowId,
-      @QueryParam("userid") String userId,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
@@ -851,19 +844,19 @@ public class TimelineReaderWebServices {
   }
 
   /**
-   * Return a list of apps for a given cluster id and flow id. If number of
-   * matching apps are more than the limit, most recent apps till the limit is
-   * reached, will be returned.
+   * Return a list of apps for a given user, cluster id and flow id. If number
+   * of matching apps are more than the limit, most recent apps till the limit
+   * is reached, will be returned.
    */
   @GET
-  @Path("/flowapps/{clusterid}/{flowid}/")
+  @Path("/flowapps/{userid}/{clusterid}/{flowid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getFlowApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
+      @PathParam("userid") String userId,
       @PathParam("clusterid") String clusterId,
       @PathParam("flowid") String flowId,
-      @QueryParam("userid") String userId,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af14edb2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/ApplicationEntityReader.java
index 6d1a2ff..8324afd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/ApplicationEntityReader.java
@@ -90,12 +90,12 @@ class ApplicationEntityReader extends GenericEntityReader {
 
   @Override
   protected void validateParams() {
-    Preconditions.checkNotNull(userId, "userId shouldn't be null");
     Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
     Preconditions.checkNotNull(entityType, "entityType shouldn't be null");
     if (singleEntityRead) {
       Preconditions.checkNotNull(appId, "appId shouldn't be null");
     } else {
+      Preconditions.checkNotNull(userId, "userId shouldn't be null");
       Preconditions.checkNotNull(flowId, "flowId shouldn't be null");
     }
   }
@@ -104,11 +104,12 @@ class ApplicationEntityReader extends GenericEntityReader {
   protected void augmentParams(Configuration hbaseConf, Connection conn)
       throws IOException {
     if (singleEntityRead) {
-      if (flowId == null || flowRunId == null) {
+      if (flowId == null || flowRunId == null || userId == null) {
         FlowContext context =
             lookupFlowContext(clusterId, appId, hbaseConf, conn);
         flowId = context.flowId;
         flowRunId = context.flowRunId;
+        userId = context.userId;
       }
     }
     if (fieldsToRetrieve == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af14edb2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.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/GenericEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
index bbca209..04fc8ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
@@ -100,6 +100,7 @@ class GenericEntityReader extends TimelineEntityReader {
     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());
     } else {
@@ -110,9 +111,11 @@ class GenericEntityReader extends TimelineEntityReader {
   }
 
   protected static class FlowContext {
+    protected final String userId;
     protected final String flowId;
     protected final Long flowRunId;
-    public FlowContext(String flowId, Long flowRunId) {
+    public FlowContext(String user, String flowId, Long flowRunId) {
+      this.userId = user;
       this.flowId = flowId;
       this.flowRunId = flowRunId;
     }
@@ -120,7 +123,6 @@ class GenericEntityReader extends TimelineEntityReader {
 
   @Override
   protected void validateParams() {
-    Preconditions.checkNotNull(userId, "userId shouldn't be null");
     Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
     Preconditions.checkNotNull(appId, "appId shouldn't be null");
     Preconditions.checkNotNull(entityType, "entityType shouldn't be null");
@@ -132,12 +134,13 @@ class GenericEntityReader extends TimelineEntityReader {
   @Override
   protected void augmentParams(Configuration hbaseConf, Connection conn)
       throws IOException {
-    // In reality both should be null or neither should be null
-    if (flowId == null || flowRunId == null) {
+    // In reality all three should be null or neither should be null
+    if (flowId == null || flowRunId == null || userId == null) {
       FlowContext context =
           lookupFlowContext(clusterId, appId, hbaseConf, conn);
       flowId = context.flowId;
       flowRunId = context.flowRunId;
+      userId = context.userId;
     }
     if (fieldsToRetrieve == null) {
       fieldsToRetrieve = EnumSet.noneOf(Field.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af14edb2/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 3649865..a57be55 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
@@ -162,8 +162,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       String flowName, String flowVersion, long flowRunId, String appId,
       TimelineEntity te) throws IOException {
     // store in App to flow table
-    storeInAppToFlowTable(clusterId, userId, flowName, flowVersion, flowRunId,
-        appId, te);
+    storeInAppToFlowTable(clusterId, userId, flowName, flowRunId, appId, te);
     // store in flow run table
     storeAppCreatedInFlowRunTable(clusterId, userId, flowName, flowVersion,
         flowRunId, appId, te);
@@ -200,11 +199,12 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   }
 
   private void storeInAppToFlowTable(String clusterId, String userId,
-      String flowName, String flowVersion, long flowRunId, String appId,
-      TimelineEntity te) throws IOException {
+      String flowName, long flowRunId, String appId, TimelineEntity te)
+      throws IOException {
     byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
     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);
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af14edb2/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
index 859fdca..7f1ecaf 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/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
@@ -28,7 +28,6 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBuffere
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
 import java.io.IOException;
-import java.util.Map;
 
 /**
  * Identifies fully qualified columns for the {@link AppToFlowTable}.
@@ -43,7 +42,12 @@ public enum AppToFlowColumn implements Column<AppToFlowTable> {
   /**
    * The flow run ID
    */
-  FLOW_RUN_ID(AppToFlowColumnFamily.MAPPING, "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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af14edb2/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 2467856..b30f253 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
@@ -49,6 +49,9 @@ import java.io.IOException;
  * |            | flowRunId:              |
  * |            | 1452828720457           |
  * |            |                         |
+ * |            | user_id:                |
+ * |            | admin                   |
+ * |            |                         |
  * |            |                         |
  * |            |                         |
  * |--------------------------------------|

http://git-wip-us.apache.org/repos/asf/hadoop/blob/af14edb2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
index 4f53fe2..3b285aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -199,6 +199,18 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     entity4.addEvent(event4);
     te4.addEntity(entity4);
 
+    TimelineEntities te5 = new TimelineEntities();
+    TimelineEntity entity5 = new TimelineEntity();
+    entity5.setId("entity1");
+    entity5.setType("type1");
+    entity5.setCreatedTime(1425016501034L);
+    te5.addEntity(entity5);
+    TimelineEntity entity6 = new TimelineEntity();
+    entity6.setId("entity2");
+    entity6.setType("type1");
+    entity6.setCreatedTime(1425016501034L);
+    te5.addEntity(entity6);
+
     HBaseTimelineWriterImpl hbi = null;
     Configuration c1 = util.getConfiguration();
     try {
@@ -209,6 +221,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       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", te5);
       hbi.flush();
     } finally {
       hbi.close();
@@ -333,7 +347,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     Client client = createClient();
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowrun/cluster1/flow_name/1002345678919?userid=user1");
+          "timeline/flowrun/user1/cluster1/flow_name/1002345678919");
       ClientResponse resp = getResponse(client, uri);
       FlowRunEntity entity = resp.getEntity(FlowRunEntity.class);
       assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
@@ -350,7 +364,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // Query without specifying cluster ID.
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowrun/flow_name/1002345678919?userid=user1");
+          "timeline/flowrun/user1/flow_name/1002345678919");
       resp = getResponse(client, uri);
       entity = resp.getEntity(FlowRunEntity.class);
       assertNotNull(entity);
@@ -374,7 +388,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     Client client = createClient();
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowruns/cluster1/flow_name?userid=user1");
+          "timeline/flowruns/user1/cluster1/flow_name");
       ClientResponse resp = getResponse(client, uri);
       Set<FlowRunEntity> entities =
           resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
@@ -393,7 +407,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowruns/cluster1/flow_name?userid=user1&limit=1");
+          "timeline/flowruns/user1/cluster1/flow_name?limit=1");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
       assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
@@ -408,7 +422,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowruns/cluster1/flow_name?userid=user1&" +
+          "timeline/flowruns/user1/cluster1/flow_name?" +
           "createdtimestart=1425016501030");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
@@ -424,7 +438,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowruns/cluster1/flow_name?userid=user1&" +
+          "timeline/flowruns/user1/cluster1/flow_name?" +
           "createdtimestart=1425016500999&createdtimeend=1425016501035");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
@@ -443,7 +457,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowruns/cluster1/flow_name?userid=user1&" +
+          "timeline/flowruns/user1/cluster1/flow_name?" +
           "createdtimeend=1425016501030");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
@@ -459,7 +473,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       }
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowruns/cluster1/flow_name?userid=user1&fields=metrics");
+          "timeline/flowruns/user1/cluster1/flow_name?fields=metrics");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
       assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
@@ -620,7 +634,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/app/cluster1/application_1111111111_1111?" +
-          "userid=user1&fields=ALL");
+          "fields=ALL");
       ClientResponse resp = getResponse(client, uri);
       TimelineEntity entity = resp.getEntity(TimelineEntity.class);
       assertNotNull(entity);
@@ -641,12 +655,48 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   }
 
   @Test
+  public void testGetEntityWithoutFlowInfo() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entity/cluster1/application_1111111111_1111/type1/entity1");
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("entity1", entity.getId());
+      assertEquals("type1", entity.getType());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesWithoutFlowInfo() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/cluster1/application_1111111111_1111/type1");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity1") ||
+            entity.getId().equals("entity2"));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
   public void testGetFlowRunApps() throws Exception {
     Client client = createClient();
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowrunapps/cluster1/flow_name/1002345678919?" +
-          "userid=user1&fields=ALL");
+          "timeline/flowrunapps/user1/cluster1/flow_name/1002345678919?" +
+          "fields=ALL");
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -662,14 +712,14 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // Query without specifying cluster ID.
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowrunapps/flow_name/1002345678919?userid=user1");
+          "timeline/flowrunapps/user1/flow_name/1002345678919");
       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/" +
-          "timeline/flowrunapps/flow_name/1002345678919?userid=user1&limit=1");
+          "timeline/flowrunapps/user1/flow_name/1002345678919?limit=1");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
       assertNotNull(entities);
@@ -684,7 +734,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     Client client = createClient();
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowapps/cluster1/flow_name?userid=user1&fields=ALL");
+          "timeline/flowapps/user1/cluster1/flow_name?fields=ALL");
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -702,14 +752,14 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       // Query without specifying cluster ID.
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowapps/flow_name?userid=user1");
+          "timeline/flowapps/user1/flow_name");
       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/" +
-          "timeline/flowapps/flow_name?userid=user1&limit=1");
+          "timeline/flowapps/user1/flow_name?limit=1");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
       assertNotNull(entities);
@@ -725,7 +775,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     try {
       String entityType = TimelineEntityType.YARN_APPLICATION.toString();
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowapps/cluster1/flow_name?userid=user1&eventfilters=" +
+          "timeline/flowapps/user1/cluster1/flow_name?eventfilters=" +
           ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
@@ -736,7 +786,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           newEntity(entityType, "application_1111111111_1111")));
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowapps/cluster1/flow_name?userid=user1&metricfilters=" +
+          "timeline/flowapps/user1/cluster1/flow_name?metricfilters=" +
           "HDFS_BYTES_READ");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -746,7 +796,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           newEntity(entityType, "application_1111111111_1111")));
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowapps/cluster1/flow_name?userid=user1&conffilters=" +
+          "timeline/flowapps/user1/cluster1/flow_name?conffilters=" +
           "cfg1:value1");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -764,7 +814,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     Client client = createClient();
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowrun/cluster1/flow_name/1002345678929?userid=user1");
+          "timeline/flowrun/user1/cluster1/flow_name/1002345678929");
       verifyHttpResponse(client, uri, Status.NOT_FOUND);
     } finally {
       client.destroy();
@@ -793,8 +843,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     Client client = createClient();
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/app/cluster1/flow_name/1002345678919/" +
-          "application_1111111111_1378?userid=user1");
+          "timeline/app/user1/cluster1/flow_name/1002345678919/" +
+          "application_1111111111_1378");
       verifyHttpResponse(client, uri, Status.NOT_FOUND);
     } finally {
       client.destroy();
@@ -806,7 +856,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     Client client = createClient();
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowrunapps/cluster2/flow_name/1002345678919");
+          "timeline/flowrunapps/user1/cluster2/flow_name/1002345678919");
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -823,7 +873,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     Client client = createClient();
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowapps/cluster2/flow_name55");
+          "timeline/flowapps/user1/cluster2/flow_name55");
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});


[41/50] [abbrv] hadoop git commit: YARN-4445. Unify the term flowId and flowName in timeline v2 codebase. Contributed by Zhan Zhang.

Posted by gt...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java
index a14d2bc..2cd9625 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java
@@ -27,14 +27,14 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStor
 public class FlowRunRowKey {
   private final String clusterId;
   private final String userId;
-  private final String flowId;
+  private final String flowName;
   private final long flowRunId;
 
-  public FlowRunRowKey(String clusterId, String userId, String flowId,
+  public FlowRunRowKey(String clusterId, String userId, String flowName,
       long flowRunId) {
     this.clusterId = clusterId;
     this.userId = userId;
-    this.flowId = flowId;
+    this.flowName = flowName;
     this.flowRunId = flowRunId;
   }
 
@@ -46,8 +46,8 @@ public class FlowRunRowKey {
     return userId;
   }
 
-  public String getFlowId() {
-    return flowId;
+  public String getFlowName() {
+    return flowName;
   }
 
   public long getFlowRunId() {
@@ -56,33 +56,33 @@ public class FlowRunRowKey {
 
   /**
    * Constructs a row key prefix for the flow run table as follows: {
-   * clusterId!userI!flowId!}
+   * clusterId!userI!flowName!}
    *
    * @param clusterId
    * @param userId
-   * @param flowId
+   * @param flowName
    * @return byte array with the row key prefix
    */
   public static byte[] getRowKeyPrefix(String clusterId, String userId,
-      String flowId) {
+      String flowName) {
     return Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId, userId,
-        flowId, ""));
+        flowName, ""));
   }
 
   /**
    * Constructs a row key for the entity table as follows: {
-   * clusterId!userI!flowId!Inverted Flow Run Id}
+   * clusterId!userI!flowName!Inverted Flow Run Id}
    *
    * @param clusterId
    * @param userId
-   * @param flowId
+   * @param flowName
    * @param flowRunId
    * @return byte array with the row key
    */
   public static byte[] getRowKey(String clusterId, String userId,
-      String flowId, Long flowRunId) {
+      String flowName, Long flowRunId) {
     byte[] first = Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId,
-        userId, flowId));
+        userId, flowName));
     // Note that flowRunId is a long, so we can't encode them all at the same
     // time.
     byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
@@ -104,10 +104,10 @@ public class FlowRunRowKey {
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[0]));
     String userId =
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[1]));
-    String flowId =
+    String flowName =
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[2]));
     long flowRunId =
         TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
-    return new FlowRunRowKey(clusterId, userId, flowId, flowRunId);
+    return new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.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/flow/FlowRunTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
index b1b93c1..2682fea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
  * |-------------------------------------------|
  * | clusterId! | flow_version:version7        |
  * | userName!  |                              |
- * | flowId!    | running_apps:1               |
+ * | flowName!  | running_apps:1               |
  * | flowRunId  |                              |
  * |            | min_start_time:1392995080000 |
  * |            | #0:""                        |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/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 91f6ee5..9c74e2d 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
@@ -190,7 +190,7 @@ public class TestTimelineReaderWebServices {
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/entity/cluster1/app1/app/id_1?userid=user1&" +
-          "flowid=flow1&flowrunid=1");
+          "flowname=flow1&flowrunid=1");
       ClientResponse resp = getResponse(client, uri);
       TimelineEntity entity = resp.getEntity(TimelineEntity.class);
       assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
index 818cd89..0918d40 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -595,7 +595,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/app/cluster1/application_1111111111_1111?" +
-          "userid=user1&fields=ALL&flowid=flow_name&flowrunid=1002345678919");
+          "userid=user1&fields=ALL&flowname=flow_name&flowrunid=1002345678919");
       ClientResponse resp = getResponse(client, uri);
       TimelineEntity entity = resp.getEntity(TimelineEntity.class);
       assertNotNull(entity);
@@ -613,7 +613,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
               "timeline/app/application_1111111111_2222?userid=user1" +
-              "&fields=metrics&flowid=flow_name&flowrunid=1002345678919");
+              "&fields=metrics&flowname=flow_name&flowrunid=1002345678919");
       resp = getResponse(client, uri);
       entity = resp.getEntity(TimelineEntity.class);
       assertNotNull(entity);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.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/TestHBaseTimelineStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index bc7b3a4..3198307 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -841,7 +841,7 @@ public class TestHBaseTimelineStorage {
 
     assertEquals(user, key.getUserId());
     assertEquals(cluster, key.getClusterId());
-    assertEquals(flow, key.getFlowId());
+    assertEquals(flow, key.getFlowName());
     assertEquals(runid, key.getFlowRunId());
     assertEquals(appName, key.getAppId());
     assertEquals(te.getType(), key.getEntityType());
@@ -856,7 +856,7 @@ public class TestHBaseTimelineStorage {
 
     assertEquals(cluster, key.getClusterId());
     assertEquals(user, key.getUserId());
-    assertEquals(flow, key.getFlowId());
+    assertEquals(flow, key.getFlowName());
     assertEquals(runid, key.getFlowRunId());
     assertEquals(appName, key.getAppId());
     return true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
index 434adac..eda1e21 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
@@ -165,7 +165,7 @@ public class TestHBaseStorageFlowActivity {
     assertNotNull(flowActivityRowKey);
     assertEquals(cluster, flowActivityRowKey.getClusterId());
     assertEquals(user, flowActivityRowKey.getUserId());
-    assertEquals(flow, flowActivityRowKey.getFlowId());
+    assertEquals(flow, flowActivityRowKey.getFlowName());
     long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(System
         .currentTimeMillis());
     assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
@@ -280,7 +280,7 @@ public class TestHBaseStorageFlowActivity {
       assertNotNull(flowActivityRowKey);
       assertEquals(cluster, flowActivityRowKey.getClusterId());
       assertEquals(user, flowActivityRowKey.getUserId());
-      assertEquals(flow, flowActivityRowKey.getFlowId());
+      assertEquals(flow, flowActivityRowKey.getFlowName());
       long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(System
           .currentTimeMillis());
       assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
@@ -409,7 +409,7 @@ public class TestHBaseStorageFlowActivity {
       assertNotNull(flowActivityRowKey);
       assertEquals(cluster, flowActivityRowKey.getClusterId());
       assertEquals(user, flowActivityRowKey.getUserId());
-      assertEquals(flow, flowActivityRowKey.getFlowId());
+      assertEquals(flow, flowActivityRowKey.getFlowName());
       long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(System
           .currentTimeMillis());
       assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());


[27/50] [abbrv] hadoop git commit: YARN-4178. [storage implementation] app id as string in row keys can cause incorrect ordering (Varun Saxena via sjlee)

Posted by gt...@apache.org.
YARN-4178. [storage implementation] app id as string in row keys can cause incorrect ordering (Varun Saxena via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: fdaa1e4e16e71d9793b4302a50a4335356e3c70f
Parents: 2b6784a
Author: Sangjin Lee <sj...@apache.org>
Authored: Tue Oct 6 16:06:28 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:58 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../storage/ApplicationEntityReader.java        |  14 +-
 .../storage/FileSystemTimelineReaderImpl.java   |  14 +-
 .../storage/GenericEntityReader.java            |  17 +-
 .../storage/HBaseTimelineWriterImpl.java        |  20 +-
 .../storage/application/ApplicationRowKey.java  |  13 +-
 .../storage/apptoflow/AppToFlowRowKey.java      |   7 +-
 .../storage/common/Separator.java               |   4 +-
 .../storage/common/TimelineReaderUtils.java     | 112 -----
 .../storage/common/TimelineStorageUtils.java    | 475 +++++++++++++++++++
 .../storage/common/TimelineWriterUtils.java     | 328 -------------
 .../storage/entity/EntityRowKey.java            |  32 +-
 .../storage/flow/FlowActivityColumnPrefix.java  |   6 +-
 .../storage/flow/FlowActivityRowKey.java        |   9 +-
 .../storage/flow/FlowRunColumn.java             |   4 +-
 .../storage/flow/FlowRunColumnPrefix.java       |   6 +-
 .../storage/flow/FlowRunCoprocessor.java        |   4 +-
 .../storage/flow/FlowRunRowKey.java             |   6 +-
 .../storage/flow/FlowScanner.java               |   6 +-
 .../storage/TestHBaseTimelineStorage.java       |  13 +-
 .../common/TestTimelineStorageUtils.java        |  56 +++
 .../storage/common/TestTimelineWriterUtils.java |  29 --
 .../flow/TestHBaseStorageFlowActivity.java      |  10 +-
 23 files changed, 629 insertions(+), 559 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 902d05e..4ed4949 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -168,6 +168,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4058. Miscellaneous issues in NodeManager project (Naganarasimha G R
     via sjlee)
 
+    YARN-4178. [storage implementation] app id as string in row keys can cause
+    incorrect ordering (Varun Saxena via sjlee)
+
 Trunk - Unreleased
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/ApplicationEntityReader.java
index 61954e1..6d1a2ff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/ApplicationEntityReader.java
@@ -37,7 +37,7 @@ 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.ApplicationTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineReaderUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 
 import com.google.common.base.Preconditions;
 
@@ -182,7 +182,7 @@ class ApplicationEntityReader extends GenericEntityReader {
         fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
       readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO,
           true);
-      if (checkIsRelatedTo && !TimelineReaderUtils.matchRelations(
+      if (checkIsRelatedTo && !TimelineStorageUtils.matchRelations(
           entity.getIsRelatedToEntities(), isRelatedTo)) {
         return null;
       }
@@ -198,7 +198,7 @@ class ApplicationEntityReader extends GenericEntityReader {
         fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
       readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO,
           false);
-      if (checkRelatesTo && !TimelineReaderUtils.matchRelations(
+      if (checkRelatesTo && !TimelineStorageUtils.matchRelations(
           entity.getRelatesToEntities(), relatesTo)) {
         return null;
       }
@@ -214,7 +214,7 @@ class ApplicationEntityReader extends GenericEntityReader {
         fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
       readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false);
       if (checkInfo &&
-          !TimelineReaderUtils.matchFilters(entity.getInfo(), infoFilters)) {
+          !TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) {
         return null;
       }
       if (!fieldsToRetrieve.contains(Field.ALL) &&
@@ -228,7 +228,7 @@ class ApplicationEntityReader extends GenericEntityReader {
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
       readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true);
-      if (checkConfigs && !TimelineReaderUtils.matchFilters(
+      if (checkConfigs && !TimelineStorageUtils.matchFilters(
           entity.getConfigs(), configFilters)) {
         return null;
       }
@@ -243,7 +243,7 @@ class ApplicationEntityReader extends GenericEntityReader {
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
       readEvents(entity, result, true);
-      if (checkEvents && !TimelineReaderUtils.matchEventFilters(
+      if (checkEvents && !TimelineStorageUtils.matchEventFilters(
           entity.getEvents(), eventFilters)) {
         return null;
       }
@@ -258,7 +258,7 @@ class ApplicationEntityReader extends GenericEntityReader {
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
       readMetrics(entity, result, ApplicationColumnPrefix.METRIC);
-      if (checkMetrics && !TimelineReaderUtils.matchMetricFilters(
+      if (checkMetrics && !TimelineStorageUtils.matchMetricFilters(
           entity.getMetrics(), metricFilters)) {
         return null;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/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 626c770..30d1d00 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
@@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineReaderUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.map.JsonMappingException;
@@ -321,31 +321,31 @@ public class FileSystemTimelineReaderImpl extends AbstractService
           continue;
         }
         if (relatesTo != null && !relatesTo.isEmpty() &&
-            !TimelineReaderUtils
+            !TimelineStorageUtils
                 .matchRelations(entity.getRelatesToEntities(), relatesTo)) {
           continue;
         }
         if (isRelatedTo != null && !isRelatedTo.isEmpty() &&
-            !TimelineReaderUtils
+            !TimelineStorageUtils
                 .matchRelations(entity.getIsRelatedToEntities(), isRelatedTo)) {
           continue;
         }
         if (infoFilters != null && !infoFilters.isEmpty() &&
-            !TimelineReaderUtils.matchFilters(entity.getInfo(), infoFilters)) {
+            !TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) {
           continue;
         }
         if (configFilters != null && !configFilters.isEmpty() &&
-            !TimelineReaderUtils.matchFilters(
+            !TimelineStorageUtils.matchFilters(
                 entity.getConfigs(), configFilters)) {
           continue;
         }
         if (metricFilters != null && !metricFilters.isEmpty() &&
-            !TimelineReaderUtils.matchMetricFilters(
+            !TimelineStorageUtils.matchMetricFilters(
                 entity.getMetrics(), metricFilters)) {
           continue;
         }
         if (eventFilters != null && !eventFilters.isEmpty() &&
-            !TimelineReaderUtils.matchEventFilters(
+            !TimelineStorageUtils.matchEventFilters(
                 entity.getEvents(), eventFilters)) {
           continue;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.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/GenericEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
index 42079d7..c18966f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
@@ -44,8 +44,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlow
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineReaderUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
@@ -220,7 +219,7 @@ class GenericEntityReader extends TimelineEntityReader {
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
       readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO, true);
-      if (checkIsRelatedTo && !TimelineReaderUtils.matchRelations(
+      if (checkIsRelatedTo && !TimelineStorageUtils.matchRelations(
           entity.getIsRelatedToEntities(), isRelatedTo)) {
         return null;
       }
@@ -235,7 +234,7 @@ class GenericEntityReader extends TimelineEntityReader {
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
       readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false);
-      if (checkRelatesTo && !TimelineReaderUtils.matchRelations(
+      if (checkRelatesTo && !TimelineStorageUtils.matchRelations(
           entity.getRelatesToEntities(), relatesTo)) {
         return null;
       }
@@ -251,7 +250,7 @@ class GenericEntityReader extends TimelineEntityReader {
         fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
       readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false);
       if (checkInfo &&
-          !TimelineReaderUtils.matchFilters(entity.getInfo(), infoFilters)) {
+          !TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) {
         return null;
       }
       if (!fieldsToRetrieve.contains(Field.ALL) &&
@@ -265,7 +264,7 @@ class GenericEntityReader extends TimelineEntityReader {
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
       readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true);
-      if (checkConfigs && !TimelineReaderUtils.matchFilters(
+      if (checkConfigs && !TimelineStorageUtils.matchFilters(
           entity.getConfigs(), configFilters)) {
         return null;
       }
@@ -280,7 +279,7 @@ class GenericEntityReader extends TimelineEntityReader {
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
       readEvents(entity, result, false);
-      if (checkEvents && !TimelineReaderUtils.matchEventFilters(
+      if (checkEvents && !TimelineStorageUtils.matchEventFilters(
           entity.getEvents(), eventFilters)) {
         return null;
       }
@@ -295,7 +294,7 @@ class GenericEntityReader extends TimelineEntityReader {
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
       readMetrics(entity, result, EntityColumnPrefix.METRIC);
-      if (checkMetrics && !TimelineReaderUtils.matchMetricFilters(
+      if (checkMetrics && !TimelineStorageUtils.matchMetricFilters(
           entity.getMetrics(), metricFilters)) {
         return null;
       }
@@ -365,7 +364,7 @@ class GenericEntityReader extends TimelineEntityReader {
       // the column name is of the form "eventId=timestamp=infoKey"
       if (karr.length == 3) {
         String id = Bytes.toString(karr[0]);
-        long ts = TimelineWriterUtils.invert(Bytes.toLong(karr[1]));
+        long ts = TimelineStorageUtils.invertLong(Bytes.toLong(karr[1]));
         String key = Separator.VALUES.joinEncoded(id, Long.toString(ts));
         TimelineEvent event = eventsMap.get(key);
         if (event == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/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 7c4a5da..3649865 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
@@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlow
 import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
@@ -125,7 +125,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
 
       // if the entity is the application, the destination is the application
       // table
-      boolean isApplication = TimelineWriterUtils.isApplicationEntity(te);
+      boolean isApplication = TimelineStorageUtils.isApplicationEntity(te);
       byte[] rowKey = isApplication ?
           ApplicationRowKey.getRowKey(clusterId, userId, flowName, flowRunId,
               appId) :
@@ -139,7 +139,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       storeRelations(rowKey, te, isApplication);
 
       if (isApplication) {
-        if (TimelineWriterUtils.isApplicationCreated(te)) {
+        if (TimelineStorageUtils.isApplicationCreated(te)) {
           onApplicationCreated(clusterId, userId, flowName, flowVersion,
               flowRunId, appId, te);
         }
@@ -149,7 +149,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
         // if application has finished, store it's finish time and write final
         // values
         // of all metrics
-        if (TimelineWriterUtils.isApplicationFinished(te)) {
+        if (TimelineStorageUtils.isApplicationFinished(te)) {
           onApplicationFinished(clusterId, userId, flowName, flowVersion,
               flowRunId, appId, te);
         }
@@ -234,7 +234,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     Attribute attributeAppId = AggregationCompactionDimension.APPLICATION_ID
         .getAttribute(appId);
     FlowRunColumn.MAX_END_TIME.store(rowKey, flowRunTable, null,
-        TimelineWriterUtils.getApplicationFinishedTime(te), attributeAppId);
+        TimelineStorageUtils.getApplicationFinishedTime(te), attributeAppId);
 
     // store the final value of metrics since application has finished
     Set<TimelineMetric> metrics = te.getMetrics();
@@ -406,9 +406,9 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
             }
             byte[] columnQualifierFirst =
                 Bytes.toBytes(Separator.VALUES.encode(eventId));
-            byte[] columnQualifierWithTsBytes =
-                Separator.VALUES.join(columnQualifierFirst,
-                    Bytes.toBytes(TimelineWriterUtils.invert(eventTimestamp)));
+            byte[] columnQualifierWithTsBytes = Separator.VALUES.
+                join(columnQualifierFirst, Bytes.toBytes(
+                    TimelineStorageUtils.invertLong(eventTimestamp)));
             Map<String, Object> eventInfo = event.getInfo();
             if ((eventInfo == null) || (eventInfo.size() == 0)) {
               // add separator since event key is empty
@@ -418,11 +418,11 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
               if (isApplication) {
                 ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
                     compoundColumnQualifierBytes, null,
-                      TimelineWriterUtils.EMPTY_BYTES);
+                      TimelineStorageUtils.EMPTY_BYTES);
               } else {
                 EntityColumnPrefix.EVENT.store(rowKey, entityTable,
                     compoundColumnQualifierBytes, null,
-                    TimelineWriterUtils.EMPTY_BYTES);
+                      TimelineStorageUtils.EMPTY_BYTES);
               }
             } else {
               for (Map.Entry<String, Object> info : eventInfo.entrySet()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java
index 10e3c2e..1cf6145 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.application;
 
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 
 /**
  * Represents a rowkey for the application table.
@@ -90,7 +90,7 @@ public class ApplicationRowKey {
       String flowId, Long flowRunId) {
     byte[] first = Bytes.toBytes(
         Separator.QUALIFIERS.joinEncoded(clusterId, userId, flowId));
-    byte[] second = Bytes.toBytes(TimelineWriterUtils.invert(flowRunId));
+    byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
     return Separator.QUALIFIERS.join(first, second, new byte[0]);
   }
 
@@ -112,8 +112,8 @@ public class ApplicationRowKey {
             flowId));
     // Note that flowRunId is a long, so we can't encode them all at the same
     // time.
-    byte[] second = Bytes.toBytes(TimelineWriterUtils.invert(flowRunId));
-    byte[] third = Bytes.toBytes(appId);
+    byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
+    byte[] third = TimelineStorageUtils.encodeAppId(appId);
     return Separator.QUALIFIERS.join(first, second, third);
   }
 
@@ -135,9 +135,8 @@ public class ApplicationRowKey {
     String flowId =
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[2]));
     long flowRunId =
-        TimelineWriterUtils.invert(Bytes.toLong(rowKeyComponents[3]));
-    String appId =
-        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[4]));
+        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
+    String appId = TimelineStorageUtils.decodeAppId(rowKeyComponents[4]);
     return new ApplicationRowKey(clusterId, userId, flowId, flowRunId, appId);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/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 ca88056..133952e 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
@@ -19,6 +19,7 @@ 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.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 
 /**
  * Represents a rowkey for the app_flow table.
@@ -49,7 +50,9 @@ public class AppToFlowRowKey {
    * @return byte array with the row key
    */
   public static byte[] getRowKey(String clusterId, String appId) {
-    return Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId, appId));
+    byte[] first = Bytes.toBytes(clusterId);
+    byte[] second = TimelineStorageUtils.encodeAppId(appId);
+    return Separator.QUALIFIERS.join(first, second);
   }
 
   /**
@@ -64,7 +67,7 @@ public class AppToFlowRowKey {
     }
 
     String clusterId = Bytes.toString(rowKeyComponents[0]);
-    String appId = Bytes.toString(rowKeyComponents[1]);
+    String appId = TimelineStorageUtils.decodeAppId(rowKeyComponents[1]);
     return new AppToFlowRowKey(clusterId, appId);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.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/Separator.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/Separator.java
index 9f91af8..1e82494 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/Separator.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/Separator.java
@@ -304,7 +304,7 @@ public enum Separator {
    * @return source split by this separator.
    */
   public byte[][] split(byte[] source, int limit) {
-    return TimelineWriterUtils.split(source, this.bytes, limit);
+    return TimelineStorageUtils.split(source, this.bytes, limit);
   }
 
   /**
@@ -315,6 +315,6 @@ public enum Separator {
    * @return source split by this separator.
    */
   public byte[][] split(byte[] source) {
-    return TimelineWriterUtils.split(source, this.bytes);
+    return TimelineStorageUtils.split(source, this.bytes);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/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/storage/common/TimelineReaderUtils.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/TimelineReaderUtils.java
deleted file mode 100644
index 91d7ba4..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineReaderUtils.java
+++ /dev/null
@@ -1,112 +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.common;
-
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
-
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-
-public class TimelineReaderUtils {
-  /**
-   *
-   * @param entityRelations the relations of an entity
-   * @param relationFilters the relations for filtering
-   * @return a boolean flag to indicate if both match
-   */
-  public static boolean matchRelations(
-      Map<String, Set<String>> entityRelations,
-      Map<String, Set<String>> relationFilters) {
-    for (Map.Entry<String, Set<String>> relation : relationFilters.entrySet()) {
-      Set<String> ids = entityRelations.get(relation.getKey());
-      if (ids == null) {
-        return false;
-      }
-      for (String id : relation.getValue()) {
-        if (!ids.contains(id)) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-
-  /**
-   *
-   * @param map the map of key/value pairs in an entity
-   * @param filters the map of key/value pairs for filtering
-   * @return a boolean flag to indicate if both match
-   */
-  public static boolean matchFilters(Map<String, ? extends Object> map,
-      Map<String, ? extends Object> filters) {
-    for (Map.Entry<String, ? extends Object> filter : filters.entrySet()) {
-      Object value = map.get(filter.getKey());
-      if (value == null) {
-        return false;
-      }
-      if (!value.equals(filter.getValue())) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /**
-   *
-   * @param entityEvents the set of event objects in an entity
-   * @param eventFilters the set of event Ids for filtering
-   * @return a boolean flag to indicate if both match
-   */
-  public static boolean matchEventFilters(Set<TimelineEvent> entityEvents,
-      Set<String> eventFilters) {
-    Set<String> eventIds = new HashSet<String>();
-    for (TimelineEvent event : entityEvents) {
-      eventIds.add(event.getId());
-    }
-    for (String eventFilter : eventFilters) {
-      if (!eventIds.contains(eventFilter)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /**
-   *
-   * @param metrics the set of metric objects in an entity
-   * @param metricFilters the set of metric Ids for filtering
-   * @return a boolean flag to indicate if both match
-   */
-  public static boolean matchMetricFilters(Set<TimelineMetric> metrics,
-      Set<String> metricFilters) {
-    Set<String> metricIds = new HashSet<String>();
-    for (TimelineMetric metric : metrics) {
-      metricIds.add(metric.getId());
-    }
-
-    for (String metricFilter : metricFilters) {
-      if (!metricIds.contains(metricFilter)) {
-        return false;
-      }
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineStorageUtils.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/TimelineStorageUtils.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/TimelineStorageUtils.java
new file mode 100644
index 0000000..c1aaf19
--- /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/common/TimelineStorageUtils.java
@@ -0,0 +1,475 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hbase.Tag;
+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.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.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+
+/**
+ * A bunch of utility functions used across TimelineReader and TimelineWriter.
+ */
+@Public
+@Unstable
+public class TimelineStorageUtils {
+
+  /** empty bytes */
+  public static final byte[] EMPTY_BYTES = new byte[0];
+
+  /** indicator for no limits for splitting */
+  public static final int NO_LIMIT_SPLIT = -1;
+
+  /** milliseconds in one day */
+  public static final long MILLIS_ONE_DAY = 86400000L;
+
+  /**
+   * Splits the source array into multiple array segments using the given
+   * separator, up to a maximum of count items. This will naturally produce
+   * copied byte arrays for each of the split segments. To identify the split
+   * ranges without the array copies, see {@link #splitRanges(byte[], byte[])}.
+   *
+   * @param source
+   * @param separator
+   * @return byte[] array after splitting the source
+   */
+  public static byte[][] split(byte[] source, byte[] separator) {
+    return split(source, separator, NO_LIMIT_SPLIT);
+  }
+
+  /**
+   * Splits the source array into multiple array segments using the given
+   * separator, up to a maximum of count items. This will naturally produce
+   * copied byte arrays for each of the split segments. To identify the split
+   * ranges without the array copies, see {@link #splitRanges(byte[], byte[])}.
+   *
+   * @param source
+   * @param separator
+   * @param limit a non-positive value indicates no limit on number of segments.
+   * @return byte[][] after splitting the input source
+   */
+  public static byte[][] split(byte[] source, byte[] separator, int limit) {
+    List<Range> segments = splitRanges(source, separator, limit);
+
+    byte[][] splits = new byte[segments.size()][];
+    for (int i = 0; i < segments.size(); i++) {
+      Range r = segments.get(i);
+      byte[] tmp = new byte[r.length()];
+      if (tmp.length > 0) {
+        System.arraycopy(source, r.start(), tmp, 0, r.length());
+      }
+      splits[i] = tmp;
+    }
+    return splits;
+  }
+
+  /**
+   * Returns a list of ranges identifying [start, end) -- closed, open --
+   * positions within the source byte array that would be split using the
+   * separator byte array.
+   */
+  public static List<Range> splitRanges(byte[] source, byte[] separator) {
+    return splitRanges(source, separator, NO_LIMIT_SPLIT);
+  }
+
+  /**
+   * Returns a list of ranges identifying [start, end) -- closed, open --
+   * positions within the source byte array that would be split using the
+   * separator byte array.
+   *
+   * @param source the source data
+   * @param separator the separator pattern to look for
+   * @param limit the maximum number of splits to identify in the source
+   */
+  public static List<Range> splitRanges(byte[] source, byte[] separator,
+      int limit) {
+    List<Range> segments = new ArrayList<Range>();
+    if ((source == null) || (separator == null)) {
+      return segments;
+    }
+    int start = 0;
+    itersource: for (int i = 0; i < source.length; i++) {
+      for (int j = 0; j < separator.length; j++) {
+        if (source[i + j] != separator[j]) {
+          continue itersource;
+        }
+      }
+      // all separator elements matched
+      if (limit > 0 && segments.size() >= (limit - 1)) {
+        // everything else goes in one final segment
+        break;
+      }
+	      segments.add(new Range(start, i));
+      start = i + separator.length;
+      // i will be incremented again in outer for loop
+      i += separator.length - 1;
+    }
+    // add in remaining to a final range
+    if (start <= source.length) {
+      segments.add(new Range(start, source.length));
+    }
+    return segments;
+  }
+
+  /**
+   * Converts a timestamp into it's inverse timestamp to be used in (row) keys
+   * where we want to have the most recent timestamp in the top of the table
+   * (scans start at the most recent timestamp first).
+   *
+   * @param key value to be inverted so that the latest version will be first in
+   *          a scan.
+   * @return inverted long
+   */
+  public static long invertLong(long key) {
+    return Long.MAX_VALUE - key;
+  }
+
+  /**
+   * Converts an int into it's inverse int to be used in (row) keys
+   * where we want to have the largest int value in the top of the table
+   * (scans start at the largest int first).
+   *
+   * @param key value to be inverted so that the latest version will be first in
+   *          a scan.
+   * @return inverted int
+   */
+  public static int invertInt(int key) {
+    return Integer.MAX_VALUE - key;
+  }
+
+
+  /**
+   * Converts/encodes a string app Id into a byte representation for (row) keys.
+   * For conversion, we extract cluster timestamp and sequence id from the
+   * string app id (calls {@link ConverterUtils#toApplicationId(String)} for
+   * conversion) and then store it in a byte array of length 12 (8 bytes (long)
+   * for cluster timestamp followed 4 bytes(int) for sequence id). Both cluster
+   * timestamp and sequence id are inverted so that the most recent cluster
+   * timestamp and highest sequence id appears first in the table (i.e.
+   * application id appears in a descending order).
+   *
+   * @param appIdStr application id in string format i.e.
+   * application_{cluster timestamp}_{sequence id with min 4 digits}
+   *
+   * @return encoded byte representation of app id.
+   */
+  public static byte[] encodeAppId(String appIdStr) {
+    ApplicationId appId = ConverterUtils.toApplicationId(appIdStr);
+    byte[] appIdBytes = new byte[Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT];
+    byte[] clusterTs = Bytes.toBytes(invertLong(appId.getClusterTimestamp()));
+    System.arraycopy(clusterTs, 0, appIdBytes, 0, Bytes.SIZEOF_LONG);
+    byte[] seqId = Bytes.toBytes(invertInt(appId.getId()));
+    System.arraycopy(seqId, 0, appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT);
+    return appIdBytes;
+  }
+
+  /**
+   * Converts/decodes a 12 byte representation of app id for (row) keys to an
+   * app id in string format which can be returned back to client.
+   * For decoding, 12 bytes are interpreted as 8 bytes of inverted cluster
+   * timestamp(long) followed by 4 bytes of inverted sequence id(int). Calls
+   * {@link ApplicationId#toString} to generate string representation of app id.
+   *
+   * @param appIdBytes application id in byte representation.
+   *
+   * @return decoded app id in string format.
+   */
+  public static String decodeAppId(byte[] appIdBytes) {
+    if (appIdBytes.length != (Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT)) {
+      throw new IllegalArgumentException("Invalid app id in byte format");
+    }
+    long clusterTs = invertLong(Bytes.toLong(appIdBytes, 0, Bytes.SIZEOF_LONG));
+    int seqId =
+        invertInt(Bytes.toInt(appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT));
+    return ApplicationId.newInstance(clusterTs, seqId).toString();
+  }
+
+  /**
+   * returns the timestamp of that day's start (which is midnight 00:00:00 AM)
+   * for a given input timestamp
+   *
+   * @param ts
+   * @return timestamp of that day's beginning (midnight)
+   */
+  public static long getTopOfTheDayTimestamp(long ts) {
+    long dayTimestamp = ts - (ts % MILLIS_ONE_DAY);
+    return dayTimestamp;
+  }
+
+  /**
+   * Combines the input array of attributes and the input aggregation operation
+   * into a new array of attributes.
+   *
+   * @param attributes
+   * @param aggOp
+   * @return array of combined attributes
+   */
+  public static Attribute[] combineAttributes(Attribute[] attributes,
+      AggregationOperation aggOp) {
+    int newLength = getNewLengthCombinedAttributes(attributes, aggOp);
+    Attribute[] combinedAttributes = new Attribute[newLength];
+
+    if (attributes != null) {
+      System.arraycopy(attributes, 0, combinedAttributes, 0, attributes.length);
+    }
+
+    if (aggOp != null) {
+      Attribute a2 = aggOp.getAttribute();
+      combinedAttributes[newLength - 1] = a2;
+    }
+    return combinedAttributes;
+  }
+
+  /**
+   * Returns a number for the new array size. The new array is the combination
+   * of input array of attributes and the input aggregation operation.
+   *
+   * @param attributes
+   * @param aggOp
+   * @return the size for the new array
+   */
+  private static int getNewLengthCombinedAttributes(Attribute[] attributes,
+      AggregationOperation aggOp) {
+    int oldLength = getAttributesLength(attributes);
+    int aggLength = getAppOpLength(aggOp);
+    return oldLength + aggLength;
+  }
+
+  private static int getAppOpLength(AggregationOperation aggOp) {
+    if (aggOp != null) {
+      return 1;
+    }
+    return 0;
+  }
+
+  private static int getAttributesLength(Attribute[] attributes) {
+    if (attributes != null) {
+      return attributes.length;
+    }
+    return 0;
+  }
+
+  /**
+   * checks if an application has finished
+   *
+   * @param te
+   * @return true if application has finished else false
+   */
+  public static boolean isApplicationFinished(TimelineEntity te) {
+    SortedSet<TimelineEvent> allEvents = te.getEvents();
+    if ((allEvents != null) && (allEvents.size() > 0)) {
+      TimelineEvent event = allEvents.last();
+      if (event.getId().equals(ApplicationMetricsConstants.FINISHED_EVENT_TYPE)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * get the time at which an app finished
+   *
+   * @param te
+   * @return true if application has finished else false
+   */
+  public static long getApplicationFinishedTime(TimelineEntity te) {
+    SortedSet<TimelineEvent> allEvents = te.getEvents();
+    if ((allEvents != null) && (allEvents.size() > 0)) {
+      TimelineEvent event = allEvents.last();
+      if (event.getId().equals(ApplicationMetricsConstants.FINISHED_EVENT_TYPE)) {
+        return event.getTimestamp();
+      }
+    }
+    return 0l;
+  }
+
+  /**
+   * Checks if the input TimelineEntity object is an ApplicationEntity.
+   *
+   * @param te
+   * @return true if input is an ApplicationEntity, false otherwise
+   */
+  public static boolean isApplicationEntity(TimelineEntity te) {
+    return te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString());
+  }
+
+  /**
+   * Checks for the APPLICATION_CREATED event.
+   *
+   * @param te
+   * @return true is application event exists, false otherwise
+   */
+  public static boolean isApplicationCreated(TimelineEntity te) {
+    if (isApplicationEntity(te)) {
+      for (TimelineEvent event : te.getEvents()) {
+        if (event.getId()
+            .equals(ApplicationMetricsConstants.CREATED_EVENT_TYPE)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Returns the first seen aggregation operation as seen in the list of input
+   * tags or null otherwise
+   *
+   * @param tags
+   * @return AggregationOperation
+   */
+  public static AggregationOperation getAggregationOperationFromTagsList(
+      List<Tag> tags) {
+    for (AggregationOperation aggOp : AggregationOperation.values()) {
+      for (Tag tag : tags) {
+        if (tag.getType() == aggOp.getTagType()) {
+          return aggOp;
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Creates a {@link Tag} from the input attribute.
+   *
+   * @param attribute
+   * @return Tag
+   */
+  public static Tag getTagFromAttribute(Entry<String, byte[]> attribute) {
+    // attribute could be either an Aggregation Operation or
+    // an Aggregation Dimension
+    // Get the Tag type from either
+    AggregationOperation aggOp = AggregationOperation
+        .getAggregationOperation(attribute.getKey());
+    if (aggOp != null) {
+      Tag t = new Tag(aggOp.getTagType(), attribute.getValue());
+      return t;
+    }
+
+    AggregationCompactionDimension aggCompactDim = AggregationCompactionDimension
+        .getAggregationCompactionDimension(attribute.getKey());
+    if (aggCompactDim != null) {
+      Tag t = new Tag(aggCompactDim.getTagType(), attribute.getValue());
+      return t;
+    }
+    return null;
+  }
+
+  /**
+   *
+   * @param entityRelations the relations of an entity
+   * @param relationFilters the relations for filtering
+   * @return a boolean flag to indicate if both match
+   */
+  public static boolean matchRelations(
+      Map<String, Set<String>> entityRelations,
+      Map<String, Set<String>> relationFilters) {
+    for (Map.Entry<String, Set<String>> relation : relationFilters.entrySet()) {
+      Set<String> ids = entityRelations.get(relation.getKey());
+      if (ids == null) {
+        return false;
+      }
+      for (String id : relation.getValue()) {
+        if (!ids.contains(id)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
+   *
+   * @param map the map of key/value pairs in an entity
+   * @param filters the map of key/value pairs for filtering
+   * @return a boolean flag to indicate if both match
+   */
+  public static boolean matchFilters(Map<String, ? extends Object> map,
+      Map<String, ? extends Object> filters) {
+    for (Map.Entry<String, ? extends Object> filter : filters.entrySet()) {
+      Object value = map.get(filter.getKey());
+      if (value == null) {
+        return false;
+      }
+      if (!value.equals(filter.getValue())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   *
+   * @param entityEvents the set of event objects in an entity
+   * @param eventFilters the set of event Ids for filtering
+   * @return a boolean flag to indicate if both match
+   */
+  public static boolean matchEventFilters(Set<TimelineEvent> entityEvents,
+      Set<String> eventFilters) {
+    Set<String> eventIds = new HashSet<String>();
+    for (TimelineEvent event : entityEvents) {
+      eventIds.add(event.getId());
+    }
+    for (String eventFilter : eventFilters) {
+      if (!eventIds.contains(eventFilter)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   *
+   * @param metrics the set of metric objects in an entity
+   * @param metricFilters the set of metric Ids for filtering
+   * @return a boolean flag to indicate if both match
+   */
+  public static boolean matchMetricFilters(Set<TimelineMetric> metrics,
+      Set<String> metricFilters) {
+    Set<String> metricIds = new HashSet<String>();
+    for (TimelineMetric metric : metrics) {
+      metricIds.add(metric.getId());
+    }
+
+    for (String metricFilter : metricFilters) {
+      if (!metricIds.contains(metricFilter)) {
+        return false;
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineWriterUtils.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/TimelineWriterUtils.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/TimelineWriterUtils.java
deleted file mode 100644
index 371371a..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineWriterUtils.java
+++ /dev/null
@@ -1,328 +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.common;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.SortedSet;
-import java.util.Map.Entry;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.Tag;
-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.TimelineEvent;
-import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
-
-/**
- * bunch of utility functions used across TimelineWriter classes
- */
-@InterfaceAudience.Public
-@InterfaceStability.Unstable
-public class TimelineWriterUtils {
-
-  /** empty bytes */
-  public static final byte[] EMPTY_BYTES = new byte[0];
-
-  /** indicator for no limits for splitting */
-  public static final int NO_LIMIT_SPLIT = -1;
-
-  /** milliseconds in one day */
-  public static final long MILLIS_ONE_DAY = 86400000L;
-
-  /**
-   * Splits the source array into multiple array segments using the given
-   * separator, up to a maximum of count items. This will naturally produce
-   * copied byte arrays for each of the split segments. To identify the split
-   * ranges without the array copies, see
-   * {@link TimelineWriterUtils#splitRanges(byte[], byte[])}.
-   *
-   * @param source
-   * @param separator
-   * @return byte[] array after splitting the source
-   */
-  public static byte[][] split(byte[] source, byte[] separator) {
-    return split(source, separator, NO_LIMIT_SPLIT);
-  }
-
-  /**
-   * Splits the source array into multiple array segments using the given
-   * separator, up to a maximum of count items. This will naturally produce
-   * copied byte arrays for each of the split segments. To identify the split
-   * ranges without the array copies, see
-   * {@link TimelineWriterUtils#splitRanges(byte[], byte[])}.
-   *
-   * @param source
-   * @param separator
-   * @param limit a non-positive value indicates no limit on number of segments.
-   * @return byte[][] after splitting the input source
-   */
-  public static byte[][] split(byte[] source, byte[] separator, int limit) {
-    List<Range> segments = splitRanges(source, separator, limit);
-
-    byte[][] splits = new byte[segments.size()][];
-    for (int i = 0; i < segments.size(); i++) {
-      Range r = segments.get(i);
-      byte[] tmp = new byte[r.length()];
-      if (tmp.length > 0) {
-        System.arraycopy(source, r.start(), tmp, 0, r.length());
-      }
-      splits[i] = tmp;
-    }
-    return splits;
-  }
-
-  /**
-   * Returns a list of ranges identifying [start, end) -- closed, open --
-   * positions within the source byte array that would be split using the
-   * separator byte array.
-   */
-  public static List<Range> splitRanges(byte[] source, byte[] separator) {
-    return splitRanges(source, separator, NO_LIMIT_SPLIT);
-  }
-
-  /**
-   * Returns a list of ranges identifying [start, end) -- closed, open --
-   * positions within the source byte array that would be split using the
-   * separator byte array.
-   *
-   * @param source the source data
-   * @param separator the separator pattern to look for
-   * @param limit the maximum number of splits to identify in the source
-   */
-  public static List<Range> splitRanges(byte[] source, byte[] separator,
-      int limit) {
-    List<Range> segments = new ArrayList<Range>();
-    if ((source == null) || (separator == null)) {
-      return segments;
-    }
-    int start = 0;
-    itersource: for (int i = 0; i < source.length; i++) {
-      for (int j = 0; j < separator.length; j++) {
-        if (source[i + j] != separator[j]) {
-          continue itersource;
-        }
-      }
-      // all separator elements matched
-      if (limit > 0 && segments.size() >= (limit - 1)) {
-        // everything else goes in one final segment
-        break;
-      }
-
-      segments.add(new Range(start, i));
-      start = i + separator.length;
-      // i will be incremented again in outer for loop
-      i += separator.length - 1;
-    }
-    // add in remaining to a final range
-    if (start <= source.length) {
-      segments.add(new Range(start, source.length));
-    }
-    return segments;
-  }
-
-  /**
-   * Converts a timestamp into it's inverse timestamp to be used in (row) keys
-   * where we want to have the most recent timestamp in the top of the table
-   * (scans start at the most recent timestamp first).
-   *
-   * @param key value to be inverted so that the latest version will be first in
-   *          a scan.
-   * @return inverted long
-   */
-  public static long invert(Long key) {
-    return Long.MAX_VALUE - key;
-  }
-
-  /**
-   * returns the timestamp of that day's start (which is midnight 00:00:00 AM)
-   * for a given input timestamp
-   *
-   * @param ts
-   * @return timestamp of that day's beginning (midnight)
-   */
-  public static long getTopOfTheDayTimestamp(long ts) {
-    long dayTimestamp = ts - (ts % MILLIS_ONE_DAY);
-    return dayTimestamp;
-  }
-
-  /**
-   * Combines the input array of attributes and the input aggregation operation
-   * into a new array of attributes.
-   *
-   * @param attributes
-   * @param aggOp
-   * @return array of combined attributes
-   */
-  public static Attribute[] combineAttributes(Attribute[] attributes,
-      AggregationOperation aggOp) {
-    int newLength = getNewLengthCombinedAttributes(attributes, aggOp);
-    Attribute[] combinedAttributes = new Attribute[newLength];
-
-    if (attributes != null) {
-      System.arraycopy(attributes, 0, combinedAttributes, 0, attributes.length);
-    }
-
-    if (aggOp != null) {
-      Attribute a2 = aggOp.getAttribute();
-      combinedAttributes[newLength - 1] = a2;
-    }
-    return combinedAttributes;
-  }
-
-  /**
-   * Returns a number for the new array size. The new array is the combination
-   * of input array of attributes and the input aggregation operation.
-   *
-   * @param attributes
-   * @param aggOp
-   * @return the size for the new array
-   */
-  private static int getNewLengthCombinedAttributes(Attribute[] attributes,
-      AggregationOperation aggOp) {
-    int oldLength = getAttributesLength(attributes);
-    int aggLength = getAppOpLength(aggOp);
-    return oldLength + aggLength;
-  }
-
-  private static int getAppOpLength(AggregationOperation aggOp) {
-    if (aggOp != null) {
-      return 1;
-    }
-    return 0;
-  }
-
-  private static int getAttributesLength(Attribute[] attributes) {
-    if (attributes != null) {
-      return attributes.length;
-    }
-    return 0;
-  }
-
-  /**
-   * checks if an application has finished
-   *
-   * @param te
-   * @return true if application has finished else false
-   */
-  public static boolean isApplicationFinished(TimelineEntity te) {
-    SortedSet<TimelineEvent> allEvents = te.getEvents();
-    if ((allEvents != null) && (allEvents.size() > 0)) {
-      TimelineEvent event = allEvents.last();
-      if (event.getId().equals(ApplicationMetricsConstants.FINISHED_EVENT_TYPE)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * get the time at which an app finished
-   *
-   * @param te
-   * @return true if application has finished else false
-   */
-  public static long getApplicationFinishedTime(TimelineEntity te) {
-    SortedSet<TimelineEvent> allEvents = te.getEvents();
-    if ((allEvents != null) && (allEvents.size() > 0)) {
-      TimelineEvent event = allEvents.last();
-      if (event.getId().equals(ApplicationMetricsConstants.FINISHED_EVENT_TYPE)) {
-        return event.getTimestamp();
-      }
-    }
-    return 0l;
-  }
-
-  /**
-   * Checks if the input TimelineEntity object is an ApplicationEntity.
-   *
-   * @param te
-   * @return true if input is an ApplicationEntity, false otherwise
-   */
-  public static boolean isApplicationEntity(TimelineEntity te) {
-    return te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString());
-  }
-
-  /**
-   * Checks for the APPLICATION_CREATED event.
-   *
-   * @param te
-   * @return true is application event exists, false otherwise
-   */
-  public static boolean isApplicationCreated(TimelineEntity te) {
-    if (isApplicationEntity(te)) {
-      for (TimelineEvent event : te.getEvents()) {
-        if (event.getId()
-            .equals(ApplicationMetricsConstants.CREATED_EVENT_TYPE)) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Returns the first seen aggregation operation as seen in the list of input
-   * tags or null otherwise
-   *
-   * @param tags
-   * @return AggregationOperation
-   */
-  public static AggregationOperation getAggregationOperationFromTagsList(
-      List<Tag> tags) {
-    for (AggregationOperation aggOp : AggregationOperation.values()) {
-      for (Tag tag : tags) {
-        if (tag.getType() == aggOp.getTagType()) {
-          return aggOp;
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Creates a {@link Tag} from the input attribute.
-   *
-   * @param attribute
-   * @return Tag
-   */
-  public static Tag getTagFromAttribute(Entry<String, byte[]> attribute) {
-    // attribute could be either an Aggregation Operation or
-    // an Aggregation Dimension
-    // Get the Tag type from either
-    AggregationOperation aggOp = AggregationOperation
-        .getAggregationOperation(attribute.getKey());
-    if (aggOp != null) {
-      Tag t = new Tag(aggOp.getTagType(), attribute.getValue());
-      return t;
-    }
-
-    AggregationCompactionDimension aggCompactDim = AggregationCompactionDimension
-        .getAggregationCompactionDimension(attribute.getKey());
-    if (aggCompactDim != null) {
-      Tag t = new Tag(aggCompactDim.getTagType(), attribute.getValue());
-      return t;
-    }
-    return null;
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java
index 6a534ed73..e0413c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.entity;
 
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 
 /**
  * Represents a rowkey for the entity table.
@@ -90,9 +90,9 @@ public class EntityRowKey {
             flowId));
     // Note that flowRunId is a long, so we can't encode them all at the same
     // time.
-    byte[] second = Bytes.toBytes(TimelineWriterUtils.invert(flowRunId));
-    byte[] third = Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(appId));
-    return Separator.QUALIFIERS.join(first, second, third);
+    byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
+    byte[] third = TimelineStorageUtils.encodeAppId(appId);
+    return Separator.QUALIFIERS.join(first, second, third, new byte[0]);
   }
 
   /**
@@ -114,10 +114,11 @@ public class EntityRowKey {
             flowId));
     // Note that flowRunId is a long, so we can't encode them all at the same
     // time.
-    byte[] second = Bytes.toBytes(TimelineWriterUtils.invert(flowRunId));
-    byte[] third =
-        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(appId, entityType, ""));
-    return Separator.QUALIFIERS.join(first, second, third);
+    byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
+    byte[] third = TimelineStorageUtils.encodeAppId(appId);
+    byte[] fourth =
+        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(entityType, ""));
+    return Separator.QUALIFIERS.join(first, second, third, fourth);
   }
 
   /**
@@ -141,11 +142,11 @@ public class EntityRowKey {
             flowId));
     // Note that flowRunId is a long, so we can't encode them all at the same
     // time.
-    byte[] second = Bytes.toBytes(TimelineWriterUtils.invert(flowRunId));
-    byte[] third =
-        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(appId, entityType,
-            entityId));
-    return Separator.QUALIFIERS.join(first, second, third);
+    byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
+    byte[] third = TimelineStorageUtils.encodeAppId(appId);
+    byte[] fourth =
+        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(entityType, entityId));
+    return Separator.QUALIFIERS.join(first, second, third, fourth);
   }
 
   /**
@@ -166,9 +167,8 @@ public class EntityRowKey {
     String flowId =
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[2]));
     long flowRunId =
-        TimelineWriterUtils.invert(Bytes.toLong(rowKeyComponents[3]));
-    String appId =
-        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[4]));
+        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
+    String appId = TimelineStorageUtils.decodeAppId(rowKeyComponents[4]);
     String entityType =
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[5]));
     String entityId =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java
index b899e5c..38c0f3f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java
@@ -27,7 +27,7 @@ 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.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
 
 /**
@@ -114,7 +114,7 @@ public enum FlowActivityColumnPrefix implements ColumnPrefix<FlowActivityTable>
 
     byte[] columnQualifier = ColumnHelper.getColumnQualifier(
         this.columnPrefixBytes, qualifier);
-    Attribute[] combinedAttributes = TimelineWriterUtils.combineAttributes(
+    Attribute[] combinedAttributes = TimelineStorageUtils.combineAttributes(
         attributes, this.aggOp);
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         combinedAttributes);
@@ -235,7 +235,7 @@ public enum FlowActivityColumnPrefix implements ColumnPrefix<FlowActivityTable>
 
     byte[] columnQualifier = ColumnHelper.getColumnQualifier(
         this.columnPrefixBytes, qualifier);
-    Attribute[] combinedAttributes = TimelineWriterUtils.combineAttributes(
+    Attribute[] combinedAttributes = TimelineStorageUtils.combineAttributes(
         attributes, this.aggOp);
     column.store(rowKey, tableMutator, columnQualifier, null, inputValue,
         combinedAttributes);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
index 18ca599..f7841e0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 
 /**
  * Represents a rowkey for the flow activity table.
@@ -71,7 +71,7 @@ public class FlowActivityRowKey {
    */
   public static byte[] getRowKey(String clusterId, String userId,
       String flowId) {
-    long dayTs = TimelineWriterUtils.getTopOfTheDayTimestamp(System
+    long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(System
         .currentTimeMillis());
     return getRowKey(clusterId, dayTs, userId, flowId);
   }
@@ -90,7 +90,7 @@ public class FlowActivityRowKey {
       String flowId) {
     return Separator.QUALIFIERS.join(
         Bytes.toBytes(Separator.QUALIFIERS.encode(clusterId)),
-        Bytes.toBytes(TimelineWriterUtils.invert(dayTs)),
+        Bytes.toBytes(TimelineStorageUtils.invertLong(dayTs)),
         Bytes.toBytes(Separator.QUALIFIERS.encode(userId)),
         Bytes.toBytes(Separator.QUALIFIERS.encode(flowId)));
   }
@@ -108,7 +108,8 @@ public class FlowActivityRowKey {
 
     String clusterId = Separator.QUALIFIERS.decode(Bytes
         .toString(rowKeyComponents[0]));
-    long dayTs = TimelineWriterUtils.invert(Bytes.toLong(rowKeyComponents[1]));
+    long dayTs =
+        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[1]));
     String userId = Separator.QUALIFIERS.decode(Bytes
         .toString(rowKeyComponents[2]));
     String flowId = Separator.QUALIFIERS.decode(Bytes

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java
index ad30add..5079cc0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java
@@ -25,7 +25,7 @@ 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.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
 
 /**
@@ -97,7 +97,7 @@ public enum FlowRunColumn implements Column<FlowRunTable> {
       TypedBufferedMutator<FlowRunTable> tableMutator, Long timestamp,
       Object inputValue, Attribute... attributes) throws IOException {
 
-    Attribute[] combinedAttributes = TimelineWriterUtils.combineAttributes(
+    Attribute[] combinedAttributes = TimelineStorageUtils.combineAttributes(
         attributes, aggOp);
     column.store(rowKey, tableMutator, columnQualifierBytes, timestamp,
         inputValue, combinedAttributes);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java
index d55f510..b090bba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java
@@ -27,7 +27,7 @@ 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.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
 
 /**
@@ -112,7 +112,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
 
     byte[] columnQualifier = ColumnHelper.getColumnQualifier(
         this.columnPrefixBytes, qualifier);
-    Attribute[] combinedAttributes = TimelineWriterUtils.combineAttributes(
+    Attribute[] combinedAttributes = TimelineStorageUtils.combineAttributes(
         attributes, this.aggOp);
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         combinedAttributes);
@@ -140,7 +140,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
 
     byte[] columnQualifier = ColumnHelper.getColumnQualifier(
         this.columnPrefixBytes, qualifier);
-    Attribute[] combinedAttributes = TimelineWriterUtils.combineAttributes(
+    Attribute[] combinedAttributes = TimelineStorageUtils.combineAttributes(
         attributes, this.aggOp);
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         combinedAttributes);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.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/flow/FlowRunCoprocessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
index f743e5e..1984157 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator;
 
 public class FlowRunCoprocessor extends BaseRegionObserver {
@@ -89,7 +89,7 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
     List<Tag> tags = new ArrayList<>();
     if ((attributes != null) && (attributes.size() > 0)) {
       for (Map.Entry<String, byte[]> attribute : attributes.entrySet()) {
-        Tag t = TimelineWriterUtils.getTagFromAttribute(attribute);
+        Tag t = TimelineStorageUtils.getTagFromAttribute(attribute);
         tags.add(t);
       }
       byte[] tagByteArray = Tag.fromList(tags);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java
index 880d481..7ed3651 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 
 /**
  * Represents a rowkey for the flow run table.
@@ -70,7 +70,7 @@ public class FlowRunRowKey {
         userId, flowId));
     // Note that flowRunId is a long, so we can't encode them all at the same
     // time.
-    byte[] second = Bytes.toBytes(TimelineWriterUtils.invert(flowRunId));
+    byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
     return Separator.QUALIFIERS.join(first, second);
   }
 
@@ -92,7 +92,7 @@ public class FlowRunRowKey {
     String flowId =
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[2]));
     long flowRunId =
-        TimelineWriterUtils.invert(Bytes.toLong(rowKeyComponents[3]));
+        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
     return new FlowRunRowKey(clusterId, userId, flowId, flowRunId);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.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/flow/FlowScanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
index 651bb3a..a537891 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
 import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 
 /**
  * Invoked via the coprocessor when a Get or a Scan is issued for flow run
@@ -136,7 +136,7 @@ class FlowScanner implements RegionScanner, Closeable {
     // So all cells in one qualifier come one after the other before we see the
     // next column qualifier
     ByteArrayComparator comp = new ByteArrayComparator();
-    byte[] currentColumnQualifier = TimelineWriterUtils.EMPTY_BYTES;
+    byte[] currentColumnQualifier = TimelineStorageUtils.EMPTY_BYTES;
     AggregationOperation currentAggOp = null;
     SortedSet<Cell> currentColumnCells = new TreeSet<>(KeyValue.COMPARATOR);
     Set<String> alreadySeenAggDim = new HashSet<>();
@@ -163,7 +163,7 @@ class FlowScanner implements RegionScanner, Closeable {
     List<Tag> tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
         cell.getTagsLength());
     // We assume that all the operations for a particular column are the same
-    return TimelineWriterUtils.getAggregationOperationFromTagsList(tags);
+    return TimelineStorageUtils.getAggregationOperationFromTagsList(tags);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.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/TestHBaseTimelineStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index 3b0921b..701615e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -54,7 +54,7 @@ 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.ApplicationTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
@@ -366,7 +366,8 @@ public class TestHBaseTimelineStorage {
       String flow = "some_flow_name";
       String flowVersion = "AB7822C10F1111";
       long runid = 1002345678919L;
-      String appName = "some app name";
+      String appName =
+          ApplicationId.newInstance(System.currentTimeMillis(), 1).toString();
       hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
       hbi.stop();
 
@@ -592,7 +593,8 @@ public class TestHBaseTimelineStorage {
         byte[][] karr = (byte[][])e.getKey();
         assertEquals(3, karr.length);
         assertEquals(eventId, Bytes.toString(karr[0]));
-        assertEquals(TimelineWriterUtils.invert(expTs), Bytes.toLong(karr[1]));
+        assertEquals(
+            TimelineStorageUtils.invertLong(expTs), Bytes.toLong(karr[1]));
         assertEquals(expKey, Bytes.toString(karr[2]));
         Object value = e.getValue();
         // there should be only one timestamp and value
@@ -667,7 +669,8 @@ public class TestHBaseTimelineStorage {
       String flow = "other_flow_name";
       String flowVersion = "1111F01C2287BA";
       long runid = 1009876543218L;
-      String appName = "some app name";
+      String appName =
+          ApplicationId.newInstance(System.currentTimeMillis(), 1).toString();
       byte[] startRow =
           EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
       hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
@@ -700,7 +703,7 @@ public class TestHBaseTimelineStorage {
             byte[][] karr = (byte[][])e.getKey();
             assertEquals(3, karr.length);
             assertEquals(eventId, Bytes.toString(karr[0]));
-            assertEquals(TimelineWriterUtils.invert(expTs),
+            assertEquals(TimelineStorageUtils.invertLong(expTs),
                 Bytes.toLong(karr[1]));
             // key must be empty
             assertEquals(0, karr[2].length);


[50/50] [abbrv] hadoop git commit: Rebase to latest trunk

Posted by gt...@apache.org.
Rebase to latest trunk


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

Branch: refs/heads/feature-YARN-2928
Commit: cab8a1d1e4ab2a8212520d0ef6de90f27a901feb
Parents: 63d9099
Author: Li Lu <gt...@apache.org>
Authored: Wed Jan 20 01:05:52 2016 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Wed Jan 20 01:05:52 2016 -0800

----------------------------------------------------------------------
 hadoop-tools/hadoop-aws/pom.xml                   |  8 ++++++++
 .../yarn/client/api/impl/TimelineClientImpl.java  |  3 ++-
 .../metrics/TimelineServiceV1Publisher.java       | 17 +++++++++++++++--
 .../metrics/TimelineServiceV2Publisher.java       | 18 +++++++++++++++---
 .../server/resourcemanager/rmapp/RMAppImpl.java   |  8 +-------
 .../TestResourceTrackerService.java               | 12 ++++++++----
 .../hadoop-yarn-server-timelineservice/pom.xml    |  6 ------
 7 files changed, 49 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cab8a1d1/hadoop-tools/hadoop-aws/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml
index eb87d77..a0bfc8d 100644
--- a/hadoop-tools/hadoop-aws/pom.xml
+++ b/hadoop-tools/hadoop-aws/pom.xml
@@ -138,6 +138,14 @@
       <artifactId>hadoop-yarn-server-tests</artifactId>
       <scope>test</scope>
       <type>test-jar</type>
+      <exclusions>
+        <!-- server-tests brings in an older version of joda-time via
+        timeline service -->
+        <exclusion>
+          <groupId>joda-time</groupId>
+          <artifactId>joda-time</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cab8a1d1/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 61789a3..8b68784 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
@@ -332,7 +332,8 @@ public class TimelineClientImpl extends TimelineClient {
   @Override
   protected void serviceStart() throws Exception {
     timelineWriter = createTimelineWriter(
-        configuration, authUgi, client, resURI);
+        configuration, authUgi, client,
+        constructResURI(getConfig(), timelineServiceAddress, false));
   }
 
   protected TimelineWriter createTimelineWriter(Configuration conf,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cab8a1d1/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 d858a6b..ddc8a16 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
@@ -87,6 +87,17 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
         app.getAmNodeLabelExpression());
     entityInfo.put(ApplicationMetricsConstants.APP_NODE_LABEL_EXPRESSION,
         app.getAppNodeLabelExpression());
+    if (app.getCallerContext() != null) {
+      if (app.getCallerContext().getContext() != null) {
+        entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_CONTEXT,
+            app.getCallerContext().getContext());
+      }
+      if (app.getCallerContext().getSignature() != null) {
+        entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_SIGNATURE,
+            app.getCallerContext().getSignature());
+      }
+    }
+
     entity.setOtherInfo(entityInfo);
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setEventType(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
@@ -185,8 +196,10 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
         appAttempt.getHost());
     eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO,
         appAttempt.getRpcPort());
-    eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
-        appAttempt.getMasterContainer().getId().toString());
+    if (appAttempt.getMasterContainer() != null) {
+      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
+          appAttempt.getMasterContainer().getId().toString());
+    }
     tEvent.setEventInfo(eventInfo);
     entity.addEvent(tEvent);
     getDispatcher().getEventHandler().handle(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cab8a1d1/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 1954783..1b3aa6f 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
@@ -117,8 +117,18 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
     entity.getConfigs().put(
         ApplicationMetricsConstants.APP_NODE_LABEL_EXPRESSION,
         app.getAppNodeLabelExpression());
-    entity.setInfo(entityInfo);
+    if (app.getCallerContext() != null) {
+      if (app.getCallerContext().getContext() != null) {
+        entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_CONTEXT,
+            app.getCallerContext().getContext());
+      }
+      if (app.getCallerContext().getSignature() != null) {
+        entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_SIGNATURE,
+            app.getCallerContext().getSignature());
+      }
+    }
 
+    entity.setInfo(entityInfo);
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
     tEvent.setTimestamp(createdTime);
@@ -220,8 +230,10 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
         appAttempt.getHost());
     eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO,
         appAttempt.getRpcPort());
-    eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
-        appAttempt.getMasterContainer().getId().toString());
+    if (appAttempt.getMasterContainer() != null) {
+      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
+          appAttempt.getMasterContainer().getId().toString());
+    }
     tEvent.setInfo(eventInfo);
     entity.addEvent(tEvent);
     getDispatcher().getEventHandler().handle(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cab8a1d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.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/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 0872553..7ee41ac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -1841,20 +1841,16 @@ public class RMAppImpl implements RMApp, Recoverable {
     }
     return amNodeLabelExpression;
   }
-<<<<<<< 93f2f13bc1c25fa441a3819c98b7073fca45b07a
-  
+
   @Override
   public CallerContext getCallerContext() {
     return callerContext;
   }
-=======
->>>>>>> YARN-4392. ApplicationCreatedEvent event time resets after RM
 
   private void sendATSCreateEvent(RMApp app, long startTime) {
     rmContext.getRMApplicationHistoryWriter().applicationStarted(app);
     rmContext.getSystemMetricsPublisher().appCreated(app, startTime);
   }
-<<<<<<< 93f2f13bc1c25fa441a3819c98b7073fca45b07a
 
   @VisibleForTesting
   public boolean isAmBlacklistingEnabled() {
@@ -1865,6 +1861,4 @@ public class RMAppImpl implements RMApp, Recoverable {
   public float getAmBlacklistingDisableThreshold() {
     return blacklistDisableThreshold;
   }
-=======
->>>>>>> YARN-4392. ApplicationCreatedEvent event time resets after RM
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cab8a1d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.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/TestResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
index b0e6c55..d98160e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
@@ -920,8 +920,10 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
     statusList.add(status1);
     NodeHealthStatus nodeHealth = NodeHealthStatus.newInstance(true,
         "", System.currentTimeMillis());
-    node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeHealth,
-        statusList, null, nodeHeartbeat1));
+    NodeStatus nodeStatus = NodeStatus.newInstance(nm1.getNodeId(), 0,
+        statusList, null, nodeHealth, null, null, null);
+    node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeStatus,
+        nodeHeartbeat1));
 
     Assert.assertEquals(1, node1.getRunningApps().size());
     Assert.assertEquals(app1.getApplicationId(), node1.getRunningApps().get(0));
@@ -935,8 +937,10 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
         ContainerState.RUNNING, "", 0);
     statusList = new ArrayList<ContainerStatus>();
     statusList.add(status2);
-    node2.handle(new RMNodeStatusEvent(nm2.getNodeId(), nodeHealth,
-        statusList, null, nodeHeartbeat2));
+    nodeStatus = NodeStatus.newInstance(nm1.getNodeId(), 0,
+        statusList, null, nodeHealth, null, null, null);
+    node2.handle(new RMNodeStatusEvent(nm2.getNodeId(), nodeStatus,
+        nodeHeartbeat2));
     Assert.assertEquals(1, node2.getRunningApps().size());
     Assert.assertEquals(app2.getApplicationId(), node2.getRunningApps().get(0));
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cab8a1d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
index 616ed06..758feb5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
@@ -124,12 +124,6 @@
     <dependency>
       <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix-core</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>joda-time</groupId>
-          <artifactId>joda-time</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>


[19/50] [abbrv] hadoop git commit: YARN-4074. [timeline reader] implement support for querying for flows and flow runs (sjlee via vrushali)

Posted by gt...@apache.org.
YARN-4074. [timeline reader] implement support for querying for flows and flow runs (sjlee via vrushali)


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

Branch: refs/heads/feature-YARN-2928
Commit: 25f58b40d3cf3ab000fcac76c67a919c9183b8ab
Parents: a469bfe
Author: Vrushali <vr...@apache.org>
Authored: Tue Sep 22 13:42:30 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:57 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../timelineservice/FlowActivityEntity.java     | 183 ++++++++
 .../api/records/timelineservice/FlowEntity.java | 103 -----
 .../records/timelineservice/FlowRunEntity.java  | 121 ++++++
 .../timelineservice/TimelineEntityType.java     |  31 +-
 .../TestTimelineServiceRecords.java             |  14 +-
 .../TestTimelineServiceClientIntegration.java   |   2 +-
 .../collector/TimelineCollectorWebService.java  |   6 +-
 .../storage/ApplicationEntityReader.java        | 229 ++++++++++
 .../storage/FlowActivityEntityReader.java       | 168 +++++++
 .../storage/FlowRunEntityReader.java            | 136 ++++++
 .../storage/GenericEntityReader.java            | 389 +++++++++++++++++
 .../storage/HBaseTimelineReaderImpl.java        | 434 +------------------
 .../storage/TimelineEntityReader.java           | 223 ++++++++++
 .../storage/TimelineEntityReaderFactory.java    |  97 +++++
 .../storage/application/ApplicationRowKey.java  |  68 ++-
 .../storage/apptoflow/AppToFlowRowKey.java      |  31 ++
 .../storage/common/BaseTable.java               |   3 +-
 .../storage/entity/EntityRowKey.java            |  76 +++-
 .../storage/flow/FlowActivityRowKey.java        |   7 +-
 .../storage/flow/FlowRunRowKey.java             |  50 ++-
 .../storage/flow/FlowScanner.java               |  18 +-
 .../storage/TestHBaseTimelineStorage.java       |  34 +-
 .../storage/flow/TestFlowDataGenerator.java     |  39 +-
 .../flow/TestHBaseStorageFlowActivity.java      | 131 ++++--
 .../storage/flow/TestHBaseStorageFlowRun.java   | 105 +++--
 26 files changed, 2021 insertions(+), 680 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 98c5e94..004fec4 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -106,6 +106,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3901. Populate flow run data in the flow_run & flow activity tables
     (Vrushali C via sjlee)
 
+    YARN-4074. [timeline reader] implement support for querying for flows
+    and flow runs (sjlee via vrushali)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java
new file mode 100644
index 0000000..163bd5c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java
@@ -0,0 +1,183 @@
+/*
+ * 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.api.records.timelineservice;
+
+import java.util.Collection;
+import java.util.Date;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import javax.xml.bind.annotation.XmlElement;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Entity that represents a record for flow activity. It's essentially a
+ * container entity for flow runs with limited information.
+ */
+@Public
+@Unstable
+public class FlowActivityEntity extends TimelineEntity {
+  public static final String CLUSTER_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX +  "CLUSTER";
+  public static final String DATE_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX +  "DATE";
+  public static final String USER_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "USER";
+  public static final String FLOW_NAME_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "FLOW_NAME";
+
+  private final NavigableSet<FlowRunEntity> flowRuns = new TreeSet<>();
+
+  public FlowActivityEntity() {
+    super(TimelineEntityType.YARN_FLOW_ACTIVITY.toString());
+    // set config to null
+    setConfigs(null);
+  }
+
+  public FlowActivityEntity(String cluster, long time, String user,
+      String flowName) {
+    this();
+    setCluster(cluster);
+    setDate(time);
+    setUser(user);
+    setFlowName(flowName);
+  }
+
+  public FlowActivityEntity(TimelineEntity entity) {
+    super(entity);
+    if (!TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entity.getType())) {
+      throw new IllegalArgumentException("Incompatible entity type: " +
+          getId());
+    }
+    // set config to null
+    setConfigs(null);
+  }
+
+  @XmlElement(name = "id")
+  @Override
+  public String getId() {
+    // flow activity: cluster/day/user@flow_name
+    String id = super.getId();
+    if (id == null) {
+      StringBuilder sb = new StringBuilder();
+      sb.append(getCluster());
+      sb.append('/');
+      sb.append(getDate().getTime());
+      sb.append('/');
+      sb.append(getUser());
+      sb.append('@');
+      sb.append(getFlowName());
+      id = sb.toString();
+      setId(id);
+    }
+    return id;
+  }
+
+  @Override
+  public int compareTo(TimelineEntity entity) {
+    int comparison = getType().compareTo(entity.getType());
+    if (comparison == 0) {
+      // order by cluster, date (descending), user, and flow name
+      FlowActivityEntity other = (FlowActivityEntity)entity;
+      int clusterComparison = getCluster().compareTo(other.getCluster());
+      if (clusterComparison != 0) {
+        return clusterComparison;
+      }
+      int dateComparisonDescending =
+          (int)(other.getDate().getTime() - getDate().getTime()); // descending
+      if (dateComparisonDescending != 0) {
+        return dateComparisonDescending; // descending
+      }
+      int userComparison = getUser().compareTo(other.getUser());
+      if (userComparison != 0) {
+        return userComparison;
+      }
+      return getFlowName().compareTo(other.getFlowName());
+    } else {
+      return comparison;
+    }
+  }
+
+  /**
+   * Reuse the base class equals method.
+   */
+  @Override
+  public boolean equals(Object obj) {
+    return super.equals(obj);
+  }
+
+  /**
+   * Reuse the base class hashCode method.
+   */
+  @Override
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  public String getCluster() {
+    return (String)getInfo().get(CLUSTER_INFO_KEY);
+  }
+
+  public void setCluster(String cluster) {
+    addInfo(CLUSTER_INFO_KEY, cluster);
+  }
+
+  public Date getDate() {
+    return (Date)getInfo().get(DATE_INFO_KEY);
+  }
+
+  public void setDate(long time) {
+    Date date = new Date(time);
+    addInfo(DATE_INFO_KEY, date);
+  }
+
+  public String getUser() {
+    return (String)getInfo().get(USER_INFO_KEY);
+  }
+
+  public void setUser(String user) {
+    addInfo(USER_INFO_KEY, user);
+  }
+
+  public String getFlowName() {
+    return (String)getInfo().get(FLOW_NAME_INFO_KEY);
+  }
+
+  public void setFlowName(String flowName) {
+    addInfo(FLOW_NAME_INFO_KEY, flowName);
+  }
+
+  public void addFlowRun(FlowRunEntity run) {
+    flowRuns.add(run);
+  }
+
+  public void addFlowRuns(Collection<FlowRunEntity> runs) {
+    flowRuns.addAll(runs);
+  }
+
+  @XmlElement(name = "flowruns")
+  public NavigableSet<FlowRunEntity> getFlowRuns() {
+    return flowRuns;
+  }
+
+  public int getNumberOfRuns() {
+    return flowRuns.size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowEntity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowEntity.java
deleted file mode 100644
index 4554778..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowEntity.java
+++ /dev/null
@@ -1,103 +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.api.records.timelineservice;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-import javax.xml.bind.annotation.XmlElement;
-
-@InterfaceAudience.Public
-@InterfaceStability.Unstable
-public class FlowEntity extends HierarchicalTimelineEntity {
-  public static final String USER_INFO_KEY =
-      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "USER";
-  public static final String FLOW_NAME_INFO_KEY =
-      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "FLOW_NAME";
-  public static final String FLOW_VERSION_INFO_KEY =
-      TimelineEntity.SYSTEM_INFO_KEY_PREFIX +  "FLOW_VERSION";
-  public static final String FLOW_RUN_ID_INFO_KEY =
-      TimelineEntity.SYSTEM_INFO_KEY_PREFIX +  "FLOW_RUN_ID";
-
-  public FlowEntity() {
-    super(TimelineEntityType.YARN_FLOW.toString());
-  }
-
-  public FlowEntity(TimelineEntity entity) {
-    super(entity);
-    if (!entity.getType().equals(TimelineEntityType.YARN_FLOW.toString())) {
-      throw new IllegalArgumentException("Incompatible entity type: " + getId());
-    }
-  }
-
-  @XmlElement(name = "id")
-  @Override
-  public String getId() {
-    //Flow id schema: user@flow_name(or id)/version/run_id
-    String id = super.getId();
-    if (id == null) {
-      StringBuilder sb = new StringBuilder();
-      sb.append(getInfo().get(USER_INFO_KEY).toString());
-      sb.append('@');
-      sb.append(getInfo().get(FLOW_NAME_INFO_KEY).toString());
-      sb.append('/');
-      sb.append(getInfo().get(FLOW_VERSION_INFO_KEY).toString());
-      sb.append('/');
-      sb.append(getInfo().get(FLOW_RUN_ID_INFO_KEY).toString());
-      id = sb.toString();
-      setId(id);
-    }
-    return id;
-  }
-
-  public String getUser() {
-    Object user = getInfo().get(USER_INFO_KEY);
-    return user == null ? null : user.toString();
-  }
-
-  public void setUser(String user) {
-    addInfo(USER_INFO_KEY, user);
-  }
-
-  public String getName() {
-    Object name = getInfo().get(FLOW_NAME_INFO_KEY);
-    return name == null ? null : name.toString();
-  }
-
-  public void setName(String name) {
-    addInfo(FLOW_NAME_INFO_KEY, name);
-  }
-
-  public String getVersion() {
-    Object version = getInfo().get(FLOW_VERSION_INFO_KEY);
-    return version == null ? null : version.toString();
-  }
-
-  public void setVersion(String version) {
-    addInfo(FLOW_VERSION_INFO_KEY, version);
-  }
-
-  public long getRunId() {
-    Object runId = getInfo().get(FLOW_RUN_ID_INFO_KEY);
-    return runId == null ? 0L : (Long) runId;
-  }
-
-  public void setRunId(long runId) {
-    addInfo(FLOW_RUN_ID_INFO_KEY, runId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowRunEntity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowRunEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowRunEntity.java
new file mode 100644
index 0000000..3c3ffb4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowRunEntity.java
@@ -0,0 +1,121 @@
+/*
+ * 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.api.records.timelineservice;
+
+import javax.xml.bind.annotation.XmlElement;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class FlowRunEntity extends HierarchicalTimelineEntity {
+  public static final String USER_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "USER";
+  public static final String FLOW_NAME_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "FLOW_NAME";
+  public static final String FLOW_VERSION_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX +  "FLOW_VERSION";
+  public static final String FLOW_RUN_ID_INFO_KEY =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX +  "FLOW_RUN_ID";
+  public static final String FLOW_RUN_END_TIME =
+      TimelineEntity.SYSTEM_INFO_KEY_PREFIX + "FLOW_RUN_END_TIME";
+
+  public FlowRunEntity() {
+    super(TimelineEntityType.YARN_FLOW_RUN.toString());
+    // set config to null
+    setConfigs(null);
+  }
+
+  public FlowRunEntity(TimelineEntity entity) {
+    super(entity);
+    if (!entity.getType().equals(TimelineEntityType.YARN_FLOW_RUN.toString())) {
+      throw new IllegalArgumentException("Incompatible entity type: " + getId());
+    }
+    // set config to null
+    setConfigs(null);
+  }
+
+  @XmlElement(name = "id")
+  @Override
+  public String getId() {
+    //Flow id schema: user@flow_name(or id)/run_id
+    String id = super.getId();
+    if (id == null) {
+      StringBuilder sb = new StringBuilder();
+      sb.append(getInfo().get(USER_INFO_KEY).toString());
+      sb.append('@');
+      sb.append(getInfo().get(FLOW_NAME_INFO_KEY).toString());
+      sb.append('/');
+      sb.append(getInfo().get(FLOW_RUN_ID_INFO_KEY).toString());
+      id = sb.toString();
+      setId(id);
+    }
+    return id;
+  }
+
+  public String getUser() {
+    return (String)getInfo().get(USER_INFO_KEY);
+  }
+
+  public void setUser(String user) {
+    addInfo(USER_INFO_KEY, user);
+  }
+
+  public String getName() {
+    return (String)getInfo().get(FLOW_NAME_INFO_KEY);
+  }
+
+  public void setName(String name) {
+    addInfo(FLOW_NAME_INFO_KEY, name);
+  }
+
+  public String getVersion() {
+    return (String)getInfo().get(FLOW_VERSION_INFO_KEY);
+  }
+
+  public void setVersion(String version) {
+    addInfo(FLOW_VERSION_INFO_KEY, version);
+  }
+
+  public long getRunId() {
+    Object runId = getInfo().get(FLOW_RUN_ID_INFO_KEY);
+    return runId == null ? 0L : (Long) runId;
+  }
+
+  public void setRunId(long runId) {
+    addInfo(FLOW_RUN_ID_INFO_KEY, runId);
+  }
+
+  public long getStartTime() {
+    return getCreatedTime();
+  }
+
+  public void setStartTime(long startTime) {
+    setCreatedTime(startTime);
+  }
+
+  public long getMaxEndTime() {
+    Object time = getInfo().get(FLOW_RUN_END_TIME);
+    return time == null ? 0L : (Long)time;
+  }
+
+  public void setMaxEndTime(long endTime) {
+    addInfo(FLOW_RUN_END_TIME, endTime);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntityType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntityType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntityType.java
index 6062fe1..ba32e20 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntityType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntityType.java
@@ -24,21 +24,25 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Unstable
 public enum TimelineEntityType {
   YARN_CLUSTER,
-  YARN_FLOW,
+  YARN_FLOW_RUN,
   YARN_APPLICATION,
   YARN_APPLICATION_ATTEMPT,
   YARN_CONTAINER,
   YARN_USER,
-  YARN_QUEUE;
+  YARN_QUEUE,
+  YARN_FLOW_ACTIVITY;
 
+  /**
+   * Whether the input type can be a parent of this entity.
+   */
   public boolean isParent(TimelineEntityType type) {
     switch (this) {
       case YARN_CLUSTER:
         return false;
-      case YARN_FLOW:
-        return YARN_FLOW == type || YARN_CLUSTER == type;
+      case YARN_FLOW_RUN:
+        return YARN_FLOW_RUN == type || YARN_CLUSTER == type;
       case YARN_APPLICATION:
-        return YARN_FLOW == type || YARN_CLUSTER == type;
+        return YARN_FLOW_RUN == type || YARN_CLUSTER == type;
       case YARN_APPLICATION_ATTEMPT:
         return YARN_APPLICATION == type;
       case YARN_CONTAINER:
@@ -50,12 +54,15 @@ public enum TimelineEntityType {
     }
   }
 
+  /**
+   * Whether the input type can be a child of this entity.
+   */
   public boolean isChild(TimelineEntityType type) {
     switch (this) {
       case YARN_CLUSTER:
-        return YARN_FLOW == type || YARN_APPLICATION == type;
-      case YARN_FLOW:
-        return YARN_FLOW == type || YARN_APPLICATION == type;
+        return YARN_FLOW_RUN == type || YARN_APPLICATION == type;
+      case YARN_FLOW_RUN:
+        return YARN_FLOW_RUN == type || YARN_APPLICATION == type;
       case YARN_APPLICATION:
         return YARN_APPLICATION_ATTEMPT == type;
       case YARN_APPLICATION_ATTEMPT:
@@ -68,4 +75,12 @@ public enum TimelineEntityType {
         return false;
     }
   }
+
+  /**
+   * Whether the type of this entity matches the type indicated by the input
+   * argument.
+   */
+  public boolean matches(String typeString) {
+    return toString().equals(typeString);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java
index 78943e0..7c9acf2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineServiceRecords.java
@@ -182,14 +182,14 @@ public class TestTimelineServiceRecords {
     ClusterEntity cluster = new ClusterEntity();
     cluster.setId("test cluster id");
 
-    FlowEntity flow1 = new FlowEntity();
+    FlowRunEntity flow1 = new FlowRunEntity();
     //flow1.setId("test flow id 1");
     flow1.setUser(user.getId());
     flow1.setName("test flow name 1");
     flow1.setVersion("test flow version 1");
     flow1.setRunId(1L);
 
-    FlowEntity flow2 = new FlowEntity();
+    FlowRunEntity flow2 = new FlowRunEntity();
     //flow2.setId("test flow run id 2");
     flow2.setUser(user.getId());
     flow2.setName("test flow name 2");
@@ -213,19 +213,19 @@ public class TestTimelineServiceRecords {
         ApplicationAttemptId.newInstance(
             ApplicationId.newInstance(0, 1), 1), 1).toString());
 
-    cluster.addChild(TimelineEntityType.YARN_FLOW.toString(), flow1.getId());
+    cluster.addChild(TimelineEntityType.YARN_FLOW_RUN.toString(), flow1.getId());
     flow1
         .setParent(TimelineEntityType.YARN_CLUSTER.toString(), cluster.getId());
-    flow1.addChild(TimelineEntityType.YARN_FLOW.toString(), flow2.getId());
-    flow2.setParent(TimelineEntityType.YARN_FLOW.toString(), flow1.getId());
+    flow1.addChild(TimelineEntityType.YARN_FLOW_RUN.toString(), flow2.getId());
+    flow2.setParent(TimelineEntityType.YARN_FLOW_RUN.toString(), flow1.getId());
     flow2.addChild(TimelineEntityType.YARN_APPLICATION.toString(), app1.getId());
     flow2.addChild(TimelineEntityType.YARN_APPLICATION.toString(), app2.getId());
-    app1.setParent(TimelineEntityType.YARN_FLOW.toString(), flow2.getId());
+    app1.setParent(TimelineEntityType.YARN_FLOW_RUN.toString(), flow2.getId());
     app1.addChild(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(),
         appAttempt.getId());
     appAttempt
         .setParent(TimelineEntityType.YARN_APPLICATION.toString(), app1.getId());
-    app2.setParent(TimelineEntityType.YARN_FLOW.toString(), flow2.getId());
+    app2.setParent(TimelineEntityType.YARN_FLOW_RUN.toString(), flow2.getId());
     appAttempt.addChild(TimelineEntityType.YARN_CONTAINER.toString(),
         container.getId());
     container.setParent(TimelineEntityType.YARN_APPLICATION_ATTEMPT.toString(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/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 69031a2..5672759 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
@@ -107,7 +107,7 @@ public class TestTimelineServiceClientIntegration {
       client.start();
       ClusterEntity cluster = new ClusterEntity();
       cluster.setId(YarnConfiguration.DEFAULT_RM_CLUSTER_ID);
-      FlowEntity flow = new FlowEntity();
+      FlowRunEntity flow = new FlowRunEntity();
       flow.setUser(UserGroupInformation.getCurrentUser().getShortUserName());
       flow.setName("test_flow_name");
       flow.setVersion("test_flow_version");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/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 42fa365..8f595e2 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
@@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationAttemptEnti
 import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.ClusterEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.ContainerEntity;
-import org.apache.hadoop.yarn.api.records.timelineservice.FlowEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.QueueEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
@@ -205,8 +205,8 @@ public class TimelineCollectorWebService {
           case YARN_CLUSTER:
             entitiesToReturn.addEntity(new ClusterEntity(entity));
             break;
-          case YARN_FLOW:
-            entitiesToReturn.addEntity(new FlowEntity(entity));
+          case YARN_FLOW_RUN:
+            entitiesToReturn.addEntity(new FlowRunEntity(entity));
             break;
           case YARN_APPLICATION:
             entitiesToReturn.addEntity(new ApplicationEntity(entity));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/ApplicationEntityReader.java
new file mode 100644
index 0000000..dfbc31d
--- /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/ApplicationEntityReader.java
@@ -0,0 +1,229 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Map;
+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.Result;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
+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;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineReaderUtils;
+
+/**
+ * Timeline entity reader for application entities that are stored in the
+ * application table.
+ */
+class ApplicationEntityReader extends GenericEntityReader {
+  private static final ApplicationTable APPLICATION_TABLE =
+      new ApplicationTable();
+
+  public ApplicationEntityReader(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
+        createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
+        relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
+        eventFilters, fieldsToRetrieve);
+  }
+
+  public ApplicationEntityReader(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      String entityId, EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowId, flowRunId, appId, entityType, entityId,
+        fieldsToRetrieve);
+  }
+
+  /**
+   * Uses the {@link ApplicationTable}.
+   */
+  protected BaseTable<?> getTable() {
+    return APPLICATION_TABLE;
+  }
+
+  @Override
+  protected Result getResult(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    byte[] rowKey =
+        ApplicationRowKey.getRowKey(clusterId, userId, flowId, flowRunId,
+            appId);
+    Get get = new Get(rowKey);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    return table.getResult(hbaseConf, conn, get);
+  }
+
+  @Override
+  protected Iterable<Result> getResults(Configuration hbaseConf,
+      Connection conn) throws IOException {
+    // If getEntities() is called for an application, there can be at most
+    // one entity. If the entity passes the filter, it is returned. Otherwise,
+    // an empty set is returned.
+    byte[] rowKey = ApplicationRowKey.getRowKey(clusterId, userId, flowId,
+        flowRunId, appId);
+    Get get = new Get(rowKey);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    Result result = table.getResult(hbaseConf, conn, get);
+    TimelineEntity entity = parseEntity(result);
+    Set<Result> set;
+    if (entity != null) {
+      set = Collections.singleton(result);
+    } else {
+      set = Collections.emptySet();
+    }
+    return set;
+  }
+
+  @Override
+  protected TimelineEntity parseEntity(Result result) throws IOException {
+    if (result == null || result.isEmpty()) {
+      return null;
+    }
+    TimelineEntity entity = new TimelineEntity();
+    entity.setType(TimelineEntityType.YARN_APPLICATION.toString());
+    String entityId = ApplicationColumn.ID.readResult(result).toString();
+    entity.setId(entityId);
+
+    // fetch created time
+    Number createdTime =
+        (Number)ApplicationColumn.CREATED_TIME.readResult(result);
+    entity.setCreatedTime(createdTime.longValue());
+    if (!singleEntityRead && (entity.getCreatedTime() < createdTimeBegin ||
+        entity.getCreatedTime() > createdTimeEnd)) {
+      return null;
+    }
+
+    // fetch modified time
+    Number modifiedTime =
+        (Number)ApplicationColumn.MODIFIED_TIME.readResult(result);
+    entity.setModifiedTime(modifiedTime.longValue());
+    if (!singleEntityRead && (entity.getModifiedTime() < modifiedTimeBegin ||
+        entity.getModifiedTime() > modifiedTimeEnd)) {
+      return null;
+    }
+
+    // fetch is related to entities
+    boolean checkIsRelatedTo = isRelatedTo != null && isRelatedTo.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
+      readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO,
+          true);
+      if (checkIsRelatedTo && !TimelineReaderUtils.matchRelations(
+          entity.getIsRelatedToEntities(), isRelatedTo)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.IS_RELATED_TO)) {
+        entity.getIsRelatedToEntities().clear();
+      }
+    }
+
+    // fetch relates to entities
+    boolean checkRelatesTo = relatesTo != null && relatesTo.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
+      readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO,
+          false);
+      if (checkRelatesTo && !TimelineReaderUtils.matchRelations(
+          entity.getRelatesToEntities(), relatesTo)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.RELATES_TO)) {
+        entity.getRelatesToEntities().clear();
+      }
+    }
+
+    // fetch info
+    boolean checkInfo = infoFilters != null && infoFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
+      readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false);
+      if (checkInfo &&
+          !TimelineReaderUtils.matchFilters(entity.getInfo(), infoFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.INFO)) {
+        entity.getInfo().clear();
+      }
+    }
+
+    // fetch configs
+    boolean checkConfigs = configFilters != null && configFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
+      readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true);
+      if (checkConfigs && !TimelineReaderUtils.matchFilters(
+          entity.getConfigs(), configFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.CONFIGS)) {
+        entity.getConfigs().clear();
+      }
+    }
+
+    // fetch events
+    boolean checkEvents = eventFilters != null && eventFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
+      readEvents(entity, result, true);
+      if (checkEvents && !TimelineReaderUtils.matchEventFilters(
+          entity.getEvents(), eventFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.EVENTS)) {
+        entity.getEvents().clear();
+      }
+    }
+
+    // fetch metrics
+    boolean checkMetrics = metricFilters != null && metricFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
+      readMetrics(entity, result, ApplicationColumnPrefix.METRIC);
+      if (checkMetrics && !TimelineReaderUtils.matchMetricFilters(
+          entity.getMetrics(), metricFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.METRICS)) {
+        entity.getMetrics().clear();
+      }
+    }
+    return entity;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.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/FlowActivityEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
new file mode 100644
index 0000000..d5ece2e
--- /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/FlowActivityEntityReader.java
@@ -0,0 +1,168 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.PageFilter;
+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.TimelineEntity;
+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.flow.FlowActivityColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Timeline entity reader for flow activity entities that are stored in the
+ * flow activity table.
+ */
+class FlowActivityEntityReader extends TimelineEntityReader {
+  private static final FlowActivityTable FLOW_ACTIVITY_TABLE =
+      new FlowActivityTable();
+
+  public FlowActivityEntityReader(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
+        createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
+        relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
+        eventFilters, fieldsToRetrieve);
+  }
+
+  public FlowActivityEntityReader(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      String entityId, EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowId, flowRunId, appId, entityType, entityId,
+        fieldsToRetrieve);
+  }
+
+  /**
+   * Uses the {@link FlowActivityTable}.
+   */
+  @Override
+  protected BaseTable<?> getTable() {
+    return FLOW_ACTIVITY_TABLE;
+  }
+
+  /**
+   * Since this is strictly sorted by the row key, it is sufficient to collect
+   * the first results as specified by the limit.
+   */
+  @Override
+  public Set<TimelineEntity> readEntities(Configuration hbaseConf,
+      Connection conn) throws IOException {
+    validateParams();
+    augmentParams(hbaseConf, conn);
+
+    NavigableSet<TimelineEntity> entities = new TreeSet<>();
+    Iterable<Result> results = getResults(hbaseConf, conn);
+    for (Result result : results) {
+      TimelineEntity entity = parseEntity(result);
+      if (entity == null) {
+        continue;
+      }
+      entities.add(entity);
+      if (entities.size() == limit) {
+        break;
+      }
+    }
+    return entities;
+  }
+
+  @Override
+  protected void validateParams() {
+    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
+  }
+
+  @Override
+  protected void augmentParams(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    if (limit == null || limit < 0) {
+      limit = TimelineReader.DEFAULT_LIMIT;
+    }
+  }
+
+  @Override
+  protected Result getResult(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "we don't support a single entity query");
+  }
+
+  @Override
+  protected Iterable<Result> getResults(Configuration hbaseConf,
+      Connection conn) throws IOException {
+    Scan scan = new Scan();
+    scan.setRowPrefixFilter(FlowActivityRowKey.getRowKeyPrefix(clusterId));
+    // use the page filter to limit the result to the page size
+    // the scanner may still return more than the limit; therefore we need to
+    // read the right number as we iterate
+    scan.setFilter(new PageFilter(limit));
+    return table.getResultScanner(hbaseConf, conn, scan);
+  }
+
+  @Override
+  protected TimelineEntity parseEntity(Result result) throws IOException {
+    FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(result.getRow());
+
+    long time = rowKey.getDayTimestamp();
+    String user = rowKey.getUserId();
+    String flowName = rowKey.getFlowId();
+
+    FlowActivityEntity flowActivity =
+        new FlowActivityEntity(clusterId, time, user, flowName);
+    // set the id
+    flowActivity.setId(flowActivity.getId());
+    // get the list of run ids along with the version that are associated with
+    // this flow on this day
+    Map<String, Object> runIdsMap =
+        FlowActivityColumnPrefix.RUN_ID.readResults(result);
+    for (Map.Entry<String, Object> e : runIdsMap.entrySet()) {
+      Long runId = Long.valueOf(e.getKey());
+      String version = (String)e.getValue();
+      FlowRunEntity flowRun = new FlowRunEntity();
+      flowRun.setUser(user);
+      flowRun.setName(flowName);
+      flowRun.setRunId(runId);
+      flowRun.setVersion(version);
+      // set the id
+      flowRun.setId(flowRun.getId());
+      flowActivity.addFlowRun(flowRun);
+    }
+
+    return flowActivity;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/FlowRunEntityReader.java
new file mode 100644
index 0000000..ced795d
--- /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/FlowRunEntityReader.java
@@ -0,0 +1,136 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Map;
+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.Result;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+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.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 com.google.common.base.Preconditions;
+
+/**
+ * Timeline entity reader for flow run entities that are stored in the flow run
+ * table.
+ */
+class FlowRunEntityReader extends TimelineEntityReader {
+  private static final FlowRunTable FLOW_RUN_TABLE = new FlowRunTable();
+
+  public FlowRunEntityReader(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
+        createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
+        relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
+        eventFilters, fieldsToRetrieve);
+  }
+
+  public FlowRunEntityReader(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      String entityId, EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowId, flowRunId, appId, entityType, entityId,
+        fieldsToRetrieve);
+  }
+
+  /**
+   * Uses the {@link FlowRunTable}.
+   */
+  @Override
+  protected BaseTable<?> getTable() {
+    return FLOW_RUN_TABLE;
+  }
+
+  @Override
+  protected void validateParams() {
+    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
+    Preconditions.checkNotNull(userId, "userId shouldn't be null");
+    Preconditions.checkNotNull(flowId, "flowId shouldn't be null");
+    Preconditions.checkNotNull(flowRunId, "flowRunId shouldn't be null");
+  }
+
+  @Override
+  protected void augmentParams(Configuration hbaseConf, Connection conn) {
+  }
+
+  @Override
+  protected Result getResult(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    byte[] rowKey =
+        FlowRunRowKey.getRowKey(clusterId, userId, flowId, flowRunId);
+    Get get = new Get(rowKey);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    return table.getResult(hbaseConf, conn, get);
+  }
+
+  @Override
+  protected Iterable<Result> getResults(Configuration hbaseConf,
+      Connection conn) throws IOException {
+    throw new UnsupportedOperationException(
+        "multiple entity query is not supported");
+  }
+
+  @Override
+  protected TimelineEntity parseEntity(Result result) throws IOException {
+    FlowRunEntity flowRun = new FlowRunEntity();
+    flowRun.setUser(userId);
+    flowRun.setName(flowId);
+    flowRun.setRunId(flowRunId);
+
+    // read the start time
+    Long startTime = (Long)FlowRunColumn.MIN_START_TIME.readResult(result);
+    if (startTime != null) {
+      flowRun.setStartTime(startTime);
+    }
+    // read the end time if available
+    Long endTime = (Long)FlowRunColumn.MAX_END_TIME.readResult(result);
+    if (endTime != null) {
+      flowRun.setMaxEndTime(endTime);
+    }
+
+    // read the flow version
+    String version = (String)FlowRunColumn.FLOW_VERSION.readResult(result);
+    if (version != null) {
+      flowRun.setVersion(version);
+    }
+
+    // read metrics
+    readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC);
+
+    // set the id
+    flowRun.setId(flowRun.getId());
+    return flowRun;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.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/GenericEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
new file mode 100644
index 0000000..466914b
--- /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/GenericEntityReader.java
@@ -0,0 +1,389 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+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.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
+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.AppToFlowRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineReaderUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Timeline entity reader for generic entities that are stored in the entity
+ * table.
+ */
+class GenericEntityReader extends TimelineEntityReader {
+  private static final EntityTable ENTITY_TABLE = new EntityTable();
+  private static final Log LOG = LogFactory.getLog(GenericEntityReader.class);
+
+  private static final long DEFAULT_BEGIN_TIME = 0L;
+  private static final long DEFAULT_END_TIME = Long.MAX_VALUE;
+
+  /**
+   * Used to look up the flow context.
+   */
+  private final AppToFlowTable appToFlowTable = new AppToFlowTable();
+
+  public GenericEntityReader(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
+        createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
+        relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
+        eventFilters, fieldsToRetrieve);
+  }
+
+  public GenericEntityReader(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      String entityId, EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowId, flowRunId, appId, entityType, entityId,
+        fieldsToRetrieve);
+  }
+
+  /**
+   * Uses the {@link EntityTable}.
+   */
+  protected BaseTable<?> getTable() {
+    return ENTITY_TABLE;
+  }
+
+  private FlowContext lookupFlowContext(String clusterId, String appId,
+      Configuration hbaseConf, Connection conn) throws IOException {
+    byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
+    Get get = new Get(rowKey);
+    Result result = appToFlowTable.getResult(hbaseConf, conn, get);
+    if (result != null && !result.isEmpty()) {
+      return new FlowContext(
+          AppToFlowColumn.FLOW_ID.readResult(result).toString(),
+          ((Number)AppToFlowColumn.FLOW_RUN_ID.readResult(result)).longValue());
+    } else {
+       throw new IOException(
+           "Unable to find the context flow ID and flow run ID for clusterId=" +
+           clusterId + ", appId=" + appId);
+    }
+  }
+
+  private static class FlowContext {
+    private final String flowId;
+    private final Long flowRunId;
+    public FlowContext(String flowId, Long flowRunId) {
+      this.flowId = flowId;
+      this.flowRunId = flowRunId;
+    }
+  }
+
+  @Override
+  protected void validateParams() {
+    Preconditions.checkNotNull(userId, "userId shouldn't be null");
+    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
+    Preconditions.checkNotNull(appId, "appId shouldn't be null");
+    Preconditions.checkNotNull(entityType, "entityType shouldn't be null");
+    if (singleEntityRead) {
+      Preconditions.checkNotNull(entityId, "entityId shouldn't be null");
+    }
+  }
+
+  @Override
+  protected void augmentParams(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    // In reality both should be null or neither should be null
+    if (flowId == null || flowRunId == null) {
+      FlowContext context =
+          lookupFlowContext(clusterId, appId, hbaseConf, conn);
+      flowId = context.flowId;
+      flowRunId = context.flowRunId;
+    }
+    if (fieldsToRetrieve == null) {
+      fieldsToRetrieve = EnumSet.noneOf(Field.class);
+    }
+    if (!singleEntityRead) {
+      if (limit == null || limit < 0) {
+        limit = TimelineReader.DEFAULT_LIMIT;
+      }
+      if (createdTimeBegin == null) {
+        createdTimeBegin = DEFAULT_BEGIN_TIME;
+      }
+      if (createdTimeEnd == null) {
+        createdTimeEnd = DEFAULT_END_TIME;
+      }
+      if (modifiedTimeBegin == null) {
+        modifiedTimeBegin = DEFAULT_BEGIN_TIME;
+      }
+      if (modifiedTimeEnd == null) {
+        modifiedTimeEnd = DEFAULT_END_TIME;
+      }
+    }
+  }
+
+  @Override
+  protected Result getResult(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    byte[] rowKey =
+        EntityRowKey.getRowKey(clusterId, userId, flowId, flowRunId, appId,
+            entityType, entityId);
+    Get get = new Get(rowKey);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    return table.getResult(hbaseConf, conn, get);
+  }
+
+  @Override
+  protected Iterable<Result> getResults(Configuration hbaseConf,
+      Connection conn) throws IOException {
+    // Scan through part of the table to find the entities belong to one app
+    // and one type
+    Scan scan = new Scan();
+    scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
+        clusterId, userId, flowId, flowRunId, appId, entityType));
+    scan.setMaxVersions(Integer.MAX_VALUE);
+    return table.getResultScanner(hbaseConf, conn, scan);
+  }
+
+  @Override
+  protected TimelineEntity parseEntity(Result result) throws IOException {
+    if (result == null || result.isEmpty()) {
+      return null;
+    }
+    TimelineEntity entity = new TimelineEntity();
+    String entityType = EntityColumn.TYPE.readResult(result).toString();
+    entity.setType(entityType);
+    String entityId = EntityColumn.ID.readResult(result).toString();
+    entity.setId(entityId);
+
+    // fetch created time
+    Number createdTime = (Number)EntityColumn.CREATED_TIME.readResult(result);
+    entity.setCreatedTime(createdTime.longValue());
+    if (!singleEntityRead && (entity.getCreatedTime() < createdTimeBegin ||
+        entity.getCreatedTime() > createdTimeEnd)) {
+      return null;
+    }
+
+    // fetch modified time
+    Number modifiedTime = (Number)EntityColumn.MODIFIED_TIME.readResult(result);
+    entity.setModifiedTime(modifiedTime.longValue());
+    if (!singleEntityRead && (entity.getModifiedTime() < modifiedTimeBegin ||
+        entity.getModifiedTime() > modifiedTimeEnd)) {
+      return null;
+    }
+
+    // fetch is related to entities
+    boolean checkIsRelatedTo = isRelatedTo != null && isRelatedTo.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
+      readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO, true);
+      if (checkIsRelatedTo && !TimelineReaderUtils.matchRelations(
+          entity.getIsRelatedToEntities(), isRelatedTo)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.IS_RELATED_TO)) {
+        entity.getIsRelatedToEntities().clear();
+      }
+    }
+
+    // fetch relates to entities
+    boolean checkRelatesTo = relatesTo != null && relatesTo.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
+      readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false);
+      if (checkRelatesTo && !TimelineReaderUtils.matchRelations(
+          entity.getRelatesToEntities(), relatesTo)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.RELATES_TO)) {
+        entity.getRelatesToEntities().clear();
+      }
+    }
+
+    // fetch info
+    boolean checkInfo = infoFilters != null && infoFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
+      readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false);
+      if (checkInfo &&
+          !TimelineReaderUtils.matchFilters(entity.getInfo(), infoFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.INFO)) {
+        entity.getInfo().clear();
+      }
+    }
+
+    // fetch configs
+    boolean checkConfigs = configFilters != null && configFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
+      readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true);
+      if (checkConfigs && !TimelineReaderUtils.matchFilters(
+          entity.getConfigs(), configFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.CONFIGS)) {
+        entity.getConfigs().clear();
+      }
+    }
+
+    // fetch events
+    boolean checkEvents = eventFilters != null && eventFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
+      readEvents(entity, result, false);
+      if (checkEvents && !TimelineReaderUtils.matchEventFilters(
+          entity.getEvents(), eventFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.EVENTS)) {
+        entity.getEvents().clear();
+      }
+    }
+
+    // fetch metrics
+    boolean checkMetrics = metricFilters != null && metricFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
+      readMetrics(entity, result, EntityColumnPrefix.METRIC);
+      if (checkMetrics && !TimelineReaderUtils.matchMetricFilters(
+          entity.getMetrics(), metricFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.METRICS)) {
+        entity.getMetrics().clear();
+      }
+    }
+    return entity;
+  }
+
+  /**
+   * Helper method for reading relationship.
+   */
+  protected <T> void readRelationship(
+      TimelineEntity entity, Result result, ColumnPrefix<T> prefix,
+      boolean isRelatedTo) throws IOException {
+    // isRelatedTo and relatesTo are of type Map<String, Set<String>>
+    Map<String, Object> columns = prefix.readResults(result);
+    for (Map.Entry<String, Object> column : columns.entrySet()) {
+      for (String id : Separator.VALUES.splitEncoded(
+          column.getValue().toString())) {
+        if (isRelatedTo) {
+          entity.addIsRelatedToEntity(column.getKey(), id);
+        } else {
+          entity.addRelatesToEntity(column.getKey(), id);
+        }
+      }
+    }
+  }
+
+  /**
+   * Helper method for reading key-value pairs for either info or config.
+   */
+  protected <T> void readKeyValuePairs(
+      TimelineEntity entity, Result result, ColumnPrefix<T> prefix,
+      boolean isConfig) throws IOException {
+    // info and configuration are of type Map<String, Object or String>
+    Map<String, Object> columns = prefix.readResults(result);
+    if (isConfig) {
+      for (Map.Entry<String, Object> column : columns.entrySet()) {
+        entity.addConfig(column.getKey(), column.getValue().toString());
+      }
+    } else {
+      entity.addInfo(columns);
+    }
+  }
+
+  /**
+   * Read events from the entity table or the application table. The column name
+   * is of the form "eventId=timestamp=infoKey" where "infoKey" may be omitted
+   * if there is no info associated with the event.
+   *
+   * See {@link EntityTable} and {@link ApplicationTable} for a more detailed
+   * schema description.
+   */
+  protected void readEvents(TimelineEntity entity, Result result,
+      boolean isApplication) throws IOException {
+    Map<String, TimelineEvent> eventsMap = new HashMap<>();
+    Map<?, Object> eventsResult = isApplication ?
+        ApplicationColumnPrefix.EVENT.
+            readResultsHavingCompoundColumnQualifiers(result) :
+        EntityColumnPrefix.EVENT.
+            readResultsHavingCompoundColumnQualifiers(result);
+    for (Map.Entry<?, Object> eventResult : eventsResult.entrySet()) {
+      byte[][] karr = (byte[][])eventResult.getKey();
+      // the column name is of the form "eventId=timestamp=infoKey"
+      if (karr.length == 3) {
+        String id = Bytes.toString(karr[0]);
+        long ts = TimelineWriterUtils.invert(Bytes.toLong(karr[1]));
+        String key = Separator.VALUES.joinEncoded(id, Long.toString(ts));
+        TimelineEvent event = eventsMap.get(key);
+        if (event == null) {
+          event = new TimelineEvent();
+          event.setId(id);
+          event.setTimestamp(ts);
+          eventsMap.put(key, event);
+        }
+        // handle empty info
+        String infoKey = karr[2].length == 0 ? null : Bytes.toString(karr[2]);
+        if (infoKey != null) {
+          event.addInfo(infoKey, eventResult.getValue());
+        }
+      } else {
+        LOG.warn("incorrectly formatted column name: it will be discarded");
+        continue;
+      }
+    }
+    Set<TimelineEvent> eventsSet = new HashSet<>(eventsMap.values());
+    entity.addEvents(eventsSet);
+  }
+}


[28/50] [abbrv] hadoop git commit: YARN-4179. [reader implementation] support flow activity queries based on time (Varun Saxena via sjlee)

Posted by gt...@apache.org.
YARN-4179. [reader implementation] support flow activity queries based on time (Varun Saxena via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: d2ae82d815e5b6737d447d4792c1f6bf5495028c
Parents: f40b1b5
Author: Sangjin Lee <sj...@apache.org>
Authored: Thu Oct 22 17:41:40 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:59 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../timelineservice/FlowActivityEntity.java     |  10 +-
 .../reader/TimelineReaderWebServices.java       | 134 ++++++++++++++++++-
 .../storage/FlowActivityEntityReader.java       |  17 ++-
 .../storage/flow/FlowActivityRowKey.java        |  21 +++
 ...stTimelineReaderWebServicesHBaseStorage.java |  59 ++++++++
 6 files changed, 235 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2ae82d8/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 4ed4949..938e9bd 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -121,6 +121,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3864. Implement support for querying single app and all apps for a
     flow run (Varun Saxena via sjlee)
 
+    YARN-4179. [reader implementation] support flow activity queries based on
+    time (Varun Saxena via sjlee)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2ae82d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java
index 163bd5c..cf19328 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/FlowActivityEntity.java
@@ -140,7 +140,15 @@ public class FlowActivityEntity extends TimelineEntity {
   }
 
   public Date getDate() {
-    return (Date)getInfo().get(DATE_INFO_KEY);
+    Object date = getInfo().get(DATE_INFO_KEY);
+    if (date != null) {
+      if (date instanceof Long) {
+        return new Date((Long)date);
+      } else if (date instanceof Date) {
+        return (Date)date;
+      }
+    }
+    return null;
   }
 
   public void setDate(long time) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2ae82d8/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 83062f3..d82a402 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
@@ -19,12 +19,18 @@
 package org.apache.hadoop.yarn.server.timelineservice.reader;
 
 import java.io.IOException;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
 import java.util.Collections;
+import java.util.Date;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
+import java.util.TimeZone;
 
 import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServletRequest;
@@ -54,6 +60,7 @@ import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.inject.Singleton;
 
 /** REST end point for Timeline Reader */
@@ -70,11 +77,96 @@ public class TimelineReaderWebServices {
   private static final String COMMA_DELIMITER = ",";
   private static final String COLON_DELIMITER = ":";
   private static final String QUERY_STRING_SEP = "?";
+  private static final String RANGE_DELIMITER = "-";
+  private static final String DATE_PATTERN = "yyyyMMdd";
+
+  @VisibleForTesting
+  static ThreadLocal<DateFormat> DATE_FORMAT = new ThreadLocal<DateFormat>() {
+    @Override
+    protected DateFormat initialValue() {
+      SimpleDateFormat format =
+          new SimpleDateFormat(DATE_PATTERN, Locale.ENGLISH);
+      format.setTimeZone(TimeZone.getTimeZone("GMT"));
+      format.setLenient(false);
+      return format;
+    }
+  };
 
   private void init(HttpServletResponse response) {
     response.setContentType(null);
   }
 
+  private static class DateRange {
+    Long dateStart;
+    Long dateEnd;
+    private DateRange(Long start, Long end) {
+      this.dateStart = start;
+      this.dateEnd = end;
+    }
+  }
+
+  private static long parseDate(String strDate) throws ParseException {
+    Date date = DATE_FORMAT.get().parse(strDate);
+    return date.getTime();
+  }
+
+  /**
+   * Parses date range which can be a single date or in the format
+   * "[startdate]-[enddate]" where either of start or end date may not exist.
+   * @param dateRange
+   * @return a {@link DateRange} object.
+   * @throws IllegalArgumentException
+   */
+  private static DateRange parseDateRange(String dateRange)
+      throws IllegalArgumentException {
+    if (dateRange == null || dateRange.isEmpty()) {
+      return new DateRange(null, null);
+    }
+    // Split date range around "-" fetching two components indicating start and
+    // end date.
+    String[] dates = dateRange.split(RANGE_DELIMITER, 2);
+    Long start = null;
+    Long end = null;
+    try {
+      String startDate = dates[0].trim();
+      if (!startDate.isEmpty()) {
+        // Start date is not in yyyyMMdd format.
+        if (startDate.length() != DATE_PATTERN.length()) {
+          throw new IllegalArgumentException("Invalid date range " + dateRange);
+        }
+        // Parse start date which exists before "-" in date range.
+        // If "-" does not exist in date range, this effectively
+        // gives single date.
+        start = parseDate(startDate);
+      }
+      if (dates.length > 1) {
+        String endDate = dates[1].trim();
+        if (!endDate.isEmpty()) {
+          // End date is not in yyyyMMdd format.
+          if (endDate.length() != DATE_PATTERN.length()) {
+            throw new IllegalArgumentException(
+                "Invalid date range " + dateRange);
+          }
+          // Parse end date which exists after "-" in date range.
+          end = parseDate(endDate);
+        }
+      } else {
+        // Its a single date(without "-" in date range), so set
+        // end equal to start.
+        end = start;
+      }
+      if (start != null && end != null) {
+        if (start > end) {
+          throw new IllegalArgumentException("Invalid date range " + dateRange);
+        }
+      }
+      return new DateRange(start, end);
+    } catch (ParseException e) {
+      // Date could not be parsed.
+      throw new IllegalArgumentException("Invalid date range " + dateRange);
+    }
+  }
+
   private static Set<String> parseValuesStr(String str, String delimiter) {
     if (str == null || str.isEmpty()) {
       return null;
@@ -205,7 +297,8 @@ public class TimelineReaderWebServices {
     if (e instanceof NumberFormatException) {
       throw new BadRequestException(invalidNumMsg + " is not a numeric value.");
     } else if (e instanceof IllegalArgumentException) {
-      throw new BadRequestException("Requested Invalid Field.");
+      throw new BadRequestException(e.getMessage() == null ?
+          "Requested Invalid Field." : e.getMessage());
     } else {
       LOG.error("Error while processing REST request", e);
       throw new WebApplicationException(e,
@@ -514,8 +607,20 @@ public class TimelineReaderWebServices {
   }
 
   /**
-   * Return a list of flows for a given cluster id. Cluster ID is not
-   * provided by client so default cluster ID has to be taken.
+   * Return a list of flows. Cluster ID is not provided by client so default
+   * cluster ID has to be taken. daterange, if specified is given as
+   * "[startdate]-[enddate]"(i.e. start and end date separated by -) or
+   * single date. Dates are interpreted in yyyyMMdd format and are assumed to
+   * be in GMT. If a single date is specified, all flows active on that date are
+   * returned. If both startdate and enddate is given, all flows active between
+   * start and end date will be returned. If only startdate is given, flows
+   * active on and after startdate are returned. If only enddate is given, flows
+   * active on and before enddate are returned.
+   * For example :
+   * "daterange=20150711" returns flows active on 20150711.
+   * "daterange=20150711-20150714" returns flows active between these 2 dates.
+   * "daterange=20150711-" returns flows active on and after 20150711.
+   * "daterange=-20150711" returns flows active on and before 20150711.
    */
   @GET
   @Path("/flows/")
@@ -524,12 +629,25 @@ public class TimelineReaderWebServices {
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @QueryParam("limit") String limit,
+      @QueryParam("daterange") String dateRange,
       @QueryParam("fields") String fields) {
-    return getFlows(req, res, null, limit, fields);
+    return getFlows(req, res, null, limit, dateRange, fields);
   }
 
   /**
-   * Return a list of flows for a given cluster id.
+   * Return a list of flows for a given cluster id. daterange, if specified is
+   * given as "[startdate]-[enddate]"(i.e. start and end date separated by -) or
+   * single date. Dates are interpreted in yyyyMMdd format and are assumed to
+   * be in GMT. If a single date is specified, all flows active on that date are
+   * returned. If both startdate and enddate is given, all flows active between
+   * start and end date will be returned. If only startdate is given, flows
+   * active on and after startdate are returned. If only enddate is given, flows
+   * active on and before enddate are returned.
+   * For example :
+   * "daterange=20150711" returns flows active on 20150711.
+   * "daterange=20150711-20150714" returns flows active between these 2 dates.
+   * "daterange=20150711-" returns flows active on and after 20150711.
+   * "daterange=-20150711" returns flows active on and before 20150711.
    */
   @GET
   @Path("/flows/{clusterid}/")
@@ -539,6 +657,7 @@ public class TimelineReaderWebServices {
       @Context HttpServletResponse res,
       @PathParam("clusterid") String clusterId,
       @QueryParam("limit") String limit,
+      @QueryParam("daterange") String dateRange,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -550,11 +669,12 @@ public class TimelineReaderWebServices {
     TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
     Set<TimelineEntity> entities = null;
     try {
+      DateRange range = parseDateRange(dateRange);
       entities = timelineReaderManager.getEntities(
           null, parseStr(clusterId), null, null, null,
           TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), parseLongStr(limit),
-          null, null, null, null, null, null, null, null, null, null,
-          parseFieldsStr(fields, COMMA_DELIMITER));
+          range.dateStart, range.dateEnd, null, null, null, null, null, null,
+          null, null, parseFieldsStr(fields, COMMA_DELIMITER));
     } catch (Exception e) {
       handleException(e, url, startTime, "limit");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2ae82d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.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/FlowActivityEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
index 70a0915..3e32128 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
@@ -87,6 +87,12 @@ class FlowActivityEntityReader extends TimelineEntityReader {
     if (limit == null || limit < 0) {
       limit = TimelineReader.DEFAULT_LIMIT;
     }
+    if (createdTimeBegin == null) {
+      createdTimeBegin = DEFAULT_BEGIN_TIME;
+    }
+    if (createdTimeEnd == null) {
+      createdTimeEnd = DEFAULT_END_TIME;
+    }
   }
 
   @Override
@@ -100,7 +106,16 @@ class FlowActivityEntityReader extends TimelineEntityReader {
   protected ResultScanner getResults(Configuration hbaseConf,
       Connection conn) throws IOException {
     Scan scan = new Scan();
-    scan.setRowPrefixFilter(FlowActivityRowKey.getRowKeyPrefix(clusterId));
+    if (createdTimeBegin == DEFAULT_BEGIN_TIME &&
+        createdTimeEnd == DEFAULT_END_TIME) {
+      scan.setRowPrefixFilter(FlowActivityRowKey.getRowKeyPrefix(clusterId));
+    } else {
+      scan.setStartRow(
+          FlowActivityRowKey.getRowKeyPrefix(clusterId, createdTimeEnd));
+      scan.setStopRow(
+          FlowActivityRowKey.getRowKeyPrefix(clusterId,
+              (createdTimeBegin <= 0 ? 0: (createdTimeBegin - 1))));
+    }
     // use the page filter to limit the result to the page size
     // the scanner may still return more than the limit; therefore we need to
     // read the right number as we iterate

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2ae82d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
index f7841e0..fc1aa70 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
@@ -55,11 +55,32 @@ public class FlowActivityRowKey {
     return flowId;
   }
 
+  /**
+   * Constructs a row key prefix for the flow activity table as follows:
+   * {@code clusterId!}
+   *
+   * @param clusterId
+   * @return byte array with the row key prefix
+   */
   public static byte[] getRowKeyPrefix(String clusterId) {
     return Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId, ""));
   }
 
   /**
+   * Constructs a row key prefix for the flow activity table as follows:
+   * {@code clusterId!dayTimestamp!}
+   *
+   * @param clusterId
+   * @param dayTs
+   * @return byte array with the row key prefix
+   */
+  public static byte[] getRowKeyPrefix(String clusterId, long dayTs) {
+    return Separator.QUALIFIERS.join(
+        Bytes.toBytes(Separator.QUALIFIERS.encode(clusterId)),
+        Bytes.toBytes(TimelineStorageUtils.invertLong(dayTs)), new byte[0]);
+  }
+
+  /**
    * Constructs a row key for the flow activity table as follows:
    * {@code clusterId!dayTimestamp!user!flowId}
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d2ae82d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
index f6a5090..4f53fe2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -27,6 +27,7 @@ import java.lang.reflect.UndeclaredThrowableException;
 import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URL;
+import java.text.DateFormat;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -48,6 +49,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 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.TimelineStorageUtils;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -70,6 +72,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   private TimelineReaderServer server;
   private static HBaseTestingUtility util;
   private static long ts = System.currentTimeMillis();
+  private static long dayTs =
+      TimelineStorageUtils.getTopOfTheDayTimestamp(ts);
 
   @BeforeClass
   public static void setup() throws Exception {
@@ -509,6 +513,61 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
       assertNotNull(entities);
       assertEquals(1, entities.size());
+
+      DateFormat fmt = TimelineReaderWebServices.DATE_FORMAT.get();
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows/cluster1?daterange=" + fmt.format(dayTs) + "-" +
+          fmt.format(dayTs + (2*86400000L)));
+      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));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows/cluster1?daterange=" +
+          fmt.format(dayTs + (4*86400000L)));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows/cluster1?daterange=-" +
+          fmt.format(dayTs + (2*86400000L)));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows/cluster1?daterange=" +
+          fmt.format(dayTs - (2*86400000L)) + "-");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows/cluster1?daterange=20150711:20150714");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows/cluster1?daterange=20150714-20150711");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows/cluster1?daterange=2015071129-20150712");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows/cluster1?daterange=20150711-2015071243");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
     } finally {
       client.destroy();
     }


[31/50] [abbrv] hadoop git commit: YARN-4053. Change the way metric values are stored in HBase Storage (Varun Saxena via sjlee)

Posted by gt...@apache.org.
YARN-4053. Change the way metric values are stored in HBase Storage (Varun Saxena via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: e6f57ecc03d63eb7183bfc76cfafbc3f5023b8a8
Parents: af14edb
Author: Sangjin Lee <sj...@apache.org>
Authored: Fri Nov 20 10:03:02 2015 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:59:39 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../storage/FlowRunEntityReader.java            |   4 +-
 .../application/ApplicationColumnPrefix.java    |  23 +++-
 .../storage/common/ColumnHelper.java            |  27 ++++-
 .../storage/common/GenericConverter.java        |  48 ++++++++
 .../storage/common/LongConverter.java           |  78 +++++++++++++
 .../storage/common/NumericValueConverter.java   |  38 +++++++
 .../storage/common/TimelineStorageUtils.java    |  11 ++
 .../storage/common/ValueConverter.java          |  45 ++++++++
 .../storage/common/package-info.java            |  28 +++++
 .../storage/entity/EntityColumn.java            |   1 -
 .../storage/entity/EntityColumnPrefix.java      |  25 ++++-
 .../storage/flow/FlowRunColumn.java             |  24 +++-
 .../storage/flow/FlowRunColumnPrefix.java       |  21 +++-
 .../storage/flow/FlowScanner.java               | 110 ++++++++++++++-----
 .../storage/TestHBaseTimelineStorage.java       |  81 +++++++++++++-
 .../storage/flow/TestHBaseStorageFlowRun.java   |  15 +--
 17 files changed, 524 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index fdd0d83..74f58f6 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -140,6 +140,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
 
     YARN-4221. Store user in app to flow table (Varun Saxena via sjlee)
 
+    YARN-4053. Change the way metric values are stored in HBase Storage (Varun
+    Saxena via sjlee)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/FlowRunEntityReader.java
index c4b4e91..ebf2d27 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/FlowRunEntityReader.java
@@ -137,7 +137,7 @@ class FlowRunEntityReader extends TimelineEntityReader {
     }
 
     // read the start time
-    Number startTime = (Number)FlowRunColumn.MIN_START_TIME.readResult(result);
+    Long startTime = (Long)FlowRunColumn.MIN_START_TIME.readResult(result);
     if (startTime != null) {
       flowRun.setStartTime(startTime.longValue());
     }
@@ -147,7 +147,7 @@ class FlowRunEntityReader extends TimelineEntityReader {
     }
 
     // read the end time if available
-    Number endTime = (Number)FlowRunColumn.MAX_END_TIME.readResult(result);
+    Long endTime = (Long)FlowRunColumn.MAX_END_TIME.readResult(result);
     if (endTime != null) {
       flowRun.setMaxEndTime(endTime.longValue());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
index d7b5773..b06f5c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
@@ -26,8 +26,11 @@ 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.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
 /**
@@ -63,7 +66,8 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
   /**
    * Metrics are stored with the metric name as the column name.
    */
-  METRIC(ApplicationColumnFamily.METRICS, null);
+  METRIC(ApplicationColumnFamily.METRICS, null,
+      LongConverter.getInstance());
 
   private final ColumnHelper<ApplicationTable> column;
   private final ColumnFamily<ApplicationTable> columnFamily;
@@ -83,7 +87,20 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
    */
   private ApplicationColumnPrefix(ColumnFamily<ApplicationTable> columnFamily,
       String columnPrefix) {
-    column = new ColumnHelper<ApplicationTable>(columnFamily);
+    this(columnFamily, columnPrefix, GenericConverter.getInstance());
+  }
+
+  /**
+   * 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.
+   */
+  private ApplicationColumnPrefix(ColumnFamily<ApplicationTable> columnFamily,
+      String columnPrefix, ValueConverter converter) {
+    column = new ColumnHelper<ApplicationTable>(columnFamily, converter);
     this.columnFamily = columnFamily;
     this.columnPrefix = columnPrefix;
     if (columnPrefix == null) {
@@ -127,7 +144,7 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
 
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         attributes);
- }
+  }
 
   /*
    * (non-Javadoc)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/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 3a2e088..1e63ce5 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
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 /**
@@ -50,9 +49,20 @@ public class ColumnHelper<T> {
    */
   private final byte[] columnFamilyBytes;
 
+  private final ValueConverter converter;
+
   public ColumnHelper(ColumnFamily<T> columnFamily) {
+    this(columnFamily, GenericConverter.getInstance());
+  }
+
+  public ColumnHelper(ColumnFamily<T> columnFamily, ValueConverter converter) {
     this.columnFamily = columnFamily;
     columnFamilyBytes = columnFamily.getBytes();
+    if (converter == null) {
+      this.converter = GenericConverter.getInstance();
+    } else {
+      this.converter = converter;
+    }
   }
 
   /**
@@ -83,7 +93,7 @@ public class ColumnHelper<T> {
     Put p = new Put(rowKey);
     timestamp = getPutTimestamp(timestamp, attributes);
     p.addColumn(columnFamilyBytes, columnQualifier, timestamp,
-        GenericObjectMapper.write(inputValue));
+        converter.encodeValue(inputValue));
     if ((attributes != null) && (attributes.length > 0)) {
       for (Attribute attribute : attributes) {
         p.setAttribute(attribute.getName(), attribute.getValue());
@@ -148,7 +158,7 @@ public class ColumnHelper<T> {
     // ByteBuffer to avoid copy, but GenericObjectMapper doesn't seem to like
     // that.
     byte[] value = result.getValue(columnFamilyBytes, columnQualifierBytes);
-    return GenericObjectMapper.read(value);
+    return converter.decodeValue(value);
   }
 
   /**
@@ -206,7 +216,7 @@ public class ColumnHelper<T> {
             if (cells != null) {
               for (Entry<Long, byte[]> cell : cells.entrySet()) {
                 V value =
-                    (V) GenericObjectMapper.read(cell.getValue());
+                    (V) converter.decodeValue(cell.getValue());
                 cellResults.put(
                     TimestampGenerator.getTruncatedTimestamp(cell.getKey()),
                     value);
@@ -266,7 +276,7 @@ public class ColumnHelper<T> {
 
           // If this column has the prefix we want
           if (columnName != null) {
-            Object value = GenericObjectMapper.read(entry.getValue());
+            Object value = converter.decodeValue(entry.getValue());
             results.put(columnName, value);
           }
         }
@@ -313,7 +323,7 @@ public class ColumnHelper<T> {
               // This is the prefix that we want
               byte[][] columnQualifierParts =
                   Separator.VALUES.split(columnNameParts[1]);
-              Object value = GenericObjectMapper.read(entry.getValue());
+              Object value = converter.decodeValue(entry.getValue());
               // we return the columnQualifier in parts since we don't know
               // which part is of which data type
               results.put(columnQualifierParts, value);
@@ -371,6 +381,11 @@ public class ColumnHelper<T> {
         Separator.QUALIFIERS.join(columnPrefixBytes, Bytes.toBytes(qualifier));
     return columnQualifier;
   }
+
+  public ValueConverter getValueConverter() {
+    return converter;
+  }
+
   /**
    * @param columnPrefixBytes The byte representation for the column prefix.
    *          Should not contain {@link Separator#QUALIFIERS}.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/GenericConverter.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/GenericConverter.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/GenericConverter.java
new file mode 100644
index 0000000..c34bfcb
--- /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/common/GenericConverter.java
@@ -0,0 +1,48 @@
+/**
+ * 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 java.io.IOException;
+
+import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
+
+/**
+ * Uses GenericObjectMapper to encode objects as bytes and decode bytes as
+ * objects.
+ */
+public final class GenericConverter implements ValueConverter {
+  private static final GenericConverter INSTANCE = new GenericConverter();
+
+  private GenericConverter() {
+  }
+
+  public static GenericConverter getInstance() {
+    return INSTANCE;
+  }
+
+  @Override
+  public byte[] encodeValue(Object value) throws IOException {
+    return GenericObjectMapper.write(value);
+  }
+
+  @Override
+  public Object decodeValue(byte[] bytes) throws IOException {
+    return GenericObjectMapper.read(bytes);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.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/LongConverter.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/LongConverter.java
new file mode 100644
index 0000000..cdb8619
--- /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/common/LongConverter.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.storage.common;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Encodes a value by interpreting it as a Long and converting it to bytes and
+ * decodes a set of bytes as a Long.
+ */
+public final class LongConverter implements NumericValueConverter {
+  private static final LongConverter INSTANCE = new LongConverter();
+
+  private LongConverter() {
+  }
+
+  public static LongConverter getInstance() {
+    return INSTANCE;
+  }
+
+  @Override
+  public byte[] encodeValue(Object value) throws IOException {
+    if (!TimelineStorageUtils.isIntegralValue(value)) {
+      throw new IOException("Expected integral value");
+    }
+    return Bytes.toBytes(((Number)value).longValue());
+  }
+
+  @Override
+  public Object decodeValue(byte[] bytes) throws IOException {
+    if (bytes == null) {
+      return null;
+    }
+    return Bytes.toLong(bytes);
+  }
+
+  /**
+   * Compares two numbers as longs. If either number is null, it will be taken
+   * as 0.
+   * @param num1
+   * @param num2
+   * @return -1 if num1 is less than num2, 0 if num1 is equal to num2 and 1 if
+   * num1 is greater than num2.
+   */
+  @Override
+  public int compare(Number num1, Number num2) {
+    return Long.compare((num1 == null) ? 0L : num1.longValue(),
+        (num2 == null) ? 0L : num2.longValue());
+  }
+
+  @Override
+  public Number add(Number num1, Number num2, Number...numbers) {
+    long sum = ((num1 == null) ? 0L : num1.longValue()) +
+        ((num2 == null) ? 0L : num2.longValue());
+    for (Number num : numbers) {
+      sum = sum + ((num == null) ? 0L : num.longValue());
+    }
+    return sum;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.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/NumericValueConverter.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/NumericValueConverter.java
new file mode 100644
index 0000000..70964cd
--- /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/common/NumericValueConverter.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;
+
+import java.util.Comparator;
+
+/**
+ * Extends ValueConverter interface for numeric converters to support numerical
+ * operations such as comparison, addition, etc.
+ */
+public interface NumericValueConverter extends ValueConverter,
+    Comparator<Number> {
+  /**
+   * Adds two or more numbers. If either of the numbers are null, it is taken as
+   * 0.
+   * @param num1
+   * @param num2
+   * @param numbers
+   * @return result after adding up the numbers.
+   */
+  Number add(Number num1, Number num2, Number...numbers);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineStorageUtils.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/TimelineStorageUtils.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/TimelineStorageUtils.java
index c1aaf19..e30f699 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/TimelineStorageUtils.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/TimelineStorageUtils.java
@@ -472,4 +472,15 @@ public class TimelineStorageUtils {
     }
     return true;
   }
+
+  /**
+   * Checks if passed object is of integral type(Short/Integer/Long).
+   * @param obj
+   * @return true if object passed is of type Short or Integer or Long, false
+   * otherwise.
+   */
+  public static boolean isIntegralValue(Object obj) {
+    return (obj instanceof Short) || (obj instanceof Integer) ||
+        (obj instanceof Long);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.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/ValueConverter.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/ValueConverter.java
new file mode 100644
index 0000000..2388ba5
--- /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/common/ValueConverter.java
@@ -0,0 +1,45 @@
+/**
+ * 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 java.io.IOException;
+
+/**
+ * Converter used to encode/decode value associated with a column prefix or a
+ * column.
+ */
+public interface ValueConverter {
+
+  /**
+   * Encode an object as a byte array depending on the converter implementation.
+   * @param value
+   * @return a byte array
+   * @throws IOException
+   */
+  byte[] encodeValue(Object value) throws IOException;
+
+  /**
+   * Decode a byte array and convert it into an object depending on the
+   * converter implementation.
+   * @param bytes
+   * @return an object
+   * @throws IOException
+   */
+  Object decodeValue(byte[] bytes) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/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/storage/common/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/storage/common/package-info.java
new file mode 100644
index 0000000..0df5b8a
--- /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/common/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.common contains
+ * a set of utility classes used across backend storage reader and writer.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.common;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java
index 8ae19b8..e12b6e0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage.entity;
 
 import java.io.IOException;
-import java.util.Map;
 
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
index 0d4e5a8..abede9c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
@@ -26,8 +26,11 @@ 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.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
 /**
@@ -63,7 +66,8 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
   /**
    * Metrics are stored with the metric name as the column name.
    */
-  METRIC(EntityColumnFamily.METRICS, null);
+  METRIC(EntityColumnFamily.METRICS, null,
+      LongConverter.getInstance());
 
   private final ColumnHelper<EntityTable> column;
   private final ColumnFamily<EntityTable> columnFamily;
@@ -83,7 +87,20 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    */
   EntityColumnPrefix(ColumnFamily<EntityTable> columnFamily,
       String columnPrefix) {
-    column = new ColumnHelper<EntityTable>(columnFamily);
+    this(columnFamily, columnPrefix, GenericConverter.getInstance());
+  }
+
+  /**
+   * 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.
+   */
+  EntityColumnPrefix(ColumnFamily<EntityTable> columnFamily,
+      String columnPrefix, ValueConverter converter) {
+    column = new ColumnHelper<EntityTable>(columnFamily, converter);
     this.columnFamily = columnFamily;
     this.columnPrefix = columnPrefix;
     if (columnPrefix == null) {
@@ -128,7 +145,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
 
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         attributes);
- }
+  }
 
   /*
    * (non-Javadoc)
@@ -155,7 +172,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
 
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         attributes);
- }
+  }
 
   /*
    * (non-Javadoc)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java
index 5079cc0..148a37f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java
@@ -24,9 +24,12 @@ 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.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 
 /**
  * Identifies fully qualified columns for the {@link FlowRunTable}.
@@ -38,14 +41,14 @@ public enum FlowRunColumn implements Column<FlowRunTable> {
    * application start times.
    */
   MIN_START_TIME(FlowRunColumnFamily.INFO, "min_start_time",
-      AggregationOperation.MIN),
+      AggregationOperation.MIN, LongConverter.getInstance()),
 
   /**
    * When the flow ended. This is the maximum of currently known application end
    * times.
    */
   MAX_END_TIME(FlowRunColumnFamily.INFO, "max_end_time",
-      AggregationOperation.MAX),
+      AggregationOperation.MAX, LongConverter.getInstance()),
 
   /**
    * The version of the flow that this flow belongs to.
@@ -60,13 +63,20 @@ public enum FlowRunColumn implements Column<FlowRunTable> {
 
   private FlowRunColumn(ColumnFamily<FlowRunTable> columnFamily,
       String columnQualifier, AggregationOperation aggOp) {
+    this(columnFamily, columnQualifier, aggOp,
+        GenericConverter.getInstance());
+  }
+
+  private FlowRunColumn(ColumnFamily<FlowRunTable> columnFamily,
+      String columnQualifier, AggregationOperation aggOp,
+      ValueConverter converter) {
     this.columnFamily = columnFamily;
     this.columnQualifier = columnQualifier;
     this.aggOp = aggOp;
     // Future-proof by ensuring the right column prefix hygiene.
     this.columnQualifierBytes = Bytes.toBytes(Separator.SPACE
         .encode(columnQualifier));
-    this.column = new ColumnHelper<FlowRunTable>(columnFamily);
+    this.column = new ColumnHelper<FlowRunTable>(columnFamily, converter);
   }
 
   /**
@@ -80,6 +90,10 @@ public enum FlowRunColumn implements Column<FlowRunTable> {
     return columnQualifierBytes.clone();
   }
 
+  public byte[] getColumnFamilyBytes() {
+    return columnFamily.getBytes();
+  }
+
   public AggregationOperation getAggregationOperation() {
     return aggOp;
   }
@@ -130,6 +144,10 @@ public enum FlowRunColumn implements Column<FlowRunTable> {
     return null;
   }
 
+  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)}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java
index b090bba..eb055fe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java
@@ -29,6 +29,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 
 /**
  * Identifies partially qualified columns for the {@link FlowRunTable}.
@@ -38,7 +40,8 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
   /**
    * To store flow run info values.
    */
-  METRIC(FlowRunColumnFamily.INFO, "m", AggregationOperation.SUM);
+  METRIC(FlowRunColumnFamily.INFO, "m", AggregationOperation.SUM,
+      LongConverter.getInstance());
 
   private final ColumnHelper<FlowRunTable> column;
   private final ColumnFamily<FlowRunTable> columnFamily;
@@ -61,8 +64,8 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
    *          for this column.
    */
   private FlowRunColumnPrefix(ColumnFamily<FlowRunTable> columnFamily,
-      String columnPrefix, AggregationOperation fra) {
-    column = new ColumnHelper<FlowRunTable>(columnFamily);
+      String columnPrefix, AggregationOperation fra, ValueConverter converter) {
+    column = new ColumnHelper<FlowRunTable>(columnFamily, converter);
     this.columnFamily = columnFamily;
     this.columnPrefix = columnPrefix;
     if (columnPrefix == null) {
@@ -86,6 +89,14 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
     return columnPrefixBytes.clone();
   }
 
+  public byte[] getColumnPrefixBytes(String qualifier) {
+    return ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
+  }
+
+  public byte[] getColumnFamilyBytes() {
+    return columnFamily.getBytes();
+  }
+
   public AggregationOperation getAttribute() {
     return aggOp;
   }
@@ -205,6 +216,10 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
     return null;
   }
 
+  public ValueConverter getValueConverter() {
+    return column.getValueConverter();
+  }
+
   /**
    * Retrieve an {@link FlowRunColumnPrefix} given a name, or null if there is
    * no match. The following holds true:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.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/flow/FlowScanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
index a537891..d541df0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
@@ -39,8 +39,10 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
-import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.NumericValueConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 
 /**
  * Invoked via the coprocessor when a Get or a Scan is issued for flow run
@@ -114,6 +116,45 @@ class FlowScanner implements RegionScanner, Closeable {
   }
 
   /**
+   * Get value converter associated with a column or a column prefix. If nothing
+   * matches, generic converter is returned.
+   * @param colQualifierBytes
+   * @return value converter implementation.
+   */
+  private static ValueConverter getValueConverter(byte[] colQualifierBytes) {
+    // Iterate over all the column prefixes for flow run table and get the
+    // appropriate converter for the column qualifier passed if prefix matches.
+    for (FlowRunColumnPrefix colPrefix : FlowRunColumnPrefix.values()) {
+      byte[] colPrefixBytes = colPrefix.getColumnPrefixBytes("");
+      if (Bytes.compareTo(colPrefixBytes, 0, colPrefixBytes.length,
+          colQualifierBytes, 0, colPrefixBytes.length) == 0) {
+        return colPrefix.getValueConverter();
+      }
+    }
+    // Iterate over all the columns for flow run table and get the
+    // appropriate converter for the column qualifier passed if match occurs.
+    for (FlowRunColumn column : FlowRunColumn.values()) {
+      if (Bytes.compareTo(
+          column.getColumnQualifierBytes(), colQualifierBytes) == 0) {
+        return column.getValueConverter();
+      }
+    }
+    // Return generic converter if nothing matches.
+    return GenericConverter.getInstance();
+  }
+
+  /**
+   * Checks if the converter is a numeric converter or not. For a converter to
+   * be numeric, it must implement {@link NumericValueConverter} interface.
+   * @param converter
+   * @return true, if converter is of type NumericValueConverter, false
+   * otherwise.
+   */
+  private static boolean isNumericConverter(ValueConverter converter) {
+    return (converter instanceof NumericValueConverter);
+  }
+
+  /**
    * This method loops through the cells in a given row of the
    * {@link FlowRunTable}. It looks at the tags of each cell to figure out how
    * to process the contents. It then calculates the sum or min or max for each
@@ -141,20 +182,32 @@ class FlowScanner implements RegionScanner, Closeable {
     SortedSet<Cell> currentColumnCells = new TreeSet<>(KeyValue.COMPARATOR);
     Set<String> alreadySeenAggDim = new HashSet<>();
     int addedCnt = 0;
+    ValueConverter converter = null;
     while (((cell = peekAtNextCell(limit)) != null)
         && (limit <= 0 || addedCnt < limit)) {
       byte[] newColumnQualifier = CellUtil.cloneQualifier(cell);
       if (comp.compare(currentColumnQualifier, newColumnQualifier) != 0) {
-        addedCnt += emitCells(cells, currentColumnCells, currentAggOp);
+        if (converter != null && isNumericConverter(converter)) {
+          addedCnt += emitCells(cells, currentColumnCells, currentAggOp,
+              (NumericValueConverter)converter);
+        }
         resetState(currentColumnCells, alreadySeenAggDim);
         currentColumnQualifier = newColumnQualifier;
         currentAggOp = getCurrentAggOp(cell);
+        converter = getValueConverter(newColumnQualifier);
       }
-      collectCells(currentColumnCells, currentAggOp, cell, alreadySeenAggDim);
+      // No operation needs to be performed on non numeric converters.
+      if (!isNumericConverter(converter)) {
+        nextCell(limit);
+        continue;
+      }
+      collectCells(currentColumnCells, currentAggOp, cell, alreadySeenAggDim,
+          (NumericValueConverter)converter);
       nextCell(limit);
     }
     if (!currentColumnCells.isEmpty()) {
-      emitCells(cells, currentColumnCells, currentAggOp);
+      emitCells(cells, currentColumnCells, currentAggOp,
+          (NumericValueConverter)converter);
     }
     return hasMore();
   }
@@ -183,7 +236,8 @@ class FlowScanner implements RegionScanner, Closeable {
 
   private void collectCells(SortedSet<Cell> currentColumnCells,
       AggregationOperation currentAggOp, Cell cell,
-      Set<String> alreadySeenAggDim) throws IOException {
+      Set<String> alreadySeenAggDim, NumericValueConverter converter)
+      throws IOException {
     if (currentAggOp == null) {
       // not a min/max/metric cell, so just return it as is
       currentColumnCells.add(cell);
@@ -197,7 +251,8 @@ class FlowScanner implements RegionScanner, Closeable {
         currentColumnCells.add(cell);
       } else {
         Cell currentMinCell = currentColumnCells.first();
-        Cell newMinCell = compareCellValues(currentMinCell, cell, currentAggOp);
+        Cell newMinCell = compareCellValues(currentMinCell, cell, currentAggOp,
+            converter);
         if (!currentMinCell.equals(newMinCell)) {
           currentColumnCells.remove(currentMinCell);
           currentColumnCells.add(newMinCell);
@@ -209,7 +264,8 @@ class FlowScanner implements RegionScanner, Closeable {
         currentColumnCells.add(cell);
       } else {
         Cell currentMaxCell = currentColumnCells.first();
-        Cell newMaxCell = compareCellValues(currentMaxCell, cell, currentAggOp);
+        Cell newMaxCell = compareCellValues(currentMaxCell, cell, currentAggOp,
+            converter);
         if (!currentMaxCell.equals(newMaxCell)) {
           currentColumnCells.remove(currentMaxCell);
           currentColumnCells.add(newMaxCell);
@@ -245,7 +301,8 @@ class FlowScanner implements RegionScanner, Closeable {
    * parameter.
    */
   private int emitCells(List<Cell> cells, SortedSet<Cell> currentColumnCells,
-      AggregationOperation currentAggOp) throws IOException {
+      AggregationOperation currentAggOp, NumericValueConverter converter)
+      throws IOException {
     if ((currentColumnCells == null) || (currentColumnCells.size() == 0)) {
       return 0;
     }
@@ -261,7 +318,7 @@ class FlowScanner implements RegionScanner, Closeable {
       return currentColumnCells.size();
     case SUM:
     case SUM_FINAL:
-      Cell sumCell = processSummation(currentColumnCells);
+      Cell sumCell = processSummation(currentColumnCells, converter);
       cells.add(sumCell);
       return 1;
     default:
@@ -276,24 +333,24 @@ class FlowScanner implements RegionScanner, Closeable {
    * sum of a metric for a flow run is the summation at the point of the last
    * metric update in that flow till that time.
    */
-  private Cell processSummation(SortedSet<Cell> currentColumnCells)
-      throws IOException {
+  private Cell processSummation(SortedSet<Cell> currentColumnCells,
+      NumericValueConverter converter) throws IOException {
     Number sum = 0;
     Number currentValue = 0;
     long ts = 0L;
-    long mostCurrentTimestamp = 0l;
+    long mostCurrentTimestamp = 0L;
     Cell mostRecentCell = null;
     for (Cell cell : currentColumnCells) {
-      currentValue = (Number) GenericObjectMapper.read(CellUtil
-          .cloneValue(cell));
+      currentValue = (Number) converter.decodeValue(CellUtil.cloneValue(cell));
       ts = cell.getTimestamp();
       if (mostCurrentTimestamp < ts) {
         mostCurrentTimestamp = ts;
         mostRecentCell = cell;
       }
-      sum = sum.longValue() + currentValue.longValue();
+      sum = converter.add(sum, currentValue);
     }
-    Cell sumCell = createNewCell(mostRecentCell, sum);
+    byte[] sumBytes = converter.encodeValue(sum);
+    Cell sumCell = createNewCell(mostRecentCell, sumBytes);
     return sumCell;
   }
 
@@ -308,18 +365,20 @@ class FlowScanner implements RegionScanner, Closeable {
    * @throws IOException
    */
   private Cell compareCellValues(Cell previouslyChosenCell, Cell currentCell,
-      AggregationOperation currentAggOp) throws IOException {
+      AggregationOperation currentAggOp, NumericValueConverter converter)
+      throws IOException {
     if (previouslyChosenCell == null) {
       return currentCell;
     }
     try {
-      long previouslyChosenCellValue = ((Number) GenericObjectMapper
-          .read(CellUtil.cloneValue(previouslyChosenCell))).longValue();
-      long currentCellValue = ((Number) GenericObjectMapper.read(CellUtil
-          .cloneValue(currentCell))).longValue();
+      Number previouslyChosenCellValue = (Number)converter.decodeValue(
+          CellUtil.cloneValue(previouslyChosenCell));
+      Number currentCellValue = (Number) converter.decodeValue(CellUtil
+          .cloneValue(currentCell));
       switch (currentAggOp) {
       case MIN:
-        if (currentCellValue < previouslyChosenCellValue) {
+        if (converter.compare(
+            currentCellValue, previouslyChosenCellValue) < 0) {
           // new value is minimum, hence return this cell
           return currentCell;
         } else {
@@ -327,7 +386,8 @@ class FlowScanner implements RegionScanner, Closeable {
           return previouslyChosenCell;
         }
       case MAX:
-        if (currentCellValue > previouslyChosenCellValue) {
+        if (converter.compare(
+            currentCellValue, previouslyChosenCellValue) > 0) {
           // new value is max, hence return this cell
           return currentCell;
         } else {
@@ -343,8 +403,8 @@ class FlowScanner implements RegionScanner, Closeable {
     }
   }
 
-  private Cell createNewCell(Cell origCell, Number number) throws IOException {
-    byte[] newValue = GenericObjectMapper.write(number);
+  private Cell createNewCell(Cell origCell, byte[] newValue)
+      throws IOException {
     return CellUtil.createCell(CellUtil.cloneRow(origCell),
         CellUtil.cloneFamily(origCell), CellUtil.cloneQualifier(origCell),
         origCell.getTimestamp(), KeyValue.Type.Put.getCode(), newValue);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.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/TestHBaseTimelineStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index 701615e..30ead40 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -61,6 +61,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.EntityTable;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -90,6 +91,15 @@ public class TestHBaseTimelineStorage {
     TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
   }
 
+  private static void matchMetrics(Map<Long, Number> m1, Map<Long, Number> m2) {
+    assertEquals(m1.size(), m2.size());
+    for (Map.Entry<Long, Number> entry : m2.entrySet()) {
+      Number val = m1.get(entry.getKey());
+      assertNotNull(val);
+      assertEquals(val.longValue(), entry.getValue().longValue());
+    }
+  }
+
   @Test
   public void testWriteApplicationToHBase() throws Exception {
     TimelineEntities te = new TimelineEntities();
@@ -243,7 +253,7 @@ public class TestHBaseTimelineStorage {
           ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result);
 
       NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
-      assertEquals(metricValues, metricMap);
+      matchMetrics(metricValues, metricMap);
 
       // read the timeline entity using the reader this time
       TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appId,
@@ -273,7 +283,7 @@ public class TestHBaseTimelineStorage {
       assertEquals(metrics, metrics2);
       for (TimelineMetric metric2 : metrics2) {
         Map<Long, Number> metricValues2 = metric2.getValues();
-        assertEquals(metricValues, metricValues2);
+        matchMetrics(metricValues, metricValues2);
       }
     } finally {
       if (hbi != null) {
@@ -451,7 +461,7 @@ public class TestHBaseTimelineStorage {
               EntityColumnPrefix.METRIC.readResultsWithTimestamps(result);
 
           NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
-          assertEquals(metricValues, metricMap);
+          matchMetrics(metricValues, metricMap);
         }
       }
       assertEquals(1, rowCount);
@@ -488,7 +498,7 @@ public class TestHBaseTimelineStorage {
       assertEquals(metrics, metrics2);
       for (TimelineMetric metric2 : metrics2) {
         Map<Long, Number> metricValues2 = metric2.getValues();
-        assertEquals(metricValues, metricValues2);
+        matchMetrics(metricValues, metricValues2);
       }
     } finally {
       if (hbi != null) {
@@ -743,6 +753,69 @@ public class TestHBaseTimelineStorage {
     }
   }
 
+  @Test
+  public void testNonIntegralMetricValues() throws IOException {
+    TimelineEntities teApp = new TimelineEntities();
+    ApplicationEntity entityApp = new ApplicationEntity();
+    String appId = "application_1000178881110_2002";
+    entityApp.setId(appId);
+    entityApp.setCreatedTime(1425016501000L);
+    entityApp.setModifiedTime(1425026901000L);
+    // add metrics with floating point values
+    Set<TimelineMetric> metricsApp = new HashSet<>();
+    TimelineMetric mApp = new TimelineMetric();
+    mApp.setId("MAP_SLOT_MILLIS");
+    Map<Long, Number> metricAppValues = new HashMap<Long, Number>();
+    long ts = System.currentTimeMillis();
+    metricAppValues.put(ts - 20, 10.5);
+    metricAppValues.put(ts - 10, 20.5);
+    mApp.setType(Type.TIME_SERIES);
+    mApp.setValues(metricAppValues);
+    metricsApp.add(mApp);
+    entityApp.addMetrics(metricsApp);
+    teApp.addEntity(entityApp);
+
+    TimelineEntities teEntity = new TimelineEntities();
+    TimelineEntity entity = new TimelineEntity();
+    entity.setId("hello");
+    entity.setType("world");
+    entity.setCreatedTime(1425016501000L);
+    entity.setModifiedTime(1425026901000L);
+    // add metrics with floating point values
+    Set<TimelineMetric> metricsEntity = new HashSet<>();
+    TimelineMetric mEntity = new TimelineMetric();
+    mEntity.setId("MAP_SLOT_MILLIS");
+    mEntity.addValue(ts - 20, 10.5);
+    metricsEntity.add(mEntity);
+    entity.addMetrics(metricsEntity);
+    teEntity.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      // Writing application entity.
+      try {
+        hbi.write("c1", "u1", "f1", "v1", 1002345678919L, appId, teApp);
+        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);
+        Assert.fail("Expected an exception as metric values are non integral");
+      } catch (IOException e) {}
+      hbi.stop();
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     util.shutdownMiniCluster();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6f57ecc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
index b0f83b7..4fb8f0e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
@@ -37,6 +37,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.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
 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;
@@ -164,10 +165,10 @@ public class TestHBaseStorageFlowRun {
         .getBytes());
 
     assertEquals(2, r1.size());
-    long starttime = (Long) GenericObjectMapper.read(values
-        .get(FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes()));
+    long starttime = Bytes.toLong(values.get(
+        FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes()));
     assertEquals(minStartTs, starttime);
-    assertEquals(endTs, GenericObjectMapper.read(values
+    assertEquals(endTs, Bytes.toLong(values
         .get(FlowRunColumn.MAX_END_TIME.getColumnQualifierBytes())));
 
     // use the timeline reader to verify data
@@ -253,10 +254,10 @@ public class TestHBaseStorageFlowRun {
         }
         switch (id) {
         case metric1:
-          assertEquals(141, value);
+          assertEquals(141L, value);
           break;
         case metric2:
-          assertEquals(57, value);
+          assertEquals(57L, value);
           break;
         default:
           fail("unrecognized metric: " + id);
@@ -292,14 +293,14 @@ public class TestHBaseStorageFlowRun {
       byte[] q = ColumnHelper.getColumnQualifier(
           FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric1);
       assertTrue(values.containsKey(q));
-      assertEquals(141, GenericObjectMapper.read(values.get(q)));
+      assertEquals(141L, Bytes.toLong(values.get(q)));
 
       // check metric2
       assertEquals(2, values.size());
       q = ColumnHelper.getColumnQualifier(
           FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric2);
       assertTrue(values.containsKey(q));
-      assertEquals(57, GenericObjectMapper.read(values.get(q)));
+      assertEquals(57L, Bytes.toLong(values.get(q)));
     }
     assertEquals(1, rowCount);
   }


[48/50] [abbrv] hadoop git commit: YARN-4200. Refactor reader classes in storage to nest under hbase specific package name. Contributed by Li Lu.

Posted by gt...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/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
new file mode 100644
index 0000000..181ec81
--- /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/reader/ApplicationEntityReader.java
@@ -0,0 +1,383 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Map;
+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.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.FamilyFilter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.FilterList.Operator;
+import org.apache.hadoop.hbase.filter.PageFilter;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+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;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnFamily;
+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.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Timeline entity reader for application entities that are stored in the
+ * application table.
+ */
+class ApplicationEntityReader extends GenericEntityReader {
+  private static final ApplicationTable APPLICATION_TABLE =
+      new ApplicationTable();
+
+  public ApplicationEntityReader(String userId, String clusterId,
+      String flowName, Long flowRunId, String appId, String entityType,
+      Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
+      EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
+        createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
+        relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
+        eventFilters, confsToRetrieve, metricsToRetrieve, fieldsToRetrieve,
+        true);
+  }
+
+  public ApplicationEntityReader(String userId, String clusterId,
+      String flowName, Long flowRunId, String appId, String entityType,
+      String entityId, TimelineFilterList confsToRetrieve,
+      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
+        confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
+  }
+
+  /**
+   * Uses the {@link ApplicationTable}.
+   */
+  protected BaseTable<?> getTable() {
+    return APPLICATION_TABLE;
+  }
+
+  @Override
+  protected FilterList constructFilterListBasedOnFields() {
+    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
+    // Fetch all the columns.
+    if (fieldsToRetrieve.contains(Field.ALL) &&
+        (confsToRetrieve == null ||
+        confsToRetrieve.getFilterList().isEmpty()) &&
+        (metricsToRetrieve == null ||
+        metricsToRetrieve.getFilterList().isEmpty())) {
+      return list;
+    }
+    FilterList infoColFamilyList = new FilterList();
+    // By default fetch everything in INFO column family.
+    FamilyFilter infoColumnFamily =
+        new FamilyFilter(CompareOp.EQUAL,
+           new BinaryComparator(ApplicationColumnFamily.INFO.getBytes()));
+    infoColFamilyList.addFilter(infoColumnFamily);
+    // Events not required.
+    if (!fieldsToRetrieve.contains(Field.EVENTS) &&
+        !fieldsToRetrieve.contains(Field.ALL) && eventFilters == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+          ApplicationColumnPrefix.EVENT.getColumnPrefixBytes(""))));
+    }
+    // info not required.
+    if (!fieldsToRetrieve.contains(Field.INFO) &&
+        !fieldsToRetrieve.contains(Field.ALL) && infoFilters == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+          ApplicationColumnPrefix.INFO.getColumnPrefixBytes(""))));
+    }
+    // is releated to not required.
+    if (!fieldsToRetrieve.contains(Field.IS_RELATED_TO) &&
+        !fieldsToRetrieve.contains(Field.ALL) && isRelatedTo == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+          ApplicationColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes(""))));
+    }
+    // relates to not required.
+    if (!fieldsToRetrieve.contains(Field.RELATES_TO) &&
+        !fieldsToRetrieve.contains(Field.ALL) && relatesTo == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+          ApplicationColumnPrefix.RELATES_TO.getColumnPrefixBytes(""))));
+    }
+    list.addFilter(infoColFamilyList);
+    if ((fieldsToRetrieve.contains(Field.CONFIGS) || configFilters != null) ||
+        (confsToRetrieve != null &&
+        !confsToRetrieve.getFilterList().isEmpty())) {
+      FilterList filterCfg =
+          new FilterList(new FamilyFilter(CompareOp.EQUAL,
+          new BinaryComparator(ApplicationColumnFamily.CONFIGS.getBytes())));
+      if (confsToRetrieve != null &&
+          !confsToRetrieve.getFilterList().isEmpty()) {
+        filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList(
+            ApplicationColumnPrefix.CONFIG, confsToRetrieve));
+      }
+      list.addFilter(filterCfg);
+    }
+    if ((fieldsToRetrieve.contains(Field.METRICS) || metricFilters != null) ||
+        (metricsToRetrieve != null &&
+        !metricsToRetrieve.getFilterList().isEmpty())) {
+      FilterList filterMetrics =
+          new FilterList(new FamilyFilter(CompareOp.EQUAL,
+          new BinaryComparator(ApplicationColumnFamily.METRICS.getBytes())));
+      if (metricsToRetrieve != null &&
+          !metricsToRetrieve.getFilterList().isEmpty()) {
+        filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList(
+            ApplicationColumnPrefix.METRIC, metricsToRetrieve));
+      }
+      list.addFilter(filterMetrics);
+    }
+    return list;
+  }
+
+  @Override
+  protected Result getResult(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException {
+    byte[] rowKey =
+        ApplicationRowKey.getRowKey(clusterId, userId, flowName, flowRunId,
+            appId);
+    Get get = new Get(rowKey);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      get.setFilter(filterList);
+    }
+    return table.getResult(hbaseConf, conn, get);
+  }
+
+  @Override
+  protected void validateParams() {
+    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
+    Preconditions.checkNotNull(entityType, "entityType shouldn't be null");
+    if (singleEntityRead) {
+      Preconditions.checkNotNull(appId, "appId shouldn't be null");
+    } else {
+      Preconditions.checkNotNull(userId, "userId shouldn't be null");
+      Preconditions.checkNotNull(flowName, "flowName shouldn't be null");
+    }
+  }
+
+  @Override
+  protected void augmentParams(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    if (singleEntityRead) {
+      if (flowName == null || flowRunId == null || userId == null) {
+        FlowContext context =
+            lookupFlowContext(clusterId, appId, hbaseConf, conn);
+        flowName = context.flowName;
+        flowRunId = context.flowRunId;
+        userId = context.userId;
+      }
+    }
+    if (fieldsToRetrieve == null) {
+      fieldsToRetrieve = EnumSet.noneOf(Field.class);
+    }
+    if (!fieldsToRetrieve.contains(Field.CONFIGS) &&
+        confsToRetrieve != null && !confsToRetrieve.getFilterList().isEmpty()) {
+      fieldsToRetrieve.add(Field.CONFIGS);
+    }
+    if (!fieldsToRetrieve.contains(Field.METRICS) &&
+        metricsToRetrieve != null &&
+        !metricsToRetrieve.getFilterList().isEmpty()) {
+      fieldsToRetrieve.add(Field.METRICS);
+    }
+    if (!singleEntityRead) {
+      if (limit == null || limit < 0) {
+        limit = TimelineReader.DEFAULT_LIMIT;
+      }
+      if (createdTimeBegin == null) {
+        createdTimeBegin = DEFAULT_BEGIN_TIME;
+      }
+      if (createdTimeEnd == null) {
+        createdTimeEnd = DEFAULT_END_TIME;
+      }
+      if (modifiedTimeBegin == null) {
+        modifiedTimeBegin = DEFAULT_BEGIN_TIME;
+      }
+      if (modifiedTimeEnd == null) {
+        modifiedTimeEnd = DEFAULT_END_TIME;
+      }
+    }
+  }
+
+  @Override
+  protected ResultScanner getResults(Configuration hbaseConf,
+      Connection conn, FilterList filterList) throws IOException {
+    Scan scan = new Scan();
+    if (flowRunId != null) {
+      scan.setRowPrefixFilter(ApplicationRowKey.
+          getRowKeyPrefix(clusterId, userId, flowName, flowRunId));
+    } else {
+      scan.setRowPrefixFilter(ApplicationRowKey.
+          getRowKeyPrefix(clusterId, userId, flowName));
+    }
+    FilterList newList = new FilterList();
+    newList.addFilter(new PageFilter(limit));
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      newList.addFilter(filterList);
+    }
+    scan.setFilter(newList);
+    return table.getResultScanner(hbaseConf, conn, scan);
+  }
+
+  @Override
+  protected TimelineEntity parseEntity(Result result) throws IOException {
+    if (result == null || result.isEmpty()) {
+      return null;
+    }
+    TimelineEntity entity = new TimelineEntity();
+    entity.setType(TimelineEntityType.YARN_APPLICATION.toString());
+    String entityId = ApplicationColumn.ID.readResult(result).toString();
+    entity.setId(entityId);
+
+    // fetch created time
+    Number createdTime =
+        (Number)ApplicationColumn.CREATED_TIME.readResult(result);
+    entity.setCreatedTime(createdTime.longValue());
+    if (!singleEntityRead && (entity.getCreatedTime() < createdTimeBegin ||
+        entity.getCreatedTime() > createdTimeEnd)) {
+      return null;
+    }
+
+    // fetch modified time
+    Number modifiedTime =
+        (Number)ApplicationColumn.MODIFIED_TIME.readResult(result);
+    entity.setModifiedTime(modifiedTime.longValue());
+    if (!singleEntityRead && (entity.getModifiedTime() < modifiedTimeBegin ||
+        entity.getModifiedTime() > modifiedTimeEnd)) {
+      return null;
+    }
+
+    // fetch is related to entities
+    boolean checkIsRelatedTo = isRelatedTo != null && isRelatedTo.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
+      readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO,
+          true);
+      if (checkIsRelatedTo && !TimelineStorageUtils.matchRelations(
+          entity.getIsRelatedToEntities(), isRelatedTo)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.IS_RELATED_TO)) {
+        entity.getIsRelatedToEntities().clear();
+      }
+    }
+
+    // fetch relates to entities
+    boolean checkRelatesTo = relatesTo != null && relatesTo.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
+      readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO,
+          false);
+      if (checkRelatesTo && !TimelineStorageUtils.matchRelations(
+          entity.getRelatesToEntities(), relatesTo)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.RELATES_TO)) {
+        entity.getRelatesToEntities().clear();
+      }
+    }
+
+    // fetch info
+    boolean checkInfo = infoFilters != null && infoFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
+      readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false);
+      if (checkInfo &&
+          !TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.INFO)) {
+        entity.getInfo().clear();
+      }
+    }
+
+    // fetch configs
+    boolean checkConfigs = configFilters != null && configFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
+      readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true);
+      if (checkConfigs && !TimelineStorageUtils.matchFilters(
+          entity.getConfigs(), configFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.CONFIGS)) {
+        entity.getConfigs().clear();
+      }
+    }
+
+    // fetch events
+    boolean checkEvents = eventFilters != null && eventFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
+      readEvents(entity, result, true);
+      if (checkEvents && !TimelineStorageUtils.matchEventFilters(
+          entity.getEvents(), eventFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.EVENTS)) {
+        entity.getEvents().clear();
+      }
+    }
+
+    // fetch metrics
+    boolean checkMetrics = metricFilters != null && metricFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
+      readMetrics(entity, result, ApplicationColumnPrefix.METRIC);
+      if (checkMetrics && !TimelineStorageUtils.matchMetricFilters(
+          entity.getMetrics(), metricFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.METRICS)) {
+        entity.getMetrics().clear();
+      }
+    }
+    return entity;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowActivityEntityReader.java
new file mode 100644
index 0000000..52ceef8
--- /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/reader/FlowActivityEntityReader.java
@@ -0,0 +1,164 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PageFilter;
+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.TimelineEntity;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
+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.flow.FlowActivityColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Timeline entity reader for flow activity entities that are stored in the
+ * flow activity table.
+ */
+class FlowActivityEntityReader extends TimelineEntityReader {
+  private static final FlowActivityTable FLOW_ACTIVITY_TABLE =
+      new FlowActivityTable();
+
+  public FlowActivityEntityReader(String userId, String clusterId,
+      String flowName, Long flowRunId, String appId, String entityType,
+      Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
+        createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
+        relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
+        eventFilters, null, null, fieldsToRetrieve, true);
+  }
+
+  public FlowActivityEntityReader(String userId, String clusterId,
+      String flowName, Long flowRunId, String appId, String entityType,
+      String entityId, EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
+        null, null, fieldsToRetrieve);
+  }
+
+  /**
+   * Uses the {@link FlowActivityTable}.
+   */
+  @Override
+  protected BaseTable<?> getTable() {
+    return FLOW_ACTIVITY_TABLE;
+  }
+
+  @Override
+  protected void validateParams() {
+    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
+  }
+
+  @Override
+  protected void augmentParams(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    if (limit == null || limit < 0) {
+      limit = TimelineReader.DEFAULT_LIMIT;
+    }
+    if (createdTimeBegin == null) {
+      createdTimeBegin = DEFAULT_BEGIN_TIME;
+    }
+    if (createdTimeEnd == null) {
+      createdTimeEnd = DEFAULT_END_TIME;
+    }
+  }
+
+  @Override
+  protected FilterList constructFilterListBasedOnFields() {
+    return null;
+  }
+
+  @Override
+  protected Result getResult(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException {
+    throw new UnsupportedOperationException(
+        "we don't support a single entity query");
+  }
+
+  @Override
+  protected ResultScanner getResults(Configuration hbaseConf,
+      Connection conn, FilterList filterList) throws IOException {
+    Scan scan = new Scan();
+    if (createdTimeBegin == DEFAULT_BEGIN_TIME &&
+        createdTimeEnd == DEFAULT_END_TIME) {
+      scan.setRowPrefixFilter(FlowActivityRowKey.getRowKeyPrefix(clusterId));
+    } else {
+      scan.setStartRow(
+          FlowActivityRowKey.getRowKeyPrefix(clusterId, createdTimeEnd));
+      scan.setStopRow(
+          FlowActivityRowKey.getRowKeyPrefix(clusterId,
+              (createdTimeBegin <= 0 ? 0: (createdTimeBegin - 1))));
+    }
+    // use the page filter to limit the result to the page size
+    // the scanner may still return more than the limit; therefore we need to
+    // read the right number as we iterate
+    scan.setFilter(new PageFilter(limit));
+    return table.getResultScanner(hbaseConf, conn, scan);
+  }
+
+  @Override
+  protected TimelineEntity parseEntity(Result result) throws IOException {
+    FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(result.getRow());
+
+    long time = rowKey.getDayTimestamp();
+    String user = rowKey.getUserId();
+    String flowName = rowKey.getFlowName();
+
+    FlowActivityEntity flowActivity =
+        new FlowActivityEntity(clusterId, time, user, flowName);
+    // set the id
+    flowActivity.setId(flowActivity.getId());
+    // get the list of run ids along with the version that are associated with
+    // this flow on this day
+    Map<String, Object> runIdsMap =
+        FlowActivityColumnPrefix.RUN_ID.readResults(result);
+    for (Map.Entry<String, Object> e : runIdsMap.entrySet()) {
+      Long runId = Long.valueOf(e.getKey());
+      String version = (String)e.getValue();
+      FlowRunEntity flowRun = new FlowRunEntity();
+      flowRun.setUser(user);
+      flowRun.setName(flowName);
+      flowRun.setRunId(runId);
+      flowRun.setVersion(version);
+      // set the id
+      flowRun.setId(flowRun.getId());
+      flowActivity.addFlowRun(flowRun);
+    }
+
+    return flowActivity;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/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
new file mode 100644
index 0000000..6286ee1
--- /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/reader/FlowRunEntityReader.java
@@ -0,0 +1,226 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Map;
+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.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
+import org.apache.hadoop.hbase.filter.FamilyFilter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PageFilter;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.FilterList.Operator;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+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;
+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.flow.FlowRunColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnFamily;
+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 com.google.common.base.Preconditions;
+
+/**
+ * Timeline entity reader for flow run entities that are stored in the flow run
+ * table.
+ */
+class FlowRunEntityReader extends TimelineEntityReader {
+  private static final FlowRunTable FLOW_RUN_TABLE = new FlowRunTable();
+
+  public FlowRunEntityReader(String userId, String clusterId,
+      String flowName, Long flowRunId, String appId, String entityType,
+      Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
+      EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
+        createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
+        relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
+        eventFilters, null, metricsToRetrieve, fieldsToRetrieve, true);
+  }
+
+  public FlowRunEntityReader(String userId, String clusterId,
+      String flowName, Long flowRunId, String appId, String entityType,
+      String entityId, TimelineFilterList confsToRetrieve,
+      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
+        null, metricsToRetrieve, fieldsToRetrieve);
+  }
+
+  /**
+   * Uses the {@link FlowRunTable}.
+   */
+  @Override
+  protected BaseTable<?> getTable() {
+    return FLOW_RUN_TABLE;
+  }
+
+  @Override
+  protected void validateParams() {
+    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
+    Preconditions.checkNotNull(userId, "userId shouldn't be null");
+    Preconditions.checkNotNull(flowName, "flowName shouldn't be null");
+    if (singleEntityRead) {
+      Preconditions.checkNotNull(flowRunId, "flowRunId shouldn't be null");
+    }
+  }
+
+  @Override
+  protected void augmentParams(Configuration hbaseConf, Connection conn) {
+    if (!singleEntityRead) {
+      if (fieldsToRetrieve == null) {
+        fieldsToRetrieve = EnumSet.noneOf(Field.class);
+      }
+      if (limit == null || limit < 0) {
+        limit = TimelineReader.DEFAULT_LIMIT;
+      }
+      if (createdTimeBegin == null) {
+        createdTimeBegin = DEFAULT_BEGIN_TIME;
+      }
+      if (createdTimeEnd == null) {
+        createdTimeEnd = DEFAULT_END_TIME;
+      }
+      if (!fieldsToRetrieve.contains(Field.METRICS) &&
+          metricsToRetrieve != null &&
+          !metricsToRetrieve.getFilterList().isEmpty()) {
+        fieldsToRetrieve.add(Field.METRICS);
+      }
+    }
+  }
+
+  @Override
+  protected FilterList constructFilterListBasedOnFields() {
+    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
+
+    // By default fetch everything in INFO column family.
+    FamilyFilter infoColumnFamily =
+        new FamilyFilter(CompareOp.EQUAL,
+           new BinaryComparator(FlowRunColumnFamily.INFO.getBytes()));
+    // Metrics not required.
+    if (!singleEntityRead && !fieldsToRetrieve.contains(Field.METRICS) &&
+        !fieldsToRetrieve.contains(Field.ALL)) {
+      FilterList infoColFamilyList = new FilterList(Operator.MUST_PASS_ONE);
+      infoColFamilyList.addFilter(infoColumnFamily);
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(""))));
+      list.addFilter(infoColFamilyList);
+    }
+    if (metricsToRetrieve != null &&
+        !metricsToRetrieve.getFilterList().isEmpty()) {
+      FilterList infoColFamilyList = new FilterList();
+      infoColFamilyList.addFilter(infoColumnFamily);
+      infoColFamilyList.addFilter(TimelineFilterUtils.createHBaseFilterList(
+          FlowRunColumnPrefix.METRIC, metricsToRetrieve));
+      list.addFilter(infoColFamilyList);
+    }
+    return list;
+  }
+
+  @Override
+  protected Result getResult(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException {
+    byte[] rowKey =
+        FlowRunRowKey.getRowKey(clusterId, userId, flowName, flowRunId);
+    Get get = new Get(rowKey);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      get.setFilter(filterList);
+    }
+    return table.getResult(hbaseConf, conn, get);
+  }
+
+  @Override
+  protected ResultScanner getResults(Configuration hbaseConf,
+      Connection conn, FilterList filterList) throws IOException {
+    Scan scan = new Scan();
+    scan.setRowPrefixFilter(
+        FlowRunRowKey.getRowKeyPrefix(clusterId, userId, flowName));
+    FilterList newList = new FilterList();
+    newList.addFilter(new PageFilter(limit));
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      newList.addFilter(filterList);
+    }
+    scan.setFilter(newList);
+    return table.getResultScanner(hbaseConf, conn, scan);
+  }
+
+  @Override
+  protected TimelineEntity parseEntity(Result result) throws IOException {
+    FlowRunEntity flowRun = new FlowRunEntity();
+    flowRun.setUser(userId);
+    flowRun.setName(flowName);
+    if (singleEntityRead) {
+      flowRun.setRunId(flowRunId);
+    } else {
+      FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(result.getRow());
+      flowRun.setRunId(rowKey.getFlowRunId());
+    }
+
+    // read the start time
+    Long startTime = (Long)FlowRunColumn.MIN_START_TIME.readResult(result);
+    if (startTime != null) {
+      flowRun.setStartTime(startTime.longValue());
+    }
+    if (!singleEntityRead && (flowRun.getStartTime() < createdTimeBegin ||
+        flowRun.getStartTime() > createdTimeEnd)) {
+      return null;
+    }
+
+    // read the end time if available
+    Long endTime = (Long)FlowRunColumn.MAX_END_TIME.readResult(result);
+    if (endTime != null) {
+      flowRun.setMaxEndTime(endTime.longValue());
+    }
+
+    // read the flow version
+    String version = (String)FlowRunColumn.FLOW_VERSION.readResult(result);
+    if (version != null) {
+      flowRun.setVersion(version);
+    }
+
+    // read metrics
+    if (singleEntityRead || fieldsToRetrieve.contains(Field.METRICS)) {
+      readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC);
+    }
+
+    // set the id
+    flowRun.setId(flowRun.getId());
+    return flowRun;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/GenericEntityReader.java
new file mode 100644
index 0000000..f3f380c
--- /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/reader/GenericEntityReader.java
@@ -0,0 +1,497 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+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.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
+import org.apache.hadoop.hbase.filter.FamilyFilter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.FilterList.Operator;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+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;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
+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.AppToFlowRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Timeline entity reader for generic entities that are stored in the entity
+ * table.
+ */
+class GenericEntityReader extends TimelineEntityReader {
+  private static final EntityTable ENTITY_TABLE = new EntityTable();
+  private static final Log LOG = LogFactory.getLog(GenericEntityReader.class);
+
+  /**
+   * Used to look up the flow context.
+   */
+  private final AppToFlowTable appToFlowTable = new AppToFlowTable();
+
+  public GenericEntityReader(String userId, String clusterId,
+      String flowName, Long flowRunId, String appId, String entityType,
+      Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
+      EnumSet<Field> fieldsToRetrieve, boolean sortedKeys) {
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
+        createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
+        relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
+        eventFilters, confsToRetrieve, metricsToRetrieve, fieldsToRetrieve,
+        sortedKeys);
+  }
+
+  public GenericEntityReader(String userId, String clusterId,
+      String flowName, Long flowRunId, String appId, String entityType,
+      String entityId, TimelineFilterList confsToRetrieve,
+      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
+        confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
+  }
+
+  /**
+   * Uses the {@link EntityTable}.
+   */
+  protected BaseTable<?> getTable() {
+    return ENTITY_TABLE;
+  }
+
+  @Override
+  protected FilterList constructFilterListBasedOnFields() {
+    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
+    // Fetch all the columns.
+    if (fieldsToRetrieve.contains(Field.ALL) &&
+        (confsToRetrieve == null ||
+        confsToRetrieve.getFilterList().isEmpty()) &&
+        (metricsToRetrieve == null ||
+        metricsToRetrieve.getFilterList().isEmpty())) {
+      return list;
+    }
+    FilterList infoColFamilyList = new FilterList();
+    // By default fetch everything in INFO column family.
+    FamilyFilter infoColumnFamily =
+        new FamilyFilter(CompareOp.EQUAL,
+           new BinaryComparator(EntityColumnFamily.INFO.getBytes()));
+    infoColFamilyList.addFilter(infoColumnFamily);
+    // Events not required.
+    if (!fieldsToRetrieve.contains(Field.EVENTS) &&
+        !fieldsToRetrieve.contains(Field.ALL) && eventFilters == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+          EntityColumnPrefix.EVENT.getColumnPrefixBytes(""))));
+    }
+    // info not required.
+    if (!fieldsToRetrieve.contains(Field.INFO) &&
+        !fieldsToRetrieve.contains(Field.ALL) && infoFilters == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+              EntityColumnPrefix.INFO.getColumnPrefixBytes(""))));
+    }
+    // is related to not required.
+    if (!fieldsToRetrieve.contains(Field.IS_RELATED_TO) &&
+        !fieldsToRetrieve.contains(Field.ALL) && isRelatedTo == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+              EntityColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes(""))));
+    }
+    // relates to not required.
+    if (!fieldsToRetrieve.contains(Field.RELATES_TO) &&
+        !fieldsToRetrieve.contains(Field.ALL) && relatesTo == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+              EntityColumnPrefix.RELATES_TO.getColumnPrefixBytes(""))));
+    }
+    list.addFilter(infoColFamilyList);
+    if ((fieldsToRetrieve.contains(Field.CONFIGS) || configFilters != null) ||
+        (confsToRetrieve != null &&
+        !confsToRetrieve.getFilterList().isEmpty())) {
+      FilterList filterCfg =
+          new FilterList(new FamilyFilter(CompareOp.EQUAL,
+              new BinaryComparator(EntityColumnFamily.CONFIGS.getBytes())));
+      if (confsToRetrieve != null &&
+          !confsToRetrieve.getFilterList().isEmpty()) {
+        filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList(
+            EntityColumnPrefix.CONFIG, confsToRetrieve));
+      }
+      list.addFilter(filterCfg);
+    }
+    if ((fieldsToRetrieve.contains(Field.METRICS) || metricFilters != null) ||
+        (metricsToRetrieve != null &&
+        !metricsToRetrieve.getFilterList().isEmpty())) {
+      FilterList filterMetrics =
+          new FilterList(new FamilyFilter(CompareOp.EQUAL,
+              new BinaryComparator(EntityColumnFamily.METRICS.getBytes())));
+      if (metricsToRetrieve != null &&
+          !metricsToRetrieve.getFilterList().isEmpty()) {
+        filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList(
+            EntityColumnPrefix.METRIC, metricsToRetrieve));
+      }
+      list.addFilter(filterMetrics);
+    }
+    return list;
+  }
+
+  protected FlowContext lookupFlowContext(String clusterId, String appId,
+      Configuration hbaseConf, Connection conn) throws IOException {
+    byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
+    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());
+    } else {
+       throw new IOException(
+           "Unable to find the context flow ID and flow run ID for clusterId=" +
+           clusterId + ", appId=" + appId);
+    }
+  }
+
+  protected static class FlowContext {
+    protected final String userId;
+    protected final String flowName;
+    protected final Long flowRunId;
+    public FlowContext(String user, String flowName, Long flowRunId) {
+      this.userId = user;
+      this.flowName = flowName;
+      this.flowRunId = flowRunId;
+    }
+  }
+
+  @Override
+  protected void validateParams() {
+    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
+    Preconditions.checkNotNull(appId, "appId shouldn't be null");
+    Preconditions.checkNotNull(entityType, "entityType shouldn't be null");
+    if (singleEntityRead) {
+      Preconditions.checkNotNull(entityId, "entityId shouldn't be null");
+    }
+  }
+
+  @Override
+  protected void augmentParams(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    // In reality all three should be null or neither should be null
+    if (flowName == null || flowRunId == null || userId == null) {
+      FlowContext context =
+          lookupFlowContext(clusterId, appId, hbaseConf, conn);
+      flowName = context.flowName;
+      flowRunId = context.flowRunId;
+      userId = context.userId;
+    }
+    if (fieldsToRetrieve == null) {
+      fieldsToRetrieve = EnumSet.noneOf(Field.class);
+    }
+    if (!fieldsToRetrieve.contains(Field.CONFIGS) &&
+        confsToRetrieve != null && !confsToRetrieve.getFilterList().isEmpty()) {
+      fieldsToRetrieve.add(Field.CONFIGS);
+    }
+    if (!fieldsToRetrieve.contains(Field.METRICS) &&
+        metricsToRetrieve != null &&
+        !metricsToRetrieve.getFilterList().isEmpty()) {
+      fieldsToRetrieve.add(Field.METRICS);
+    }
+    if (!singleEntityRead) {
+      if (limit == null || limit < 0) {
+        limit = TimelineReader.DEFAULT_LIMIT;
+      }
+      if (createdTimeBegin == null) {
+        createdTimeBegin = DEFAULT_BEGIN_TIME;
+      }
+      if (createdTimeEnd == null) {
+        createdTimeEnd = DEFAULT_END_TIME;
+      }
+      if (modifiedTimeBegin == null) {
+        modifiedTimeBegin = DEFAULT_BEGIN_TIME;
+      }
+      if (modifiedTimeEnd == null) {
+        modifiedTimeEnd = DEFAULT_END_TIME;
+      }
+    }
+  }
+
+  @Override
+  protected Result getResult(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException {
+    byte[] rowKey =
+        EntityRowKey.getRowKey(clusterId, userId, flowName, flowRunId, appId,
+            entityType, entityId);
+    Get get = new Get(rowKey);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      get.setFilter(filterList);
+    }
+    return table.getResult(hbaseConf, conn, get);
+  }
+
+  @Override
+  protected ResultScanner getResults(Configuration hbaseConf,
+      Connection conn, FilterList filterList) throws IOException {
+    // Scan through part of the table to find the entities belong to one app
+    // and one type
+    Scan scan = new Scan();
+    scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
+        clusterId, userId, flowName, flowRunId, appId, entityType));
+    scan.setMaxVersions(Integer.MAX_VALUE);
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      scan.setFilter(filterList);
+    }
+    return table.getResultScanner(hbaseConf, conn, scan);
+  }
+
+  @Override
+  protected TimelineEntity parseEntity(Result result) throws IOException {
+    if (result == null || result.isEmpty()) {
+      return null;
+    }
+    TimelineEntity entity = new TimelineEntity();
+    String entityType = EntityColumn.TYPE.readResult(result).toString();
+    entity.setType(entityType);
+    String entityId = EntityColumn.ID.readResult(result).toString();
+    entity.setId(entityId);
+
+    // fetch created time
+    Number createdTime = (Number)EntityColumn.CREATED_TIME.readResult(result);
+    entity.setCreatedTime(createdTime.longValue());
+    if (!singleEntityRead && (entity.getCreatedTime() < createdTimeBegin ||
+        entity.getCreatedTime() > createdTimeEnd)) {
+      return null;
+    }
+
+    // fetch modified time
+    Number modifiedTime = (Number)EntityColumn.MODIFIED_TIME.readResult(result);
+    entity.setModifiedTime(modifiedTime.longValue());
+    if (!singleEntityRead && (entity.getModifiedTime() < modifiedTimeBegin ||
+        entity.getModifiedTime() > modifiedTimeEnd)) {
+      return null;
+    }
+
+    // fetch is related to entities
+    boolean checkIsRelatedTo = isRelatedTo != null && isRelatedTo.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
+      readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO, true);
+      if (checkIsRelatedTo && !TimelineStorageUtils.matchRelations(
+          entity.getIsRelatedToEntities(), isRelatedTo)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.IS_RELATED_TO)) {
+        entity.getIsRelatedToEntities().clear();
+      }
+    }
+
+    // fetch relates to entities
+    boolean checkRelatesTo = relatesTo != null && relatesTo.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
+      readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false);
+      if (checkRelatesTo && !TimelineStorageUtils.matchRelations(
+          entity.getRelatesToEntities(), relatesTo)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.RELATES_TO)) {
+        entity.getRelatesToEntities().clear();
+      }
+    }
+
+    // fetch info
+    boolean checkInfo = infoFilters != null && infoFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
+      readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false);
+      if (checkInfo &&
+          !TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.INFO)) {
+        entity.getInfo().clear();
+      }
+    }
+
+    // fetch configs
+    boolean checkConfigs = configFilters != null && configFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
+      readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true);
+      if (checkConfigs && !TimelineStorageUtils.matchFilters(
+          entity.getConfigs(), configFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.CONFIGS)) {
+        entity.getConfigs().clear();
+      }
+    }
+
+    // fetch events
+    boolean checkEvents = eventFilters != null && eventFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
+      readEvents(entity, result, false);
+      if (checkEvents && !TimelineStorageUtils.matchEventFilters(
+          entity.getEvents(), eventFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.EVENTS)) {
+        entity.getEvents().clear();
+      }
+    }
+
+    // fetch metrics
+    boolean checkMetrics = metricFilters != null && metricFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
+      readMetrics(entity, result, EntityColumnPrefix.METRIC);
+      if (checkMetrics && !TimelineStorageUtils.matchMetricFilters(
+          entity.getMetrics(), metricFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.METRICS)) {
+        entity.getMetrics().clear();
+      }
+    }
+    return entity;
+  }
+
+  /**
+   * Helper method for reading relationship.
+   */
+  protected <T> void readRelationship(
+      TimelineEntity entity, Result result, ColumnPrefix<T> prefix,
+      boolean isRelatedTo) throws IOException {
+    // isRelatedTo and relatesTo are of type Map<String, Set<String>>
+    Map<String, Object> columns = prefix.readResults(result);
+    for (Map.Entry<String, Object> column : columns.entrySet()) {
+      for (String id : Separator.VALUES.splitEncoded(
+          column.getValue().toString())) {
+        if (isRelatedTo) {
+          entity.addIsRelatedToEntity(column.getKey(), id);
+        } else {
+          entity.addRelatesToEntity(column.getKey(), id);
+        }
+      }
+    }
+  }
+
+  /**
+   * Helper method for reading key-value pairs for either info or config.
+   */
+  protected <T> void readKeyValuePairs(
+      TimelineEntity entity, Result result, ColumnPrefix<T> prefix,
+      boolean isConfig) throws IOException {
+    // info and configuration are of type Map<String, Object or String>
+    Map<String, Object> columns = prefix.readResults(result);
+    if (isConfig) {
+      for (Map.Entry<String, Object> column : columns.entrySet()) {
+        entity.addConfig(column.getKey(), column.getValue().toString());
+      }
+    } else {
+      entity.addInfo(columns);
+    }
+  }
+
+  /**
+   * Read events from the entity table or the application table. The column name
+   * is of the form "eventId=timestamp=infoKey" where "infoKey" may be omitted
+   * if there is no info associated with the event.
+   *
+   * See {@link EntityTable} and {@link ApplicationTable} for a more detailed
+   * schema description.
+   */
+  protected void readEvents(TimelineEntity entity, Result result,
+      boolean isApplication) throws IOException {
+    Map<String, TimelineEvent> eventsMap = new HashMap<>();
+    Map<?, Object> eventsResult = isApplication ?
+        ApplicationColumnPrefix.EVENT.
+            readResultsHavingCompoundColumnQualifiers(result) :
+        EntityColumnPrefix.EVENT.
+            readResultsHavingCompoundColumnQualifiers(result);
+    for (Map.Entry<?, Object> eventResult : eventsResult.entrySet()) {
+      byte[][] karr = (byte[][])eventResult.getKey();
+      // the column name is of the form "eventId=timestamp=infoKey"
+      if (karr.length == 3) {
+        String id = Bytes.toString(karr[0]);
+        long ts = TimelineStorageUtils.invertLong(Bytes.toLong(karr[1]));
+        String key = Separator.VALUES.joinEncoded(id, Long.toString(ts));
+        TimelineEvent event = eventsMap.get(key);
+        if (event == null) {
+          event = new TimelineEvent();
+          event.setId(id);
+          event.setTimestamp(ts);
+          eventsMap.put(key, event);
+        }
+        // handle empty info
+        String infoKey = karr[2].length == 0 ? null : Bytes.toString(karr[2]);
+        if (infoKey != null) {
+          event.addInfo(infoKey, eventResult.getValue());
+        }
+      } else {
+        LOG.warn("incorrectly formatted column name: it will be discarded");
+        continue;
+      }
+    }
+    Set<TimelineEvent> eventsSet = new HashSet<>(eventsMap.values());
+    entity.addEvents(eventsSet);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReader.java
new file mode 100644
index 0000000..e801466
--- /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/reader/TimelineEntityReader.java
@@ -0,0 +1,274 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.filter.FilterList;
+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.reader.filter.TimelineFilterList;
+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.ColumnPrefix;
+
+/**
+ * The base class for reading and deserializing timeline entities from the
+ * HBase storage. Different types can be defined for different types of the
+ * entities that are being requested.
+ */
+public abstract class TimelineEntityReader {
+  private static final Log LOG = LogFactory.getLog(TimelineEntityReader.class);
+  protected static final long DEFAULT_BEGIN_TIME = 0L;
+  protected static final long DEFAULT_END_TIME = Long.MAX_VALUE;
+
+  protected final boolean singleEntityRead;
+
+  protected String userId;
+  protected String clusterId;
+  protected String flowName;
+  protected Long flowRunId;
+  protected String appId;
+  protected String entityType;
+  protected EnumSet<Field> fieldsToRetrieve;
+  // used only for a single entity read mode
+  protected String entityId;
+  // used only for multiple entity read mode
+  protected Long limit;
+  protected Long createdTimeBegin;
+  protected Long createdTimeEnd;
+  protected Long modifiedTimeBegin;
+  protected Long modifiedTimeEnd;
+  protected Map<String, Set<String>> relatesTo;
+  protected Map<String, Set<String>> isRelatedTo;
+  protected Map<String, Object> infoFilters;
+  protected Map<String, String> configFilters;
+  protected Set<String> metricFilters;
+  protected Set<String> eventFilters;
+  protected TimelineFilterList confsToRetrieve;
+  protected TimelineFilterList metricsToRetrieve;
+
+  /**
+   * Main table the entity reader uses.
+   */
+  protected BaseTable<?> table;
+
+  /**
+   * Specifies whether keys for this table are sorted in a manner where entities
+   * can be retrieved by created time. If true, it will be sufficient to collect
+   * the first results as specified by the limit. Otherwise all matched entities
+   * will be fetched and then limit applied.
+   */
+  private boolean sortedKeys = false;
+
+  /**
+   * Instantiates a reader for multiple-entity reads.
+   */
+  protected TimelineEntityReader(String userId, String clusterId,
+      String flowName, Long flowRunId, String appId, String entityType,
+      Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
+      EnumSet<Field> fieldsToRetrieve, boolean sortedKeys) {
+    this.singleEntityRead = false;
+    this.sortedKeys = sortedKeys;
+    this.userId = userId;
+    this.clusterId = clusterId;
+    this.flowName = flowName;
+    this.flowRunId = flowRunId;
+    this.appId = appId;
+    this.entityType = entityType;
+    this.fieldsToRetrieve = fieldsToRetrieve;
+    this.limit = limit;
+    this.createdTimeBegin = createdTimeBegin;
+    this.createdTimeEnd = createdTimeEnd;
+    this.modifiedTimeBegin = modifiedTimeBegin;
+    this.modifiedTimeEnd = modifiedTimeEnd;
+    this.relatesTo = relatesTo;
+    this.isRelatedTo = isRelatedTo;
+    this.infoFilters = infoFilters;
+    this.configFilters = configFilters;
+    this.metricFilters = metricFilters;
+    this.eventFilters = eventFilters;
+    this.confsToRetrieve = confsToRetrieve;
+    this.metricsToRetrieve = metricsToRetrieve;
+
+    this.table = getTable();
+  }
+
+  /**
+   * Instantiates a reader for single-entity reads.
+   */
+  protected TimelineEntityReader(String userId, String clusterId,
+      String flowName, Long flowRunId, String appId, String entityType,
+      String entityId, TimelineFilterList confsToRetrieve,
+      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
+    this.singleEntityRead = true;
+    this.userId = userId;
+    this.clusterId = clusterId;
+    this.flowName = flowName;
+    this.flowRunId = flowRunId;
+    this.appId = appId;
+    this.entityType = entityType;
+    this.fieldsToRetrieve = fieldsToRetrieve;
+    this.entityId = entityId;
+    this.confsToRetrieve = confsToRetrieve;
+    this.metricsToRetrieve = metricsToRetrieve;
+
+    this.table = getTable();
+  }
+
+  /**
+   * Creates a {@link FilterList} based on fields, confs and metrics to
+   * retrieve. This filter list will be set in Scan/Get objects to trim down
+   * results fetched from HBase back-end storage.
+   * @return a {@link FilterList} object.
+   */
+  protected abstract FilterList constructFilterListBasedOnFields();
+
+  /**
+   * Reads and deserializes a single timeline entity from the HBase storage.
+   */
+  public TimelineEntity readEntity(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    validateParams();
+    augmentParams(hbaseConf, conn);
+
+    FilterList filterList = constructFilterListBasedOnFields();
+    Result result = getResult(hbaseConf, conn, filterList);
+    if (result == null || result.isEmpty()) {
+      // Could not find a matching row.
+      LOG.info("Cannot find matching entity of type " + entityType);
+      return null;
+    }
+    return parseEntity(result);
+  }
+
+  /**
+   * Reads and deserializes a set of timeline entities from the HBase storage.
+   * It goes through all the results available, and returns the number of
+   * entries as specified in the limit in the entity's natural sort order.
+   */
+  public Set<TimelineEntity> readEntities(Configuration hbaseConf,
+      Connection conn) throws IOException {
+    validateParams();
+    augmentParams(hbaseConf, conn);
+
+    NavigableSet<TimelineEntity> entities = new TreeSet<>();
+    FilterList filterList = constructFilterListBasedOnFields();
+    ResultScanner results = getResults(hbaseConf, conn, filterList);
+    try {
+      for (Result result : results) {
+        TimelineEntity entity = parseEntity(result);
+        if (entity == null) {
+          continue;
+        }
+        entities.add(entity);
+        if (!sortedKeys) {
+          if (entities.size() > limit) {
+            entities.pollLast();
+          }
+        } else {
+          if (entities.size() == limit) {
+            break;
+          }
+        }
+      }
+      return entities;
+    } finally {
+      results.close();
+    }
+  }
+
+  /**
+   * Returns the main table to be used by the entity reader.
+   */
+  protected abstract BaseTable<?> getTable();
+
+  /**
+   * Validates the required parameters to read the entities.
+   */
+  protected abstract void validateParams();
+
+  /**
+   * Sets certain parameters to defaults if the values are not provided.
+   */
+  protected abstract void augmentParams(Configuration hbaseConf,
+      Connection conn) throws IOException;
+
+  /**
+   * Fetches a {@link Result} instance for a single-entity read.
+   *
+   * @return the {@link Result} instance or null if no such record is found.
+   */
+  protected abstract Result getResult(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException;
+
+  /**
+   * Fetches a {@link ResultScanner} for a multi-entity read.
+   */
+  protected abstract ResultScanner getResults(Configuration hbaseConf,
+      Connection conn, FilterList filterList) throws IOException;
+
+  /**
+   * Given a {@link Result} instance, deserializes and creates a
+   * {@link TimelineEntity}.
+   *
+   * @return the {@link TimelineEntity} instance, or null if the {@link Result}
+   * is null or empty.
+   */
+  protected abstract TimelineEntity parseEntity(Result result)
+      throws IOException;
+
+  /**
+   * Helper method for reading and deserializing {@link TimelineMetric} objects
+   * using the specified column prefix. The timeline metrics then are added to
+   * the given timeline entity.
+   */
+  protected void readMetrics(TimelineEntity entity, Result result,
+      ColumnPrefix<?> columnPrefix) throws IOException {
+    NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
+        columnPrefix.readResultsWithTimestamps(result);
+    for (Map.Entry<String, NavigableMap<Long, Number>> metricResult:
+        metricsResult.entrySet()) {
+      TimelineMetric metric = new TimelineMetric();
+      metric.setId(metricResult.getKey());
+      // Simply assume that if the value set contains more than 1 elements, the
+      // metric is a TIME_SERIES metric, otherwise, it's a SINGLE_VALUE metric
+      metric.setType(metricResult.getValue().size() > 1 ?
+          TimelineMetric.Type.TIME_SERIES : TimelineMetric.Type.SINGLE_VALUE);
+      metric.addValues(metricResult.getValue());
+      entity.addMetric(metric);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/TimelineEntityReaderFactory.java
new file mode 100644
index 0000000..c77897a
--- /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/reader/TimelineEntityReaderFactory.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
+
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
+
+/**
+ * Factory methods for instantiating a timeline entity reader.
+ */
+public class TimelineEntityReaderFactory {
+  /**
+   * Creates a timeline entity reader instance for reading a single entity with
+   * the specified input.
+   */
+  public static TimelineEntityReader createSingleEntityReader(String userId,
+      String clusterId, String flowName, Long flowRunId, String appId,
+      String entityType, String entityId, TimelineFilterList confs,
+      TimelineFilterList metrics, EnumSet<Field> fieldsToRetrieve) {
+    // currently the types that are handled separate from the generic entity
+    // table are application, flow run, and flow activity entities
+    if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) {
+      return new ApplicationEntityReader(userId, clusterId, flowName, flowRunId,
+          appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
+    } else if (TimelineEntityType.YARN_FLOW_RUN.matches(entityType)) {
+      return new FlowRunEntityReader(userId, clusterId, flowName, flowRunId,
+          appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
+    } else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) {
+      return new FlowActivityEntityReader(userId, clusterId, flowName, flowRunId,
+          appId, entityType, entityId, fieldsToRetrieve);
+    } else {
+      // assume we're dealing with a generic entity read
+      return new GenericEntityReader(userId, clusterId, flowName, flowRunId,
+        appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
+    }
+  }
+
+  /**
+   * Creates a timeline entity reader instance for reading set of entities with
+   * the specified input and predicates.
+   */
+  public static TimelineEntityReader createMultipleEntitiesReader(String userId,
+      String clusterId, String flowName, Long flowRunId, String appId,
+      String entityType, Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confs, TimelineFilterList metrics,
+      EnumSet<Field> fieldsToRetrieve) {
+    // currently the types that are handled separate from the generic entity
+    // table are application, flow run, and flow activity entities
+    if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) {
+      return new ApplicationEntityReader(userId, clusterId, flowName, flowRunId,
+          appId, entityType, limit, createdTimeBegin, createdTimeEnd,
+          modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
+          infoFilters, configFilters, metricFilters, eventFilters, confs,
+          metrics, fieldsToRetrieve);
+    } else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) {
+      return new FlowActivityEntityReader(userId, clusterId, flowName, flowRunId,
+          appId, entityType, limit, createdTimeBegin, createdTimeEnd,
+          modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
+          infoFilters, configFilters, metricFilters, eventFilters,
+          fieldsToRetrieve);
+    } else if (TimelineEntityType.YARN_FLOW_RUN.matches(entityType)) {
+      return new FlowRunEntityReader(userId, clusterId, flowName, flowRunId,
+          appId, entityType, limit, createdTimeBegin, createdTimeEnd,
+          modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
+          infoFilters, configFilters, metricFilters, eventFilters, confs,
+          metrics, fieldsToRetrieve);
+    } else {
+      // assume we're dealing with a generic entity read
+      return new GenericEntityReader(userId, clusterId, flowName, flowRunId,
+          appId, entityType, limit, createdTimeBegin, createdTimeEnd,
+          modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
+          infoFilters, configFilters, metricFilters, eventFilters, confs,
+          metrics, fieldsToRetrieve, false);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/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/storage/reader/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/storage/reader/package-info.java
new file mode 100644
index 0000000..0b3fa38
--- /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/reader/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file


[17/50] [abbrv] hadoop git commit: YARN-3814. REST API implementation for getting raw entities in TimelineReader (Varun Saxena via sjlee)

Posted by gt...@apache.org.
YARN-3814. REST API implementation for getting raw entities in TimelineReader (Varun Saxena via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: 01f5e1895229ceaa3e8b49a39fc6b653a3555ad8
Parents: a057b28
Author: Sangjin Lee <sj...@apache.org>
Authored: Fri Aug 21 19:10:23 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:57 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../reader/TimelineReaderManager.java           |  41 ++
 .../reader/TimelineReaderServer.java            |   2 +-
 .../reader/TimelineReaderWebServices.java       | 245 +++++++++-
 .../storage/FileSystemTimelineReaderImpl.java   |   5 +
 .../reader/TestTimelineReaderWebServices.java   | 456 ++++++++++++++++++-
 6 files changed, 741 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/01f5e189/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 68ef1f7..f8775fc 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -97,6 +97,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4025. Deal with byte representations of Longs in writer code.
     (Sangjin Lee and Vrushali C via junping_du)
 
+    YARN-3814. REST API implementation for getting raw entities in
+    TimelineReader (Varun Saxena via sjlee)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01f5e189/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 5573185..7fafd82 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
@@ -18,10 +18,18 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.reader;
 
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 
 @Private
 @Unstable
@@ -33,4 +41,37 @@ public class TimelineReaderManager extends AbstractService {
     super(TimelineReaderManager.class.getName());
     this.reader = timelineReader;
   }
+
+  /**
+   * Get a set of entities matching given predicates. The meaning of each
+   * argument has been documented with {@link TimelineReader#getEntities}.
+   *
+   * @see TimelineReader#getEntities
+   */
+  Set<TimelineEntity> getEntities(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String>  metricFilters, Set<String> eventFilters,
+      EnumSet<Field> fieldsToRetrieve) throws IOException {
+    return reader.getEntities(userId, clusterId, flowId, flowRunId, appId,
+        entityType, limit, createdTimeBegin, createdTimeEnd, modifiedTimeBegin,
+        modifiedTimeEnd, relatesTo, isRelatedTo, infoFilters, configFilters,
+        metricFilters, eventFilters, fieldsToRetrieve);
+  }
+
+  /**
+   * Get single timeline entity. The meaning of each argument has been
+   * documented with {@link TimelineReader#getEntity}.
+   *
+   * @see TimelineReader#getEntity
+   */
+  public TimelineEntity getEntity(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      String entityId, EnumSet<Field> fields) throws IOException {
+    return reader.getEntity(userId, clusterId, flowId, flowRunId, appId,
+        entityType, entityId, fields);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01f5e189/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 874112c..319cfb0 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
@@ -54,7 +54,7 @@ import com.google.common.annotations.VisibleForTesting;
 public class TimelineReaderServer extends CompositeService {
   private static final Log LOG = LogFactory.getLog(TimelineReaderServer.class);
   private static final int SHUTDOWN_HOOK_PRIORITY = 30;
-  private static final String TIMELINE_READER_MANAGER_ATTR =
+  static final String TIMELINE_READER_MANAGER_ATTR =
       "timeline.reader.manager";
 
   private HttpServer2 readerWebServer;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01f5e189/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 3655a72..0b5fde0 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
@@ -18,42 +18,283 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.reader;
 
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
 
+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.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
+import org.apache.hadoop.yarn.webapp.BadRequestException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
 
 import com.google.inject.Singleton;
 
-
 /** REST end point for Timeline Reader */
 @Private
 @Unstable
 @Singleton
 @Path("/ws/v2/timeline")
 public class TimelineReaderWebServices {
+  private static final Log LOG =
+      LogFactory.getLog(TimelineReaderWebServices.class);
+
+  @Context private ServletContext ctxt;
+
+  private static final String COMMA_DELIMITER = ",";
+  private static final String COLON_DELIMITER = ":";
 
   private void init(HttpServletResponse response) {
     response.setContentType(null);
   }
 
+  private static Set<String> parseValuesStr(String str, String delimiter) {
+    if (str == null || str.isEmpty()) {
+      return null;
+    }
+    Set<String> strSet = new HashSet<String>();
+    String[] strs = str.split(delimiter);
+    for (String aStr : strs) {
+      strSet.add(aStr.trim());
+    }
+    return strSet;
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <T> void parseKeyValues(Map<String,T> map, String str,
+      String pairsDelim, String keyValuesDelim, boolean stringValue,
+      boolean multipleValues) {
+    String[] pairs = str.split(pairsDelim);
+    for (String pair : pairs) {
+      if (pair == null || pair.trim().isEmpty()) {
+        continue;
+      }
+      String[] pairStrs = pair.split(keyValuesDelim);
+      if (pairStrs.length < 2) {
+        continue;
+      }
+      if (!stringValue) {
+        try {
+          Object value =
+              GenericObjectMapper.OBJECT_READER.readValue(pairStrs[1].trim());
+          map.put(pairStrs[0].trim(), (T) value);
+        } catch (IOException e) {
+          map.put(pairStrs[0].trim(), (T) pairStrs[1].trim());
+        }
+      } else {
+        String key = pairStrs[0].trim();
+        if (multipleValues) {
+          Set<String> values = new HashSet<String>();
+          for (int i = 1; i < pairStrs.length; i++) {
+            values.add(pairStrs[i].trim());
+          }
+          map.put(key, (T) values);
+        } else {
+          map.put(key, (T) pairStrs[1].trim());
+        }
+      }
+    }
+  }
+
+  private static Map<String, Set<String>> parseKeyStrValuesStr(String str,
+      String pairsDelim, String keyValuesDelim) {
+    if (str == null) {
+      return null;
+    }
+    Map<String, Set<String>> map = new HashMap<String, Set<String>>();
+    parseKeyValues(map, str,pairsDelim, keyValuesDelim, true, true);
+    return map;
+  }
+
+  private static Map<String, String> parseKeyStrValueStr(String str,
+      String pairsDelim, String keyValDelim) {
+    if (str == null) {
+      return null;
+    }
+    Map<String, String> map = new HashMap<String, String>();
+    parseKeyValues(map, str, pairsDelim, keyValDelim, true, false);
+    return map;
+  }
+
+  private static Map<String, Object> parseKeyStrValueObj(String str,
+      String pairsDelim, String keyValDelim) {
+    if (str == null) {
+      return null;
+    }
+    Map<String, Object> map = new HashMap<String, Object>();
+    parseKeyValues(map, str, pairsDelim, keyValDelim, false, false);
+    return map;
+  }
+
+  private static EnumSet<Field> parseFieldsStr(String str, String delimiter) {
+    if (str == null) {
+      return null;
+    }
+    String[] strs = str.split(delimiter);
+    EnumSet<Field> fieldList = EnumSet.noneOf(Field.class);
+    for (String s : strs) {
+      fieldList.add(Field.valueOf(s.trim().toUpperCase()));
+    }
+    return fieldList;
+  }
+
+  private static Long parseLongStr(String str) {
+    return str == null ? null : Long.parseLong(str.trim());
+  }
+
+  private static String parseStr(String str) {
+    return str == null ? null : str.trim();
+  }
+
+  private static UserGroupInformation getUser(HttpServletRequest req) {
+    String remoteUser = req.getRemoteUser();
+    UserGroupInformation callerUGI = null;
+    if (remoteUser != null) {
+      callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
+    }
+    return callerUGI;
+  }
+
+  private TimelineReaderManager getTimelineReaderManager() {
+    return (TimelineReaderManager)
+        ctxt.getAttribute(TimelineReaderServer.TIMELINE_READER_MANAGER_ATTR);
+  }
+
   /**
    * Return the description of the timeline reader web services.
    */
   @GET
-  @Produces({ MediaType.APPLICATION_JSON /* , MediaType.APPLICATION_XML */})
+  @Produces(MediaType.APPLICATION_JSON)
   public TimelineAbout about(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res) {
     init(res);
     return TimelineUtils.createTimelineAbout("Timeline Reader API");
   }
+
+  /**
+   * Return a set of entities that match the given parameters.
+   */
+  @GET
+  @Path("/entities/{clusterId}/{appId}/{entityType}")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Set<TimelineEntity> getEntities(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("clusterId") String clusterId,
+      @PathParam("appId") String appId,
+      @PathParam("entityType") String entityType,
+      @QueryParam("userId") String userId,
+      @QueryParam("flowId") String flowId,
+      @QueryParam("flowRunId") String flowRunId,
+      @QueryParam("limit") String limit,
+      @QueryParam("createdTimeStart") String createdTimeStart,
+      @QueryParam("createdTimeEnd") String createdTimeEnd,
+      @QueryParam("modifiedTimeStart") String modifiedTimeStart,
+      @QueryParam("modifiedTimeEnd") String modifiedTimeEnd,
+      @QueryParam("relatesto") String relatesTo,
+      @QueryParam("isrelatedto") String isRelatedTo,
+      @QueryParam("infofilters") String infofilters,
+      @QueryParam("conffilters") String conffilters,
+      @QueryParam("metricfilters") String metricfilters,
+      @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("fields") String fields) {
+    init(res);
+    TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
+    UserGroupInformation callerUGI = getUser(req);
+    try {
+      return timelineReaderManager.getEntities(
+          callerUGI != null && (userId == null || userId.isEmpty()) ?
+          callerUGI.getUserName().trim() : parseStr(userId),
+          parseStr(clusterId), parseStr(flowId),
+          parseLongStr(flowRunId), parseStr(appId), parseStr(entityType),
+          parseLongStr(limit), parseLongStr(createdTimeStart),
+          parseLongStr(createdTimeEnd), parseLongStr(modifiedTimeStart),
+          parseLongStr(modifiedTimeEnd),
+          parseKeyStrValuesStr(relatesTo, COMMA_DELIMITER, COLON_DELIMITER),
+          parseKeyStrValuesStr(isRelatedTo, COMMA_DELIMITER, COLON_DELIMITER),
+          parseKeyStrValueObj(infofilters, COMMA_DELIMITER, COLON_DELIMITER),
+          parseKeyStrValueStr(conffilters, COMMA_DELIMITER, COLON_DELIMITER),
+          parseValuesStr(metricfilters, COMMA_DELIMITER),
+          parseValuesStr(eventfilters, COMMA_DELIMITER),
+          parseFieldsStr(fields, COMMA_DELIMITER));
+    } catch (NumberFormatException e) {
+      throw new BadRequestException(
+          "createdTime or modifiedTime start/end or limit or flowId is not" +
+          " a numeric value.");
+    } catch (IllegalArgumentException e) {
+      throw new BadRequestException("Requested Invalid Field.");
+    } catch (Exception e) {
+      LOG.error("Error getting entities", e);
+      throw new WebApplicationException(e,
+          Response.Status.INTERNAL_SERVER_ERROR);
+    }
+  }
+
+  /**
+   * Return a single entity of the given entity type and Id.
+   */
+  @GET
+  @Path("/entity/{clusterId}/{appId}/{entityType}/{entityId}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public TimelineEntity getEntity(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("clusterId") String clusterId,
+      @PathParam("appId") String appId,
+      @PathParam("entityType") String entityType,
+      @PathParam("entityId") String entityId,
+      @QueryParam("userId") String userId,
+      @QueryParam("flowId") String flowId,
+      @QueryParam("flowRunId") String flowRunId,
+      @QueryParam("fields") String fields) {
+    init(res);
+    TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
+    UserGroupInformation callerUGI = getUser(req);
+    TimelineEntity entity = null;
+    try {
+      entity = timelineReaderManager.getEntity(
+          callerUGI != null && (userId == null || userId.isEmpty()) ?
+          callerUGI.getUserName().trim() : parseStr(userId),
+          parseStr(clusterId), parseStr(flowId), parseLongStr(flowRunId),
+          parseStr(appId), parseStr(entityType), parseStr(entityId),
+          parseFieldsStr(fields, COMMA_DELIMITER));
+    } catch (NumberFormatException e) {
+      throw new BadRequestException("flowRunId is not a numeric value.");
+    } catch (IllegalArgumentException e) {
+      throw new BadRequestException("Requested Invalid Field.");
+    } catch (Exception e) {
+      LOG.error("Error getting entity", e);
+      throw new WebApplicationException(e,
+          Response.Status.INTERNAL_SERVER_ERROR);
+    }
+    if (entity == null) {
+      throw new NotFoundException("Timeline entity {id: " + parseStr(entityId) +
+          ", type: " + parseStr(entityType) + " } is not found");
+    }
+    return entity;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01f5e189/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 45ddd1d..626c770 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
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.timelineservice.storage;
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.nio.charset.Charset;
@@ -397,6 +398,10 @@ public class FileSystemTimelineReaderImpl extends AbstractService
                  new FileInputStream(entityFile), Charset.forName("UTF-8")))) {
       TimelineEntity entity = readEntityFromFile(reader);
       return createEntityToBeReturned(entity, fieldsToRetrieve);
+    } catch (FileNotFoundException e) {
+      LOG.info("Cannot find entity {id:" + entityId + " , type:" + entityType +
+          "}. Will send HTTP 404 in response.");
+      return null;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01f5e189/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 a9145d0..0f7c22f 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
@@ -18,25 +18,37 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.reader;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+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.Set;
 
 import javax.ws.rs.core.MediaType;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TestFileSystemTimelineReaderImpl;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 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;
@@ -46,12 +58,23 @@ public class TestTimelineReaderWebServices {
   private int serverPort;
   private TimelineReaderServer server;
 
+  @BeforeClass
+  public static void setup() throws Exception {
+    TestFileSystemTimelineReaderImpl.setup();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    TestFileSystemTimelineReaderImpl.tearDown();
+  }
+
   @Before
   public void init() throws Exception {
     try {
       Configuration config = new YarnConfiguration();
       config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, 
           "localhost:0");
+      config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
       server = new TimelineReaderServer();
       server.init(config);
       server.start();
@@ -69,6 +92,22 @@ public class TestTimelineReaderWebServices {
     }
   }
 
+  private static TimelineEntity newEntity(String type, String id) {
+    TimelineEntity entity = new TimelineEntity();
+    entity.setIdentifier(new TimelineEntity.Identifier(type, id));
+    return entity;
+  }
+
+  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().equals(status));
+  }
+
   private static Client createClient() {
     ClientConfig cfg = new DefaultClientConfig();
     cfg.getClasses().add(YarnJacksonJaxbJsonProvider.class);
@@ -76,14 +115,19 @@ public class TestTimelineReaderWebServices {
         new DummyURLConnectionFactory()), cfg);
   }
 
-  private static ClientResponse getResponse(Client client, URI uri) throws Exception {
+  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) {
-       System.out.println(resp.getClientResponseStatus());
-      throw new IOException("Incorrect response from timeline reader.");
+      String msg = new String();
+      if (resp != null) {
+        msg = resp.getClientResponseStatus().toString();
+      }
+      throw new IOException("Incorrect response from timeline reader. " +
+          "Status=" + msg);
     }
     return resp;
   }
@@ -102,8 +146,7 @@ public class TestTimelineReaderWebServices {
   }
 
   @Test
-  public void testAbout()
-      throws IOException {
+  public void testAbout() throws Exception {
     URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/");
     Client client = createClient();
     try {
@@ -111,9 +154,406 @@ public class TestTimelineReaderWebServices {
       TimelineAbout about = resp.getEntity(TimelineAbout.class);
       Assert.assertNotNull(about);
       Assert.assertEquals("Timeline Reader API", about.getAbout());
-    } catch (Exception re) {
-      throw new IOException(
-          "Failed to get the response from timeline reader.", re);
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntityDefaultView() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entity/cluster1/app1/app/id_1");
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entity);
+      assertEquals("id_1", entity.getId());
+      assertEquals("app", entity.getType());
+      assertEquals(1425016502000L, entity.getCreatedTime());
+      assertEquals(1425016503000L, entity.getModifiedTime());
+      // Default view i.e. when no fields are specified, entity contains only
+      // entity id, entity type, created and modified time.
+      assertEquals(0, entity.getConfigs().size());
+      assertEquals(0, entity.getMetrics().size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntityWithUserAndFlowInfo() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entity/cluster1/app1/app/id_1?userId=user1&" +
+          "flowId=flow1&flowRunId=1");
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entity);
+      assertEquals("id_1", entity.getId());
+      assertEquals("app", entity.getType());
+      assertEquals(1425016502000L, entity.getCreatedTime());
+      assertEquals(1425016503000L, entity.getModifiedTime());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntityCustomFields() throws Exception {
+    Client client = createClient();
+    try {
+      // Fields are case insensitive.
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entity/cluster1/app1/app/id_1?fields=CONFIGS,Metrics,info");
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entity);
+      assertEquals("id_1", entity.getId());
+      assertEquals("app", entity.getType());
+      assertEquals(3, entity.getConfigs().size());
+      assertEquals(3, entity.getMetrics().size());
+      assertEquals(1, entity.getInfo().size());
+      // No events will be returned as events are not part of fields.
+      assertEquals(0, entity.getEvents().size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntityAllFields() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entity/cluster1/app1/app/id_1?fields=ALL");
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entity);
+      assertEquals("id_1", entity.getId());
+      assertEquals("app", entity.getType());
+      assertEquals(3, entity.getConfigs().size());
+      assertEquals(3, entity.getMetrics().size());
+      assertEquals(1, entity.getInfo().size());
+      assertEquals(2, entity.getEvents().size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntityNotPresent() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entity/cluster1/app1/app/id_10");
+      verifyHttpResponse(client, uri, Status.NOT_FOUND);
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntities() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/cluster1/app1/app");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(4, entities.size());
+      assertTrue("Entities id_1, id_2, id_3 and id_4 should have been" +
+          " present in response",
+          entities.contains(newEntity("app", "id_1")) &&
+          entities.contains(newEntity("app", "id_2")) &&
+          entities.contains(newEntity("app", "id_3")) &&
+          entities.contains(newEntity("app", "id_4")));
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesWithLimit() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/cluster1/app1/app?limit=2");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      // Entities returned are based on most recent created time.
+      assertTrue("Entities with id_1 and id_4 should have been present " +
+          "in response based on entity created time.",
+          entities.contains(newEntity("app", "id_1")) &&
+          entities.contains(newEntity("app", "id_4")));
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "entities/cluster1/app1/app?limit=3");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      // Even though 2 entities out of 4 have same created time, one entity
+      // is left out due to limit
+      assertEquals(3, entities.size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesBasedOnCreatedTime() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/cluster1/app1/app?createdTimeStart=1425016502030&"
+          + "createdTimeEnd=1425016502060");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      assertTrue("Entity with id_4 should have been present in response.",
+          entities.contains(newEntity("app", "id_4")));
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "entities/cluster1/app1/app?createdTimeEnd=1425016502010");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(3, entities.size());
+      assertFalse("Entity with id_4 should not have been present in response.",
+          entities.contains(newEntity("app", "id_4")));
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "entities/cluster1/app1/app?createdTimeStart=1425016502010");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      assertTrue("Entity with id_4 should have been present in response.",
+          entities.contains(newEntity("app", "id_4")));
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesBasedOnModifiedTime() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/cluster1/app1/app?modifiedTimeStart=1425016502090"
+          + "&modifiedTimeEnd=1425016503020");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      assertTrue("Entities with id_1 and id_4 should have been" +
+          " present in response.",
+          entities.contains(newEntity("app", "id_1")) &&
+          entities.contains(newEntity("app", "id_4")));
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "entities/cluster1/app1/app?modifiedTimeEnd=1425016502090");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      assertTrue("Entities with id_2 and id_3 should have been " +
+          "present in response.",
+          entities.contains(newEntity("app", "id_2")) &&
+          entities.contains(newEntity("app", "id_3")));
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "entities/cluster1/app1/app?modifiedTimeStart=1425016503005");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      assertTrue("Entity with id_4 should have been present in response.",
+          entities.contains(newEntity("app", "id_4")));
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesByRelations() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/cluster1/app1/app?relatesto=flow:flow1");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      assertTrue("Entity with id_1 should have been present in response.",
+          entities.contains(newEntity("app", "id_1")));
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "entities/cluster1/app1/app?isrelatedto=type1:tid1_2,type2:" +
+          "tid2_1%60");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      assertTrue("Entity with id_1 should have been present in response.",
+          entities.contains(newEntity("app", "id_1")));
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "entities/cluster1/app1/app?isrelatedto=type1:tid1_1:tid1_2" +
+          ",type2:tid2_1%60");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      assertTrue("Entity with id_1 should have been present in response.",
+          entities.contains(newEntity("app", "id_1")));
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesByConfigFilters() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/cluster1/app1/app?conffilters=config_1:123," +
+          "config_3:abc");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      assertTrue("Entities with id_1 and id_3 should have been present" +
+          " in response.",
+          entities.contains(newEntity("app", "id_1")) &&
+          entities.contains(newEntity("app", "id_3")));
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesByInfoFilters() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/cluster1/app1/app?infofilters=info2:3.5");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      assertTrue("Entity with id_3 should have been present in response.",
+          entities.contains(newEntity("app", "id_3")));
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesByMetricFilters() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/cluster1/app1/app?metricfilters=metric3");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      assertTrue("Entities with id_1 and id_2 should have been present" +
+          " in response.",
+          entities.contains(newEntity("app", "id_1")) &&
+          entities.contains(newEntity("app", "id_2")));
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesByEventFilters() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/cluster1/app1/app?eventfilters=event_2,event_4");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      assertTrue("Entity with id_3 should have been present in response.",
+          entities.contains(newEntity("app", "id_3")));
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesNoMatch() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/cluster1/app1/app?metricfilters=metric7&" +
+          "isrelatedto=type1:tid1_1;tid1_2,type2:tid2_1%60&relatesto=" +
+          "flow:flow1&eventfilters=event_2,event_4&infofilters=info2:3.5" +
+          "&createdTimeStart=1425016502030&createdTimeEnd=1425016502060");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testInvalidValuesHandling() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/cluster1/app1/app?flowRunId=a23b");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "entity/cluster1/app1/app/id_1?flowRunId=2ab15");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "entities/cluster1/app1/app/?limit=#$561av");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
     } finally {
       client.destroy();
     }


[09/50] [abbrv] hadoop git commit: YARN-3906. Split the application table from the entity table. Contributed by Sangjin Lee.

Posted by gt...@apache.org.
YARN-3906. Split the application table from the entity table. Contributed by Sangjin Lee.

(cherry picked from commit bcd755eba9466ce277d3c14192c31da6462c4ab3)


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

Branch: refs/heads/feature-YARN-2928
Commit: 3007622955ba25dc4aba5b449dba5b3e80ade349
Parents: 0bed3fb
Author: Junping Du <ju...@apache.org>
Authored: Tue Aug 11 16:59:21 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:37:47 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../storage/HBaseTimelineReaderImpl.java        | 202 ++++++++----
 .../storage/HBaseTimelineWriterImpl.java        | 145 ++++++---
 .../storage/TimelineSchemaCreator.java          |  10 +
 .../storage/application/ApplicationColumn.java  | 136 ++++++++
 .../application/ApplicationColumnFamily.java    |  65 ++++
 .../application/ApplicationColumnPrefix.java    | 217 +++++++++++++
 .../storage/application/ApplicationRowKey.java  |  67 ++++
 .../storage/application/ApplicationTable.java   | 164 ++++++++++
 .../storage/application/package-info.java       |  25 ++
 .../storage/entity/EntityColumnPrefix.java      |   2 +-
 .../storage/entity/EntityTable.java             |  59 ++--
 .../storage/TestHBaseTimelineWriterImpl.java    | 321 ++++++++++++++++---
 13 files changed, 1230 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/30076229/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0378df8..c3a8172 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -85,6 +85,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3049. [Storage Implementation] Implement storage reader interface to
     fetch raw data from HBase backend (Zhijie Shen via sjlee)
 
+    YARN-3906. Split the application table from the entity table. (Sangjin Lee 
+    via junping_du)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30076229/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.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/HBaseTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
index 5258b9c..094f868 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
@@ -18,7 +18,19 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage;
 
 
-import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.TreeSet;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -32,11 +44,17 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.service.AbstractService;
 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.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+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;
+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.AppToFlowRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineReaderUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
@@ -45,18 +63,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.EntityTable;
 
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.Set;
-import java.util.TreeSet;
+import com.google.common.base.Preconditions;
 
 public class HBaseTimelineReaderImpl
     extends AbstractService implements TimelineReader {
@@ -70,6 +77,7 @@ public class HBaseTimelineReaderImpl
   private Connection conn;
   private EntityTable entityTable;
   private AppToFlowTable appToFlowTable;
+  private ApplicationTable applicationTable;
 
   public HBaseTimelineReaderImpl() {
     super(HBaseTimelineReaderImpl.class.getName());
@@ -82,6 +90,7 @@ public class HBaseTimelineReaderImpl
     conn = ConnectionFactory.createConnection(hbaseConf);
     entityTable = new EntityTable();
     appToFlowTable = new AppToFlowTable();
+    applicationTable = new ApplicationTable();
   }
 
   @Override
@@ -109,14 +118,24 @@ public class HBaseTimelineReaderImpl
       fieldsToRetrieve = EnumSet.noneOf(Field.class);
     }
 
-    byte[] rowKey = EntityRowKey.getRowKey(
-        clusterId, userId, flowId, flowRunId, appId, entityType, entityId);
+    boolean isApplication = isApplicationEntity(entityType);
+    byte[] rowKey = isApplication ?
+        ApplicationRowKey.getRowKey(clusterId, userId, flowId, flowRunId,
+            appId) :
+        EntityRowKey.getRowKey(clusterId, userId, flowId, flowRunId, appId,
+            entityType, entityId);
     Get get = new Get(rowKey);
     get.setMaxVersions(Integer.MAX_VALUE);
-    return parseEntity(
-        entityTable.getResult(hbaseConf, conn, get), fieldsToRetrieve,
+    Result result = isApplication ?
+        applicationTable.getResult(hbaseConf, conn, get) :
+        entityTable.getResult(hbaseConf, conn, get);
+    return parseEntity(result, fieldsToRetrieve,
         false, DEFAULT_BEGIN_TIME, DEFAULT_END_TIME, false, DEFAULT_BEGIN_TIME,
-        DEFAULT_END_TIME, null, null, null, null, null, null);
+        DEFAULT_END_TIME, null, null, null, null, null, null, isApplication);
+  }
+
+  private static boolean isApplicationEntity(String entityType) {
+    return TimelineEntityType.YARN_APPLICATION.toString().equals(entityType);
   }
 
   @Override
@@ -155,26 +174,46 @@ public class HBaseTimelineReaderImpl
     }
 
     NavigableSet<TimelineEntity> entities = new TreeSet<>();
-    // Scan through part of the table to find the entities belong to one app and
-    // one type
-    Scan scan = new Scan();
-    scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
-        clusterId, userId, flowId, flowRunId, appId, entityType));
-    scan.setMaxVersions(Integer.MAX_VALUE);
-    ResultScanner scanner = entityTable.getResultScanner(hbaseConf, conn, scan);
-    for (Result result : scanner) {
+    boolean isApplication = isApplicationEntity(entityType);
+    if (isApplication) {
+      // If getEntities() is called for an application, there can be at most
+      // one entity. If the entity passes the filter, it is returned. Otherwise,
+      // an empty set is returned.
+      byte[] rowKey = ApplicationRowKey.getRowKey(clusterId, userId, flowId,
+          flowRunId, appId);
+      Get get = new Get(rowKey);
+      get.setMaxVersions(Integer.MAX_VALUE);
+      Result result = applicationTable.getResult(hbaseConf, conn, get);
       TimelineEntity entity = parseEntity(result, fieldsToRetrieve,
-          true, createdTimeBegin, createdTimeEnd,
-          true, modifiedTimeBegin, modifiedTimeEnd,
-          isRelatedTo, relatesTo, infoFilters, configFilters, eventFilters,
-          metricFilters);
-      if (entity == null) {
-        continue;
+          true, createdTimeBegin, createdTimeEnd, true, modifiedTimeBegin,
+          modifiedTimeEnd, isRelatedTo, relatesTo, infoFilters, configFilters,
+          eventFilters, metricFilters, isApplication);
+      if (entity != null) {
+        entities.add(entity);
       }
-      if (entities.size() > limit) {
-        entities.pollLast();
+    } else {
+      // Scan through part of the table to find the entities belong to one app
+      // and one type
+      Scan scan = new Scan();
+      scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
+          clusterId, userId, flowId, flowRunId, appId, entityType));
+      scan.setMaxVersions(Integer.MAX_VALUE);
+      ResultScanner scanner =
+          entityTable.getResultScanner(hbaseConf, conn, scan);
+      for (Result result : scanner) {
+        TimelineEntity entity = parseEntity(result, fieldsToRetrieve,
+            true, createdTimeBegin, createdTimeEnd,
+            true, modifiedTimeBegin, modifiedTimeEnd,
+            isRelatedTo, relatesTo, infoFilters, configFilters, eventFilters,
+            metricFilters, isApplication);
+        if (entity == null) {
+          continue;
+        }
+        if (entities.size() > limit) {
+          entities.pollLast();
+        }
+        entities.add(entity);
       }
-      entities.add(entity);
     }
     return entities;
   }
@@ -221,26 +260,37 @@ public class HBaseTimelineReaderImpl
       boolean checkModifiedTime, long modifiedTimeBegin, long modifiedTimeEnd,
       Map<String, Set<String>> isRelatedTo, Map<String, Set<String>> relatesTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
-      Set<String> eventFilters, Set<String> metricFilters)
+      Set<String> eventFilters, Set<String> metricFilters,
+      boolean isApplication)
           throws IOException {
     if (result == null || result.isEmpty()) {
       return null;
     }
     TimelineEntity entity = new TimelineEntity();
-    entity.setType(EntityColumn.TYPE.readResult(result).toString());
-    entity.setId(EntityColumn.ID.readResult(result).toString());
+    String entityType = isApplication ?
+        TimelineEntityType.YARN_APPLICATION.toString() :
+        EntityColumn.TYPE.readResult(result).toString();
+    entity.setType(entityType);
+    String entityId = isApplication ?
+        ApplicationColumn.ID.readResult(result).toString() :
+        EntityColumn.ID.readResult(result).toString();
+    entity.setId(entityId);
 
     // fetch created time
-    entity.setCreatedTime(
-        ((Number) EntityColumn.CREATED_TIME.readResult(result)).longValue());
+    Number createdTime = isApplication ?
+        (Number)ApplicationColumn.CREATED_TIME.readResult(result) :
+        (Number)EntityColumn.CREATED_TIME.readResult(result);
+    entity.setCreatedTime(createdTime.longValue());
     if (checkCreatedTime && (entity.getCreatedTime() < createdTimeBegin ||
         entity.getCreatedTime() > createdTimeEnd)) {
       return null;
     }
 
     // fetch modified time
-    entity.setCreatedTime(
-        ((Number) EntityColumn.MODIFIED_TIME.readResult(result)).longValue());
+    Number modifiedTime = isApplication ?
+        (Number)ApplicationColumn.MODIFIED_TIME.readResult(result) :
+        (Number)EntityColumn.MODIFIED_TIME.readResult(result);
+    entity.setModifiedTime(modifiedTime.longValue());
     if (checkModifiedTime && (entity.getModifiedTime() < modifiedTimeBegin ||
         entity.getModifiedTime() > modifiedTimeEnd)) {
       return null;
@@ -250,7 +300,13 @@ public class HBaseTimelineReaderImpl
     boolean checkIsRelatedTo = isRelatedTo != null && isRelatedTo.size() > 0;
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
-      readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO);
+      if (isApplication) {
+        readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO,
+            true);
+      } else {
+        readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO,
+            true);
+      }
       if (checkIsRelatedTo && !TimelineReaderUtils.matchRelations(
           entity.getIsRelatedToEntities(), isRelatedTo)) {
         return null;
@@ -265,7 +321,12 @@ public class HBaseTimelineReaderImpl
     boolean checkRelatesTo = relatesTo != null && relatesTo.size() > 0;
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
-      readRelationship(entity, result, EntityColumnPrefix.RELATES_TO);
+      if (isApplication) {
+        readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO,
+            false);
+      } else {
+        readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false);
+      }
       if (checkRelatesTo && !TimelineReaderUtils.matchRelations(
           entity.getRelatesToEntities(), relatesTo)) {
         return null;
@@ -280,7 +341,11 @@ public class HBaseTimelineReaderImpl
     boolean checkInfo = infoFilters != null && infoFilters.size() > 0;
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
-      readKeyValuePairs(entity, result, EntityColumnPrefix.INFO);
+      if (isApplication) {
+        readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false);
+      } else {
+        readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false);
+      }
       if (checkInfo &&
           !TimelineReaderUtils.matchFilters(entity.getInfo(), infoFilters)) {
         return null;
@@ -295,7 +360,11 @@ public class HBaseTimelineReaderImpl
     boolean checkConfigs = configFilters != null && configFilters.size() > 0;
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
-      readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG);
+      if (isApplication) {
+        readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true);
+      } else {
+        readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true);
+      }
       if (checkConfigs && !TimelineReaderUtils.matchFilters(
           entity.getConfigs(), configFilters)) {
         return null;
@@ -310,7 +379,7 @@ public class HBaseTimelineReaderImpl
     boolean checkEvents = eventFilters != null && eventFilters.size() > 0;
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
-      readEvents(entity, result);
+      readEvents(entity, result, isApplication);
       if (checkEvents && !TimelineReaderUtils.matchEventFilters(
           entity.getEvents(), eventFilters)) {
         return null;
@@ -325,7 +394,7 @@ public class HBaseTimelineReaderImpl
     boolean checkMetrics = metricFilters != null && metricFilters.size() > 0;
     if (fieldsToRetrieve.contains(Field.ALL) ||
         fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
-      readMetrics(entity, result);
+      readMetrics(entity, result, isApplication);
       if (checkMetrics && !TimelineReaderUtils.matchMetricFilters(
           entity.getMetrics(), metricFilters)) {
         return null;
@@ -338,15 +407,15 @@ public class HBaseTimelineReaderImpl
     return entity;
   }
 
-  private static void readRelationship(
-      TimelineEntity entity, Result result, EntityColumnPrefix prefix)
-          throws IOException {
+  private static <T> void readRelationship(
+      TimelineEntity entity, Result result, ColumnPrefix<T> prefix,
+      boolean isRelatedTo) throws IOException {
     // isRelatedTo and relatesTo are of type Map<String, Set<String>>
     Map<String, Object> columns = prefix.readResults(result);
     for (Map.Entry<String, Object> column : columns.entrySet()) {
       for (String id : Separator.VALUES.splitEncoded(
           column.getValue().toString())) {
-        if (prefix.equals(EntityColumnPrefix.IS_RELATED_TO)) {
+        if (isRelatedTo) {
           entity.addIsRelatedToEntity(column.getKey(), id);
         } else {
           entity.addRelatesToEntity(column.getKey(), id);
@@ -355,12 +424,12 @@ public class HBaseTimelineReaderImpl
     }
   }
 
-  private static void readKeyValuePairs(
-      TimelineEntity entity, Result result, EntityColumnPrefix prefix)
-          throws IOException {
+  private static <T> void readKeyValuePairs(
+      TimelineEntity entity, Result result, ColumnPrefix<T> prefix,
+      boolean isConfig) throws IOException {
     // info and configuration are of type Map<String, Object or String>
     Map<String, Object> columns = prefix.readResults(result);
-    if (prefix.equals(EntityColumnPrefix.CONFIG)) {
+    if (isConfig) {
       for (Map.Entry<String, Object> column : columns.entrySet()) {
         entity.addConfig(column.getKey(), column.getKey().toString());
       }
@@ -369,10 +438,11 @@ public class HBaseTimelineReaderImpl
     }
   }
 
-  private static void readEvents(TimelineEntity entity, Result result)
-      throws IOException {
+  private static void readEvents(TimelineEntity entity, Result result,
+      boolean isApplication) throws IOException {
     Map<String, TimelineEvent> eventsMap = new HashMap<>();
-    Map<String, Object> eventsResult =
+    Map<String, Object> eventsResult = isApplication ?
+        ApplicationColumnPrefix.EVENT.readResults(result) :
         EntityColumnPrefix.EVENT.readResults(result);
     for (Map.Entry<String,Object> eventResult : eventsResult.entrySet()) {
       Collection<String> tokens =
@@ -405,10 +475,16 @@ public class HBaseTimelineReaderImpl
     entity.addEvents(eventsSet);
   }
 
-  private static void readMetrics(TimelineEntity entity, Result result)
-      throws IOException {
-    NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
-        EntityColumnPrefix.METRIC.readResultsWithTimestamps(result);
+  private static void readMetrics(TimelineEntity entity, Result result,
+      boolean isApplication) throws IOException {
+    NavigableMap<String, NavigableMap<Long, Number>> metricsResult;
+    if (isApplication) {
+      metricsResult =
+          ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result);
+    } else {
+      metricsResult =
+          EntityColumnPrefix.METRIC.readResultsWithTimestamps(result);
+    }
     for (Map.Entry<String, NavigableMap<Long, Number>> metricResult:
         metricsResult.entrySet()) {
       TimelineMetric metric = new TimelineMetric();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30076229/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 5290415..96192cc 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
@@ -38,9 +38,14 @@ 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.storage.application.ApplicationColumn;
+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.AppToFlowRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
@@ -61,6 +66,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   private Connection conn;
   private TypedBufferedMutator<EntityTable> entityTable;
   private TypedBufferedMutator<AppToFlowTable> appToFlowTable;
+  private TypedBufferedMutator<ApplicationTable> applicationTable;
 
   private static final Log LOG = LogFactory
       .getLog(HBaseTimelineWriterImpl.class);
@@ -84,6 +90,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     conn = ConnectionFactory.createConnection(hbaseConf);
     entityTable = new EntityTable().getTableMutator(hbaseConf, conn);
     appToFlowTable = new AppToFlowTable().getTableMutator(hbaseConf, conn);
+    applicationTable = new ApplicationTable().getTableMutator(hbaseConf, conn);
   }
 
   /**
@@ -102,18 +109,20 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
         continue;
       }
 
-      byte[] rowKey =
+      // if the entity is the application, the destination is the application
+      // table
+      boolean isApplication = isApplicationEntity(te);
+      byte[] rowKey = isApplication ?
+          ApplicationRowKey.getRowKey(clusterId, userId, flowName, flowRunId,
+              appId) :
           EntityRowKey.getRowKey(clusterId, userId, flowName, flowRunId, appId,
               te.getType(), te.getId());
 
-      storeInfo(rowKey, te, flowVersion);
-      storeEvents(rowKey, te.getEvents());
-      storeConfig(rowKey, te.getConfigs());
-      storeMetrics(rowKey, te.getMetrics());
-      storeRelations(rowKey, te.getIsRelatedToEntities(),
-          EntityColumnPrefix.IS_RELATED_TO);
-      storeRelations(rowKey, te.getRelatesToEntities(),
-          EntityColumnPrefix.RELATES_TO);
+      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 (isApplicationCreated(te)) {
         onApplicationCreated(
@@ -123,9 +132,12 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     return putStatus;
   }
 
+  private static boolean isApplicationEntity(TimelineEntity te) {
+    return te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString());
+  }
+
   private static boolean isApplicationCreated(TimelineEntity te) {
-    if (te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString())) {
-      boolean isAppCreated = false;
+    if (isApplicationEntity(te)) {
       for (TimelineEvent event : te.getEvents()) {
         if (event.getId().equals(
             ApplicationMetricsConstants.CREATED_EVENT_TYPE)) {
@@ -145,41 +157,74 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
         rowKey, appToFlowTable, null, flowRunId);
   }
 
+  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 void storeRelations(byte[] rowKey,
+  private <T> void storeRelations(byte[] rowKey,
       Map<String, Set<String>> connectedEntities,
-      EntityColumnPrefix entityColumnPrefix) throws IOException {
+      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());
 
-      entityColumnPrefix.store(rowKey, entityTable, connectedEntity.getKey(),
-          null, compoundValue);
+      columnPrefix.store(rowKey, table, connectedEntity.getKey(), null,
+          compoundValue);
     }
   }
 
   /**
    * Stores information from the {@linkplain TimelineEntity} object
    */
-  private void storeInfo(byte[] rowKey, TimelineEntity te, String flowVersion)
-      throws IOException {
+  private void storeInfo(byte[] rowKey, TimelineEntity te, String flowVersion,
+      boolean isApplication) throws IOException {
 
-    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.MODIFIED_TIME.store(rowKey, entityTable, null,
-        te.getModifiedTime());
-    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, entry.getKey(),
-            null, entry.getValue());
+    if (isApplication) {
+      ApplicationColumn.ID.store(rowKey, applicationTable, null, te.getId());
+      ApplicationColumn.CREATED_TIME.store(rowKey, applicationTable, null,
+          te.getCreatedTime());
+      ApplicationColumn.MODIFIED_TIME.store(rowKey, applicationTable, null,
+          te.getModifiedTime());
+      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,
+              entry.getKey(), null, entry.getValue());
+        }
+      }
+    } else {
+      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.MODIFIED_TIME.store(rowKey, entityTable, null,
+          te.getModifiedTime());
+      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, entry.getKey(),
+              null, entry.getValue());
+        }
       }
     }
   }
@@ -187,14 +232,19 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   /**
    * stores the config information from {@linkplain TimelineEntity}
    */
-  private void storeConfig(byte[] rowKey, Map<String, String> config)
-      throws IOException {
+  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()) {
-      EntityColumnPrefix.CONFIG.store(rowKey, entityTable, entry.getKey(),
+      if (isApplication) {
+        ApplicationColumnPrefix.CONFIG.store(rowKey, applicationTable,
+          entry.getKey(), null, entry.getValue());
+      } else {
+        EntityColumnPrefix.CONFIG.store(rowKey, entityTable, entry.getKey(),
           null, entry.getValue());
+      }
     }
   }
 
@@ -202,16 +252,21 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
    * stores the {@linkplain TimelineMetric} information from the
    * {@linkplain TimelineEvent} object
    */
-  private void storeMetrics(byte[] rowKey, Set<TimelineMetric> metrics)
-      throws IOException {
+  private void storeMetrics(byte[] rowKey, Set<TimelineMetric> metrics,
+      boolean isApplication) throws IOException {
     if (metrics != null) {
       for (TimelineMetric metric : metrics) {
         String metricColumnQualifier = metric.getId();
         Map<Long, Number> timeseries = metric.getValues();
         for (Map.Entry<Long, Number> timeseriesEntry : timeseries.entrySet()) {
           Long timestamp = timeseriesEntry.getKey();
-          EntityColumnPrefix.METRIC.store(rowKey, entityTable,
+          if (isApplication) {
+            ApplicationColumnPrefix.METRIC.store(rowKey, applicationTable,
               metricColumnQualifier, timestamp, timeseriesEntry.getValue());
+          } else {
+            EntityColumnPrefix.METRIC.store(rowKey, entityTable,
+              metricColumnQualifier, timestamp, timeseriesEntry.getValue());
+          }
         }
       }
     }
@@ -220,8 +275,8 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   /**
    * Stores the events from the {@linkplain TimelineEvent} object
    */
-  private void storeEvents(byte[] rowKey, Set<TimelineEvent> events)
-      throws IOException {
+  private void storeEvents(byte[] rowKey, Set<TimelineEvent> events,
+      boolean isApplication) throws IOException {
     if (events != null) {
       for (TimelineEvent event : events) {
         if (event != null) {
@@ -258,8 +313,13 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
                 // convert back to string to avoid additional API on store.
                 String compoundColumnQualifier =
                     Bytes.toString(compoundColumnQualifierBytes);
-                EntityColumnPrefix.EVENT.store(rowKey, entityTable,
+                if (isApplication) {
+                  ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
                     compoundColumnQualifier, null, info.getValue());
+                } else {
+                  EntityColumnPrefix.EVENT.store(rowKey, entityTable,
+                    compoundColumnQualifier, null, info.getValue());
+                }
               } // for info: eventInfo
             }
           }
@@ -279,6 +339,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     // flush all buffered mutators
     entityTable.flush();
     appToFlowTable.flush();
+    applicationTable.flush();
   }
 
   /**
@@ -288,15 +349,19 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   @Override
   protected void serviceStop() throws Exception {
     if (entityTable != null) {
-      LOG.info("closing entity table");
+      LOG.info("closing the entity table");
       // The close API performs flushing and releases any resources held
       entityTable.close();
     }
     if (appToFlowTable != null) {
-      LOG.info("closing app_flow table");
+      LOG.info("closing the app_flow table");
       // The close API performs flushing and releases any resources held
       appToFlowTable.close();
     }
+    if (applicationTable != null) {
+      LOG.info("closing the application table");
+      applicationTable.close();
+    }
     if (conn != null) {
       LOG.info("closing the hbase Connection");
       conn.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30076229/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
index 2c3897d..3a22ed6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
@@ -37,6 +37,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.server.timelineservice.storage.application.ApplicationTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
 
@@ -76,6 +77,12 @@ public class TimelineSchemaCreator {
     if (StringUtils.isNotBlank(appToflowTableName)) {
       hbaseConf.set(AppToFlowTable.TABLE_NAME_CONF_NAME, appToflowTableName);
     }
+    // Grab the applicationTableName argument
+    String applicationTableName = commandLine.getOptionValue("a");
+    if (StringUtils.isNotBlank(applicationTableName)) {
+      hbaseConf.set(ApplicationTable.TABLE_NAME_CONF_NAME,
+          applicationTableName);
+    }
     createAllTables(hbaseConf);
   }
 
@@ -103,6 +110,8 @@ public class TimelineSchemaCreator {
 
     o = new Option("a2f", "appToflowTableName", true, "app to flow table name");
     o.setArgName("appToflowTableName");
+    o = new Option("a", "applicationTableName", true, "application table name");
+    o.setArgName("applicationTableName");
     o.setRequired(false);
     options.addOption(o);
 
@@ -132,6 +141,7 @@ public class TimelineSchemaCreator {
       }
       new EntityTable().createTable(admin, hbaseConf);
       new AppToFlowTable().createTable(admin, hbaseConf);
+      new ApplicationTable().createTable(admin, hbaseConf);
     } finally {
       if (conn != null) {
         conn.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30076229/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java
new file mode 100644
index 0000000..c028386
--- /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/application/ApplicationColumn.java
@@ -0,0 +1,136 @@
+/**
+ * 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.application;
+
+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.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+
+/**
+ * Identifies fully qualified columns for the {@link ApplicationTable}.
+ */
+public enum ApplicationColumn implements Column<ApplicationTable> {
+
+  /**
+   * App id
+   */
+  ID(ApplicationColumnFamily.INFO, "id"),
+
+  /**
+   * When the application was created.
+   */
+  CREATED_TIME(ApplicationColumnFamily.INFO, "created_time"),
+
+  /**
+   * When it was modified.
+   */
+  MODIFIED_TIME(ApplicationColumnFamily.INFO, "modified_time"),
+
+  /**
+   * The version of the flow that this app belongs to.
+   */
+  FLOW_VERSION(ApplicationColumnFamily.INFO, "flow_version");
+
+  private final ColumnHelper<ApplicationTable> column;
+  private final ColumnFamily<ApplicationTable> columnFamily;
+  private final String columnQualifier;
+  private final byte[] columnQualifierBytes;
+
+  private ApplicationColumn(ColumnFamily<ApplicationTable> 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<ApplicationTable>(columnFamily);
+  }
+
+  /**
+   * @return the column name value
+   */
+  private String getColumnQualifier() {
+    return columnQualifier;
+  }
+
+  public void store(byte[] rowKey,
+      TypedBufferedMutator<ApplicationTable> tableMutator, Long timestamp,
+      Object inputValue) throws IOException {
+    column.store(rowKey, tableMutator, columnQualifierBytes, timestamp,
+        inputValue);
+  }
+
+  public Object readResult(Result result) throws IOException {
+    return column.readResult(result, columnQualifierBytes);
+  }
+
+  /**
+   * 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/30076229/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.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/application/ApplicationColumnFamily.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnFamily.java
new file mode 100644
index 0000000..97e5f7b
--- /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/application/ApplicationColumnFamily.java
@@ -0,0 +1,65 @@
+/**
+ * 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.application;
+
+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 application table column families.
+ */
+public enum ApplicationColumnFamily implements ColumnFamily<ApplicationTable> {
+
+  /**
+   * 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.
+   */
+  private ApplicationColumnFamily(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/30076229/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
new file mode 100644
index 0000000..cd9e845
--- /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/application/ApplicationColumnPrefix.java
@@ -0,0 +1,217 @@
+/**
+ * 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.application;
+
+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.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+
+/**
+ * Identifies partially qualified columns for the application table.
+ */
+public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
+
+  /**
+   * To store TimelineEntity getIsRelatedToEntities values.
+   */
+  IS_RELATED_TO(ApplicationColumnFamily.INFO, "s"),
+
+  /**
+   * To store TimelineEntity getRelatesToEntities values.
+   */
+  RELATES_TO(ApplicationColumnFamily.INFO, "r"),
+
+  /**
+   * To store TimelineEntity info values.
+   */
+  INFO(ApplicationColumnFamily.INFO, "i"),
+
+  /**
+   * Lifecycle events for an application
+   */
+  EVENT(ApplicationColumnFamily.INFO, "e"),
+
+  /**
+   * Config column stores configuration with config key as the column name.
+   */
+  CONFIG(ApplicationColumnFamily.CONFIGS, null),
+
+  /**
+   * Metrics are stored with the metric name as the column name.
+   */
+  METRIC(ApplicationColumnFamily.METRICS, null);
+
+  private final ColumnHelper<ApplicationTable> column;
+  private final ColumnFamily<ApplicationTable> 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.
+   */
+  private ApplicationColumnPrefix(ColumnFamily<ApplicationTable> columnFamily,
+      String columnPrefix) {
+    column = new ColumnHelper<ApplicationTable>(columnFamily);
+    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
+   */
+  private String getColumnPrefix() {
+    return columnPrefix;
+  }
+
+  /*
+   * (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<ApplicationTable> tableMutator, String qualifier,
+      Long timestamp, Object inputValue) throws IOException {
+
+    // Null check
+    if (qualifier == null) {
+      throw new IOException("Cannot store column with null qualifier in "
+          + tableMutator.getName().getNameAsString());
+    }
+
+    byte[] columnQualifier =
+        ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
+
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue);
+  }
+
+  /*
+   * (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)
+   */
+  public Map<String, Object> readResults(Result result) throws IOException {
+    return column.readResults(result, columnPrefixBytes);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
+   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result)
+   */
+  public <V> NavigableMap<String, NavigableMap<Long, V>>
+      readResultsWithTimestamps(Result result) throws IOException {
+    return column.readResultsWithTimestamps(result, columnPrefixBytes);
+  }
+
+  /**
+   * 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/30076229/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java
new file mode 100644
index 0000000..5f3868b
--- /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/application/ApplicationRowKey.java
@@ -0,0 +1,67 @@
+/**
+ * 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.application;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+
+/**
+ * Represents a rowkey for the application table.
+ */
+public class ApplicationRowKey {
+  // TODO: more methods are needed for this class.
+
+  // TODO: API needs to be cleaned up.
+
+  /**
+   * Constructs a row key for the application table as follows:
+   * {@code clusterId!userName!flowId!flowRunId!AppId}
+   *
+   * @param clusterId
+   * @param userId
+   * @param flowId
+   * @param flowRunId
+   * @param appId
+   * @return byte array with the row key
+   */
+  public static byte[] getRowKey(String clusterId, String userId,
+      String flowId, Long flowRunId, String appId) {
+    byte[] first =
+        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId, userId,
+            flowId));
+    // Note that flowRunId is a long, so we can't encode them all at the same
+    // time.
+    byte[] second = Bytes.toBytes(ApplicationRowKey.invert(flowRunId));
+    byte[] third = Bytes.toBytes(appId);
+    return Separator.QUALIFIERS.join(first, second, third);
+  }
+
+  /**
+   * Converts a timestamp into its inverse timestamp to be used in (row) keys
+   * where we want to have the most recent timestamp in the top of the table
+   * (scans start at the most recent timestamp first).
+   *
+   * @param key value to be inverted so that the latest version will be first in
+   *          a scan.
+   * @return inverted long
+   */
+  public static long invert(Long key) {
+    return Long.MAX_VALUE - key;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30076229/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
new file mode 100644
index 0000000..d2a2cb9
--- /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/application/ApplicationTable.java
@@ -0,0 +1,164 @@
+/**
+ * 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.application;
+
+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 application table as column families info, config and metrics. Info
+ * stores information about a YARN application entity, config stores
+ * configuration data of a YARN application, metrics stores the metrics of a
+ * YARN application. This table is entirely analogous to the entity table but
+ * created for better performance.
+ *
+ * Example application table record:
+ *
+ * <pre>
+ * |-------------------------------------------------------------------------|
+ * |  Row       | Column Family                | Column Family| Column Family|
+ * |  key       | info                         | metrics      | config       |
+ * |-------------------------------------------------------------------------|
+ * | clusterId! | id:appId                     | metricId1:   | configKey1:  |
+ * | userName!  |                              | metricValue1 | configValue1 |
+ * | flowId!    | created_time:                | @timestamp1  |              |
+ * | flowRunId! | 1392993084018                |              | configKey2:  |
+ * | AppId      |                              | metriciD1:   | configValue2 |
+ * |            | modified_time:               | metricValue2 |              |
+ * |            | 1392995081012                | @timestamp2  |              |
+ * |            |                              |              |              |
+ * |            | i!infoKey:                   | metricId2:   |              |
+ * |            | infoValue                    | metricValue1 |              |
+ * |            |                              | @timestamp2  |              |
+ * |            | r!relatesToKey:              |              |              |
+ * |            | id3?id4?id5                  |              |              |
+ * |            |                              |              |              |
+ * |            | s!isRelatedToKey:            |              |              |
+ * |            | id7?id9?id6                  |              |              |
+ * |            |                              |              |              |
+ * |            | e!eventId?timestamp?infoKey: |              |              |
+ * |            | eventInfoValue               |              |              |
+ * |            |                              |              |              |
+ * |            | flowVersion:                 |              |              |
+ * |            | versionValue                 |              |              |
+ * |-------------------------------------------------------------------------|
+ * </pre>
+ */
+public class ApplicationTable extends BaseTable<ApplicationTable> {
+  /** application prefix */
+  private static final String PREFIX =
+      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";
+
+  /**
+   * config param name that specifies the TTL for metrics column family in
+   * application table
+   */
+  private static final String METRICS_TTL_CONF_NAME = PREFIX
+      + ".table.metrics.ttl";
+
+  /** default value for application table name */
+  private static final String DEFAULT_TABLE_NAME =
+      "timelineservice.application";
+
+  /** 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 = 1000;
+
+  private static final Log LOG = LogFactory.getLog(ApplicationTable.class);
+
+  public ApplicationTable() {
+    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 applicationTableDescp = new HTableDescriptor(table);
+    HColumnDescriptor infoCF =
+        new HColumnDescriptor(ApplicationColumnFamily.INFO.getBytes());
+    infoCF.setBloomFilterType(BloomType.ROWCOL);
+    applicationTableDescp.addFamily(infoCF);
+
+    HColumnDescriptor configCF =
+        new HColumnDescriptor(ApplicationColumnFamily.CONFIGS.getBytes());
+    configCF.setBloomFilterType(BloomType.ROWCOL);
+    configCF.setBlockCacheEnabled(true);
+    applicationTableDescp.addFamily(configCF);
+
+    HColumnDescriptor metricsCF =
+        new HColumnDescriptor(ApplicationColumnFamily.METRICS.getBytes());
+    applicationTableDescp.addFamily(metricsCF);
+    metricsCF.setBlockCacheEnabled(true);
+    // always keep 1 version (the latest)
+    metricsCF.setMinVersions(1);
+    metricsCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS);
+    metricsCF.setTimeToLive(hbaseConf.getInt(METRICS_TTL_CONF_NAME,
+        DEFAULT_METRICS_TTL));
+    applicationTableDescp
+        .setRegionSplitPolicyClassName("org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy");
+    applicationTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length",
+        TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH);
+    admin.createTable(applicationTableDescp,
+        TimelineHBaseSchemaConstants.getUsernameSplits());
+    LOG.info("Status of table creation for " + table.getNameAsString() + "="
+        + admin.tableExists(table));
+  }
+
+  /**
+   * @param metricsTTL time to live parameter for the metrics in this table.
+   * @param hbaseConf configuration 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/30076229/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/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/storage/application/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/storage/application/package-info.java
new file mode 100644
index 0000000..c60e6f5
--- /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/application/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.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.application;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30076229/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
index 58272ab..c8485c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
@@ -157,7 +157,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
    * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result)
    */
-  public <T> NavigableMap<String, NavigableMap<Long, T>>
+  public <V> NavigableMap<String, NavigableMap<Long, V>>
       readResultsWithTimestamps(Result result) throws IOException {
     return column.readResultsWithTimestamps(result, columnPrefixBytes);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30076229/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.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/entity/EntityTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
index f657a14..9a8bd8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
@@ -40,36 +40,35 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBas
  * Example entity table record:
  *
  * <pre>
- * |--------------------------------------------------------------------|
- * |  Row       | Column Family           | Column Family| Column Family|
- * |  key       | info                    | metrics      | config       |
- * |--------------------------------------------------------------------|
- * | userName!  | id:entityId             | metricId1:   | configKey1:  |
- * | clusterId! |                         | metricValue1 | configValue1 |
- * | flowId!    | type:entityType         | @timestamp1  |              |
- * | flowRunId! |                         |              | configKey2:  |
- * | AppId!     | created_time:           | metriciD1:   | configValue2 |
- * | entityType!| 1392993084018           | metricValue2 |              |
- * | entityId   |                         | @timestamp2  |              |
- * |            | modified_time:          |              |              |
- * |            | 1392995081012           | metricId2:   |              |
- * |            |                         | metricValue1 |              |
- * |            | i!infoKey:              | @timestamp2  |              |
- * |            | infoValue               |              |              |
- * |            |                         |              |              |
- * |            | r!relatesToKey:         |              |              |
- * |            | id3?id4?id5             |              |              |
- * |            |                         |              |              |
- * |            | s!isRelatedToKey        |              |              |
- * |            | id7?id9?id6             |              |              |
- * |            |                         |              |              |
- * |            | e!eventId?eventInfoKey: |              |              |
- * |            | eventInfoValue          |              |              |
- * |            | @timestamp              |              |              |
- * |            |                         |              |              |
- * |            | flowVersion:            |              |              |
- * |            | versionValue            |              |              |
- * |--------------------------------------------------------------------|
+ * |-------------------------------------------------------------------------|
+ * |  Row       | Column Family                | Column Family| Column Family|
+ * |  key       | info                         | metrics      | config       |
+ * |-------------------------------------------------------------------------|
+ * | userName!  | id:entityId                  | metricId1:   | configKey1:  |
+ * | clusterId! |                              | metricValue1 | configValue1 |
+ * | flowId!    | type:entityType              | @timestamp1  |              |
+ * | flowRunId! |                              |              | configKey2:  |
+ * | AppId!     | created_time:                | metriciD1:   | configValue2 |
+ * | entityType!| 1392993084018                | metricValue2 |              |
+ * | entityId   |                              | @timestamp2  |              |
+ * |            | modified_time:               |              |              |
+ * |            | 1392995081012                | metricId2:   |              |
+ * |            |                              | metricValue1 |              |
+ * |            | i!infoKey:                   | @timestamp2  |              |
+ * |            | infoValue                    |              |              |
+ * |            |                              |              |              |
+ * |            | r!relatesToKey:              |              |              |
+ * |            | id3?id4?id5                  |              |              |
+ * |            |                              |              |              |
+ * |            | s!isRelatedToKey             |              |              |
+ * |            | id7?id9?id6                  |              |              |
+ * |            |                              |              |              |
+ * |            | e!eventId?timestamp?infoKey: |              |              |
+ * |            | eventInfoValue               |              |              |
+ * |            |                              |              |              |
+ * |            | flowVersion:                 |              |              |
+ * |            | versionValue                 |              |              |
+ * |-------------------------------------------------------------------------|
  * </pre>
  */
 public class EntityTable extends BaseTable<EntityTable> {


[21/50] [abbrv] hadoop git commit: YARN-4237 Support additional queries for ATSv2 Web UI. Contributed by Varun Saxena.

Posted by gt...@apache.org.
YARN-4237 Support additional queries for ATSv2 Web UI. Contributed by
Varun Saxena.


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

Branch: refs/heads/feature-YARN-2928
Commit: f40b1b5b2991e182efad6a31b558be2fbc06438f
Parents: fdaa1e4
Author: Li Lu <gt...@apache.org>
Authored: Thu Oct 15 10:49:36 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:58 2016 -0800

----------------------------------------------------------------------
 .../reader/TimelineReaderWebServices.java       |  64 ++++++++++
 .../storage/FlowRunEntityReader.java            |  45 ++++++-
 .../storage/GenericEntityReader.java            |   3 -
 .../storage/TimelineEntityReader.java           |   3 +
 .../storage/flow/FlowRunRowKey.java             |  15 +++
 ...stTimelineReaderWebServicesHBaseStorage.java | 117 ++++++++++++++++++-
 6 files changed, 236 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f40b1b5b/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 610f74c..83062f3 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
@@ -450,6 +450,70 @@ public class TimelineReaderWebServices {
   }
 
   /**
+   * Return a set of flows runs for the given flow id.
+   * Cluster ID is not provided by client so default cluster ID has to be taken.
+   */
+  @GET
+  @Path("/flowruns/{flowid}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Set<TimelineEntity> getFlowRuns(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("flowid") String flowId,
+      @QueryParam("userid") String userId,
+      @QueryParam("limit") String limit,
+      @QueryParam("createdtimestart") String createdTimeStart,
+      @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("fields") String fields) {
+    return getFlowRuns(req, res, null, flowId, userId, limit, createdTimeStart,
+        createdTimeEnd, fields);
+  }
+
+  /**
+   * Return a set of flow runs for the given cluster and flow id.
+   */
+  @GET
+  @Path("/flowruns/{clusterid}/{flowid}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Set<TimelineEntity> getFlowRuns(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("clusterid") String clusterId,
+      @PathParam("flowid") String flowId,
+      @QueryParam("userid") String userId,
+      @QueryParam("limit") String limit,
+      @QueryParam("createdtimestart") String createdTimeStart,
+      @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("fields") String fields) {
+    String url = req.getRequestURI() +
+        (req.getQueryString() == null ? "" :
+            QUERY_STRING_SEP + req.getQueryString());
+    UserGroupInformation callerUGI = getUser(req);
+    LOG.info("Received URL " + url + " from user " + getUserName(callerUGI));
+    long startTime = Time.monotonicNow();
+    init(res);
+    TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
+    Set<TimelineEntity> entities = null;
+    try {
+      entities = timelineReaderManager.getEntities(
+          parseUser(callerUGI, userId), parseStr(clusterId), parseStr(flowId),
+          null, null, TimelineEntityType.YARN_FLOW_RUN.toString(),
+          parseLongStr(limit), parseLongStr(createdTimeStart),
+          parseLongStr(createdTimeEnd), null, null, null, null, null, null,
+          null, null, parseFieldsStr(fields, COMMA_DELIMITER));
+    } catch (Exception e) {
+      handleException(e, url, startTime, "createdTime start/end or limit");
+    }
+    long endTime = Time.monotonicNow();
+    if (entities == null) {
+      entities = Collections.emptySet();
+    }
+    LOG.info("Processed URL " + url +
+        " (Took " + (endTime - startTime) + " ms.)");
+    return entities;
+  }
+
+  /**
    * Return a list of flows for a given cluster id. Cluster ID is not
    * provided by client so default cluster ID has to be taken.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f40b1b5b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/FlowRunEntityReader.java
index 90ce28f..c4b4e91 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/FlowRunEntityReader.java
@@ -27,6 +27,8 @@ 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.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.PageFilter;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
@@ -56,7 +58,7 @@ class FlowRunEntityReader extends TimelineEntityReader {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, fieldsToRetrieve, false);
+        eventFilters, fieldsToRetrieve, true);
   }
 
   public FlowRunEntityReader(String userId, String clusterId,
@@ -79,11 +81,27 @@ class FlowRunEntityReader extends TimelineEntityReader {
     Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
     Preconditions.checkNotNull(userId, "userId shouldn't be null");
     Preconditions.checkNotNull(flowId, "flowId shouldn't be null");
-    Preconditions.checkNotNull(flowRunId, "flowRunId shouldn't be null");
+    if (singleEntityRead) {
+      Preconditions.checkNotNull(flowRunId, "flowRunId shouldn't be null");
+    }
   }
 
   @Override
   protected void augmentParams(Configuration hbaseConf, Connection conn) {
+    if (!singleEntityRead) {
+      if (fieldsToRetrieve == null) {
+        fieldsToRetrieve = EnumSet.noneOf(Field.class);
+      }
+      if (limit == null || limit < 0) {
+        limit = TimelineReader.DEFAULT_LIMIT;
+      }
+      if (createdTimeBegin == null) {
+        createdTimeBegin = DEFAULT_BEGIN_TIME;
+      }
+      if (createdTimeEnd == null) {
+        createdTimeEnd = DEFAULT_END_TIME;
+      }
+    }
   }
 
   @Override
@@ -99,8 +117,11 @@ class FlowRunEntityReader extends TimelineEntityReader {
   @Override
   protected ResultScanner getResults(Configuration hbaseConf,
       Connection conn) throws IOException {
-    throw new UnsupportedOperationException(
-        "multiple entity query is not supported");
+    Scan scan = new Scan();
+    scan.setRowPrefixFilter(
+        FlowRunRowKey.getRowKeyPrefix(clusterId, userId, flowId));
+    scan.setFilter(new PageFilter(limit));
+    return table.getResultScanner(hbaseConf, conn, scan);
   }
 
   @Override
@@ -108,13 +129,23 @@ class FlowRunEntityReader extends TimelineEntityReader {
     FlowRunEntity flowRun = new FlowRunEntity();
     flowRun.setUser(userId);
     flowRun.setName(flowId);
-    flowRun.setRunId(flowRunId);
+    if (singleEntityRead) {
+      flowRun.setRunId(flowRunId);
+    } else {
+      FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(result.getRow());
+      flowRun.setRunId(rowKey.getFlowRunId());
+    }
 
     // read the start time
     Number startTime = (Number)FlowRunColumn.MIN_START_TIME.readResult(result);
     if (startTime != null) {
       flowRun.setStartTime(startTime.longValue());
     }
+    if (!singleEntityRead && (flowRun.getStartTime() < createdTimeBegin ||
+        flowRun.getStartTime() > createdTimeEnd)) {
+      return null;
+    }
+
     // read the end time if available
     Number endTime = (Number)FlowRunColumn.MAX_END_TIME.readResult(result);
     if (endTime != null) {
@@ -128,7 +159,9 @@ class FlowRunEntityReader extends TimelineEntityReader {
     }
 
     // read metrics
-    readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC);
+    if (singleEntityRead || fieldsToRetrieve.contains(Field.METRICS)) {
+      readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC);
+    }
 
     // set the id
     flowRun.setId(flowRun.getId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f40b1b5b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.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/GenericEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
index c18966f..bbca209 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
@@ -60,9 +60,6 @@ class GenericEntityReader extends TimelineEntityReader {
   private static final EntityTable ENTITY_TABLE = new EntityTable();
   private static final Log LOG = LogFactory.getLog(GenericEntityReader.class);
 
-  protected static final long DEFAULT_BEGIN_TIME = 0L;
-  protected static final long DEFAULT_END_TIME = Long.MAX_VALUE;
-
   /**
    * Used to look up the flow context.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f40b1b5b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.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/TimelineEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
index d4a659c..adaf42e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
@@ -44,6 +44,9 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
  */
 abstract class TimelineEntityReader {
   private static final Log LOG = LogFactory.getLog(TimelineEntityReader.class);
+  protected static final long DEFAULT_BEGIN_TIME = 0L;
+  protected static final long DEFAULT_END_TIME = Long.MAX_VALUE;
+
   protected final boolean singleEntityRead;
 
   protected String userId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f40b1b5b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java
index 7ed3651..a14d2bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java
@@ -55,6 +55,21 @@ public class FlowRunRowKey {
   }
 
   /**
+   * Constructs a row key prefix for the flow run table as follows: {
+   * clusterId!userI!flowId!}
+   *
+   * @param clusterId
+   * @param userId
+   * @param flowId
+   * @return byte array with the row key prefix
+   */
+  public static byte[] getRowKeyPrefix(String clusterId, String userId,
+      String flowId) {
+    return Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId, userId,
+        flowId, ""));
+  }
+
+  /**
    * Constructs a row key for the entity table as follows: {
    * clusterId!userI!flowId!Inverted Flow Run Id}
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f40b1b5b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
index a89d2fc..f6a5090 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -172,11 +172,11 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     id = "application_11111111111111_2223";
     entity3.setId(id);
     entity3.setType(type);
-    cTime = 1425016501030L;
+    cTime = 1425016501037L;
     entity3.setCreatedTime(cTime);
     TimelineEvent event2 = new TimelineEvent();
     event2.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    event2.setTimestamp(1436512802030L);
+    event2.setTimestamp(1436512802037L);
     event2.addInfo("foo_event", "test");
     entity3.addEvent(event2);
     te3.addEntity(entity3);
@@ -364,6 +364,119 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     }
   }
 
+
+  @Test
+  public void testGetFlowRuns() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowruns/cluster1/flow_name?userid=user1");
+      ClientResponse resp = getResponse(client, uri);
+      Set<FlowRunEntity> entities =
+          resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      for (FlowRunEntity entity : entities) {
+        assertTrue("Id, run id or start time does not match.",
+            ((entity.getId().equals("user1@flow_name/1002345678919")) &&
+            (entity.getRunId() == 1002345678919L) &&
+            (entity.getStartTime() == 1425016501000L)) ||
+            ((entity.getId().equals("user1@flow_name/1002345678920")) &&
+            (entity.getRunId() == 1002345678920L) &&
+            (entity.getStartTime() == 1425016501034L)));
+        assertEquals(0, entity.getMetrics().size());
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowruns/cluster1/flow_name?userid=user1&limit=1");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (FlowRunEntity entity : entities) {
+        assertTrue("Id, run id or start time does not match.",
+            entity.getId().equals("user1@flow_name/1002345678920") &&
+            entity.getRunId() == 1002345678920L &&
+            entity.getStartTime() == 1425016501034L);
+        assertEquals(0, entity.getMetrics().size());
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowruns/cluster1/flow_name?userid=user1&" +
+          "createdtimestart=1425016501030");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (FlowRunEntity entity : entities) {
+        assertTrue("Id, run id or start time does not match.",
+            entity.getId().equals("user1@flow_name/1002345678920") &&
+            entity.getRunId() == 1002345678920L &&
+            entity.getStartTime() == 1425016501034L);
+        assertEquals(0, entity.getMetrics().size());
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowruns/cluster1/flow_name?userid=user1&" +
+          "createdtimestart=1425016500999&createdtimeend=1425016501035");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      for (FlowRunEntity entity : entities) {
+        assertTrue("Id, run id or start time does not match.",
+            ((entity.getId().equals("user1@flow_name/1002345678919")) &&
+            (entity.getRunId() == 1002345678919L) &&
+            (entity.getStartTime() == 1425016501000L)) ||
+            ((entity.getId().equals("user1@flow_name/1002345678920")) &&
+            (entity.getRunId() == 1002345678920L) &&
+            (entity.getStartTime() == 1425016501034L)));
+        assertEquals(0, entity.getMetrics().size());
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowruns/cluster1/flow_name?userid=user1&" +
+          "createdtimeend=1425016501030");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (FlowRunEntity entity : entities) {
+        assertTrue("Id, run id or start time does not match.",
+             entity.getId().equals("user1@flow_name/1002345678919") &&
+             entity.getRunId() == 1002345678919L &&
+             entity.getStartTime() == 1425016501000L);
+        assertEquals(0, entity.getMetrics().size());
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowruns/cluster1/flow_name?userid=user1&fields=metrics");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      for (FlowRunEntity entity : entities) {
+        assertTrue("Id, run id or start time does not match.",
+            ((entity.getId().equals("user1@flow_name/1002345678919")) &&
+            (entity.getRunId() == 1002345678919L) &&
+            (entity.getStartTime() == 1425016501000L) &&
+            (entity.getMetrics().size() == 2)) ||
+            ((entity.getId().equals("user1@flow_name/1002345678920")) &&
+            (entity.getRunId() == 1002345678920L) &&
+            (entity.getStartTime() == 1425016501034L) &&
+            (entity.getMetrics().size() == 0)));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
   @Test
   public void testGetFlows() throws Exception {
     Client client = createClient();


[45/50] [abbrv] hadoop git commit: YARN-3995. Some of the NM events are not getting published due race condition when AM container finishes in NM (Naganarasimha G R via sjlee)

Posted by gt...@apache.org.
YARN-3995. Some of the NM events are not getting published due race condition when AM container finishes in NM (Naganarasimha G R via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: 5157c306d8915e0cf07921b0f713508848ffe451
Parents: 0352b97
Author: Sangjin Lee <sj...@apache.org>
Authored: Mon Jan 11 10:09:34 2016 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 18:03:31 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../hadoop/yarn/conf/YarnConfiguration.java     |  5 ++++
 .../src/main/resources/yarn-default.xml         |  7 ++++++
 .../PerNodeTimelineCollectorsAuxService.java    | 25 +++++++++++++-------
 ...TestPerNodeTimelineCollectorsAuxService.java | 11 +++++----
 5 files changed, 38 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5157c306/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 5bac262..7827d77 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -208,6 +208,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4350. TestDistributedShell fails for V2 scenarios. (Naganarasimha G R
     via varunsaxena)
 
+    YARN-3995. Some of the NM events are not getting published due race
+    condition when AM container finishes in NM (Naganarasimha G R via sjlee)
+
 Trunk - Unreleased
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5157c306/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 61b37c6..6a12d8f 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
@@ -1756,6 +1756,11 @@ public class YarnConfiguration extends Configuration {
   public static final int
       DEFAULT_TIMELINE_SERVICE_WRITER_FLUSH_INTERVAL_SECONDS = 60;
 
+  public static final String ATS_APP_COLLECTOR_LINGER_PERIOD_IN_MS =
+      TIMELINE_SERVICE_PREFIX + "app-collector.linger-period.ms";
+
+  public static final int DEFAULT_ATS_APP_COLLECTOR_LINGER_PERIOD_IN_MS = 1000;
+
   // 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/5157c306/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 077fb5d..b521599 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
@@ -2074,6 +2074,13 @@
     <value>60</value>
   </property>
 
+  <property>
+    <description>Time period till which the application collector will be alive
+     in NM, after the  application master container finishes.</description>
+    <name>yarn.timeline-service.app-collector.linger-period.ms</name>
+    <value>1000</value>
+  </property>
+
   <!--  Shared Cache Configuration -->
 
   <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5157c306/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 0319e34..b738530 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
@@ -19,6 +19,9 @@
 package org.apache.hadoop.yarn.server.timelineservice.collector;
 
 import java.nio.ByteBuffer;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -54,6 +57,8 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
   private static final int SHUTDOWN_HOOK_PRIORITY = 30;
 
   private final NodeTimelineCollectorManager collectorManager;
+  private long collectorLingerPeriod;
+  private ScheduledExecutorService scheduler;
 
   public PerNodeTimelineCollectorsAuxService() {
     this(new NodeTimelineCollectorManager());
@@ -70,6 +75,10 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
     if (!YarnConfiguration.timelineServiceV2Enabled(conf)) {
       throw new YarnException("Timeline service v2 is not enabled");
     }
+    collectorLingerPeriod =
+        conf.getLong(YarnConfiguration.ATS_APP_COLLECTOR_LINGER_PERIOD_IN_MS,
+            YarnConfiguration.DEFAULT_ATS_APP_COLLECTOR_LINGER_PERIOD_IN_MS);
+    scheduler = Executors.newSingleThreadScheduledExecutor();
     collectorManager.init(conf);
     super.serviceInit(conf);
   }
@@ -82,6 +91,12 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
 
   @Override
   protected void serviceStop() throws Exception {
+    scheduler.shutdown();
+    if (!scheduler.awaitTermination(collectorLingerPeriod,
+        TimeUnit.MILLISECONDS)) {
+      LOG.warn(
+          "Scheduler terminated before removing the application collectors");
+    }
     collectorManager.stop();
     super.serviceStop();
   }
@@ -141,17 +156,11 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
     if (context.getContainerType() == ContainerType.APPLICATION_MASTER) {
       final ApplicationId appId =
           context.getContainerId().getApplicationAttemptId().getApplicationId();
-      new Thread(new Runnable() {
+      scheduler.schedule(new Runnable() {
         public void run() {
-          try {
-            // TODO Temporary Fix until solution for YARN-3995 is finalized.
-            Thread.sleep(1000l);
-          } catch (InterruptedException e) {
-            e.printStackTrace();
-          }
           removeApplication(appId);
         }
-      }).start();
+      }, collectorLingerPeriod, TimeUnit.MILLISECONDS);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5157c306/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.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/TestPerNodeTimelineCollectorsAuxService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
index 4fdf47e..f2775d5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
@@ -22,12 +22,14 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.Mockito.any;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.Shell;
@@ -45,8 +47,6 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorCon
 import org.junit.After;
 import org.junit.Test;
 
-import java.io.IOException;
-
 public class TestPerNodeTimelineCollectorsAuxService {
   private ApplicationAttemptId appAttemptId;
   private PerNodeTimelineCollectorsAuxService auxService;
@@ -103,8 +103,9 @@ public class TestPerNodeTimelineCollectorsAuxService {
     when(context.getContainerType()).thenReturn(
         ContainerType.APPLICATION_MASTER);
     auxService.stopContainer(context);
-
-    // TODO Temporary Fix until solution for YARN-3995 is finalized
+    // auxService should have the app's collector and need to remove only after
+    // a configured period
+    assertTrue(auxService.hasApplication(appAttemptId.getApplicationId()));
     for (int i = 0; i < 4; i++) {
       Thread.sleep(500l);
       if (!auxService.hasApplication(appAttemptId.getApplicationId())) {


[14/50] [abbrv] hadoop git commit: YARN-3901. Populate flow run data in the flow_run & flow activity tables (Vrushali C via sjlee)

Posted by gt...@apache.org.
YARN-3901. Populate flow run data in the flow_run & flow activity tables (Vrushali C via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: a469bfe7c5d0b71cc0a3cc4f010b17e8f4267872
Parents: 983e729
Author: Sangjin Lee <sj...@apache.org>
Authored: Thu Sep 17 10:34:52 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:57 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../hadoop-yarn-server-timelineservice/pom.xml  |  13 +
 .../storage/HBaseTimelineWriterImpl.java        | 179 ++++++-
 .../storage/TimelineSchemaCreator.java          |  22 +-
 .../storage/application/ApplicationColumn.java  |   5 +-
 .../application/ApplicationColumnPrefix.java    |  15 +-
 .../storage/apptoflow/AppToFlowColumn.java      |   6 +-
 .../timelineservice/storage/common/Column.java  |   6 +-
 .../storage/common/ColumnHelper.java            |  93 +++-
 .../storage/common/ColumnPrefix.java            |  28 +-
 .../storage/common/TimelineWriterUtils.java     | 185 +++++++
 .../storage/common/TimestampGenerator.java      | 112 +++++
 .../storage/common/package-info.java            |  24 -
 .../storage/entity/EntityColumn.java            |   6 +-
 .../storage/entity/EntityColumnPrefix.java      |  20 +-
 .../flow/AggregationCompactionDimension.java    |  63 +++
 .../storage/flow/AggregationOperation.java      |  87 ++++
 .../timelineservice/storage/flow/Attribute.java |  39 ++
 .../storage/flow/FlowActivityColumnFamily.java  |  54 +++
 .../storage/flow/FlowActivityColumnPrefix.java  | 243 ++++++++++
 .../storage/flow/FlowActivityRowKey.java        | 113 +++++
 .../storage/flow/FlowActivityTable.java         | 107 ++++
 .../storage/flow/FlowRunColumn.java             | 161 ++++++
 .../storage/flow/FlowRunColumnFamily.java       |  54 +++
 .../storage/flow/FlowRunColumnPrefix.java       | 239 +++++++++
 .../storage/flow/FlowRunCoprocessor.java        | 210 ++++++++
 .../storage/flow/FlowRunRowKey.java             |  50 ++
 .../storage/flow/FlowRunTable.java              | 141 ++++++
 .../storage/flow/FlowScanner.java               | 486 +++++++++++++++++++
 .../storage/TestHBaseTimelineStorage.java       |  28 +-
 .../storage/flow/TestFlowDataGenerator.java     | 213 ++++++++
 .../flow/TestHBaseStorageFlowActivity.java      | 372 ++++++++++++++
 .../storage/flow/TestHBaseStorageFlowRun.java   | 290 +++++++++++
 33 files changed, 3562 insertions(+), 105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 667efd7..98c5e94 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -103,6 +103,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4102. Add a "skip existing table" mode for timeline schema creator (Li
     Lu via sjlee)
 
+    YARN-3901. Populate flow run data in the flow_run & flow activity tables
+    (Vrushali C via sjlee)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
index 3c41bce..616ed06 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/pom.xml
@@ -180,6 +180,19 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-javadoc-plugin</artifactId>
+        <configuration>
+          <additionnalDependencies>
+            <additionnalDependency>
+              <groupId>junit</groupId>
+              <artifactId>junit</artifactId>
+              <version>4.11</version>
+            </additionnalDependency>
+          </additionnalDependencies>
+        </configuration>
+      </plugin>
     </plugins>
   </build>
 </project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/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 772002d..7c4a5da 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
@@ -33,11 +33,10 @@ import org.apache.hadoop.hbase.util.Bytes;
 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.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.TimelineWriteResponse;
-import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
 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;
@@ -53,23 +52,36 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
+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;
 
 /**
- * This implements a hbase based backend for storing application timeline entity
+ * This implements a hbase based backend for storing the timeline entity
  * information.
+ * It writes to multiple tables at the backend
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class HBaseTimelineWriterImpl extends AbstractService implements
     TimelineWriter {
 
+  private static final Log LOG = LogFactory
+      .getLog(HBaseTimelineWriterImpl.class);
+
   private Connection conn;
   private TypedBufferedMutator<EntityTable> entityTable;
   private TypedBufferedMutator<AppToFlowTable> appToFlowTable;
   private TypedBufferedMutator<ApplicationTable> applicationTable;
-
-  private static final Log LOG = LogFactory
-      .getLog(HBaseTimelineWriterImpl.class);
+  private TypedBufferedMutator<FlowActivityTable> flowActivityTable;
+  private TypedBufferedMutator<FlowRunTable> flowRunTable;
 
   public HBaseTimelineWriterImpl() {
     super(HBaseTimelineWriterImpl.class.getName());
@@ -91,6 +103,8 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     entityTable = new EntityTable().getTableMutator(hbaseConf, conn);
     appToFlowTable = new AppToFlowTable().getTableMutator(hbaseConf, conn);
     applicationTable = new ApplicationTable().getTableMutator(hbaseConf, conn);
+    flowRunTable = new FlowRunTable().getTableMutator(hbaseConf, conn);
+    flowActivityTable = new FlowActivityTable().getTableMutator(hbaseConf, conn);
   }
 
   /**
@@ -111,7 +125,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
 
       // if the entity is the application, the destination is the application
       // table
-      boolean isApplication = isApplicationEntity(te);
+      boolean isApplication = TimelineWriterUtils.isApplicationEntity(te);
       byte[] rowKey = isApplication ?
           ApplicationRowKey.getRowKey(clusterId, userId, flowName, flowRunId,
               appId) :
@@ -124,37 +138,139 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       storeMetrics(rowKey, te.getMetrics(), isApplication);
       storeRelations(rowKey, te, isApplication);
 
-      if (isApplicationCreated(te)) {
-        onApplicationCreated(
-            clusterId, userId, flowName, flowVersion, flowRunId, appId, te);
+      if (isApplication) {
+        if (TimelineWriterUtils.isApplicationCreated(te)) {
+          onApplicationCreated(clusterId, userId, flowName, flowVersion,
+              flowRunId, appId, te);
+        }
+        // if it's an application entity, store metrics
+        storeFlowMetricsAppRunning(clusterId, userId, flowName, flowRunId,
+            appId, te);
+        // if application has finished, store it's finish time and write final
+        // values
+        // of all metrics
+        if (TimelineWriterUtils.isApplicationFinished(te)) {
+          onApplicationFinished(clusterId, userId, flowName, flowVersion,
+              flowRunId, appId, te);
+        }
       }
     }
     return putStatus;
   }
 
-  private static boolean isApplicationEntity(TimelineEntity te) {
-    return te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString());
+  private void onApplicationCreated(String clusterId, String userId,
+      String flowName, String flowVersion, long flowRunId, String appId,
+      TimelineEntity te) throws IOException {
+    // store in App to flow table
+    storeInAppToFlowTable(clusterId, userId, flowName, flowVersion, flowRunId,
+        appId, te);
+    // store in flow run table
+    storeAppCreatedInFlowRunTable(clusterId, userId, flowName, flowVersion,
+        flowRunId, appId, te);
+    // store in flow activity table
+    storeInFlowActivityTable(clusterId, userId, flowName, flowVersion,
+        flowRunId, appId, te);
   }
 
-  private static boolean isApplicationCreated(TimelineEntity te) {
-    if (isApplicationEntity(te)) {
-      for (TimelineEvent event : te.getEvents()) {
-        if (event.getId().equals(
-            ApplicationMetricsConstants.CREATED_EVENT_TYPE)) {
-          return true;
-        }
-      }
-    }
-    return false;
+  /*
+   * updates the {@link FlowActivityTable} with the Application TimelineEntity
+   * information
+   */
+  private void storeInFlowActivityTable(String clusterId, String userId,
+      String flowName, String flowVersion, long flowRunId, String appId,
+      TimelineEntity te) throws IOException {
+    byte[] rowKey = FlowActivityRowKey.getRowKey(clusterId, userId, flowName);
+    byte[] qualifier = GenericObjectMapper.write(flowRunId);
+    FlowActivityColumnPrefix.RUN_ID.store(rowKey, flowActivityTable, qualifier,
+        null, flowVersion,
+        AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
   }
 
-  private void onApplicationCreated(String clusterId, String userId,
+  /*
+   * updates the {@link FlowRunTable} with Application Created information
+   */
+  private void storeAppCreatedInFlowRunTable(String clusterId, String userId,
+      String flowName, String flowVersion, long flowRunId, String appId,
+      TimelineEntity te) throws IOException {
+    byte[] rowKey = FlowRunRowKey.getRowKey(clusterId, userId, flowName,
+        flowRunId);
+    FlowRunColumn.MIN_START_TIME.store(rowKey, flowRunTable, null,
+        te.getCreatedTime(),
+        AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
+  }
+
+  private void storeInAppToFlowTable(String clusterId, String userId,
       String flowName, String flowVersion, long flowRunId, String appId,
       TimelineEntity te) throws IOException {
     byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
     AppToFlowColumn.FLOW_ID.store(rowKey, appToFlowTable, null, flowName);
-    AppToFlowColumn.FLOW_RUN_ID.store(
-        rowKey, appToFlowTable, null, flowRunId);
+    AppToFlowColumn.FLOW_RUN_ID.store(rowKey, appToFlowTable, null, flowRunId);
+  }
+
+  /*
+   * updates the {@link FlowRunTable} and {@link FlowActivityTable} when an
+   * application has finished
+   */
+  private void onApplicationFinished(String clusterId, String userId,
+      String flowName, String flowVersion, long flowRunId, String appId,
+      TimelineEntity te) throws IOException {
+    // store in flow run table
+    storeAppFinishedInFlowRunTable(clusterId, userId, flowName, flowRunId,
+        appId, te);
+
+    // indicate in the flow activity table that the app has finished
+    storeInFlowActivityTable(clusterId, userId, flowName, flowVersion,
+        flowRunId, appId, te);
+  }
+
+  /*
+   * Update the {@link FlowRunTable} with Application Finished information
+   */
+  private void storeAppFinishedInFlowRunTable(String clusterId, String userId,
+      String flowName, long flowRunId, String appId, TimelineEntity te)
+      throws IOException {
+    byte[] rowKey = FlowRunRowKey.getRowKey(clusterId, userId, flowName,
+        flowRunId);
+    Attribute attributeAppId = AggregationCompactionDimension.APPLICATION_ID
+        .getAttribute(appId);
+    FlowRunColumn.MAX_END_TIME.store(rowKey, flowRunTable, null,
+        TimelineWriterUtils.getApplicationFinishedTime(te), attributeAppId);
+
+    // store the final value of metrics since application has finished
+    Set<TimelineMetric> metrics = te.getMetrics();
+    if (metrics != null) {
+      storeFlowMetrics(rowKey, metrics, attributeAppId,
+          AggregationOperation.SUM_FINAL.getAttribute());
+    }
+  }
+
+  /*
+   * Updates the {@link FlowRunTable} with Application Metrics
+   */
+  private void storeFlowMetricsAppRunning(String clusterId, String userId,
+      String flowName, long flowRunId, String appId, TimelineEntity te)
+      throws IOException {
+    Set<TimelineMetric> metrics = te.getMetrics();
+    if (metrics != null) {
+      byte[] rowKey = FlowRunRowKey.getRowKey(clusterId, userId, flowName,
+          flowRunId);
+      storeFlowMetrics(rowKey, metrics,
+          AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
+    }
+  }
+
+  private void storeFlowMetrics(byte[] rowKey, Set<TimelineMetric> metrics,
+      Attribute... attributes) throws IOException {
+    for (TimelineMetric metric : metrics) {
+      String metricColumnQualifier = metric.getId();
+      Map<Long, Number> timeseries = metric.getValues();
+      for (Map.Entry<Long, Number> timeseriesEntry : timeseries.entrySet()) {
+        Long timestamp = timeseriesEntry.getKey();
+        FlowRunColumnPrefix.METRIC.store(rowKey, flowRunTable,
+            metricColumnQualifier, timestamp, timeseriesEntry.getValue(),
+            attributes);
+      }
+    }
   }
 
   private void storeRelations(byte[] rowKey, TimelineEntity te,
@@ -184,7 +300,6 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       // id3?id4?id5
       String compoundValue =
           Separator.VALUES.joinEncoded(connectedEntity.getValue());
-
       columnPrefix.store(rowKey, table, connectedEntity.getKey(), null,
           compoundValue);
     }
@@ -342,6 +457,8 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     entityTable.flush();
     appToFlowTable.flush();
     applicationTable.flush();
+    flowRunTable.flush();
+    flowActivityTable.flush();
   }
 
   /**
@@ -364,6 +481,16 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       LOG.info("closing the application table");
       applicationTable.close();
     }
+    if (flowRunTable != null) {
+      LOG.info("closing the flow run table");
+      // The close API performs flushing and releases any resources held
+      flowRunTable.close();
+    }
+    if (flowActivityTable != null) {
+      LOG.info("closing the flowActivityTable table");
+      // The close API performs flushing and releases any resources held
+      flowActivityTable.close();
+    }
     if (conn != null) {
       LOG.info("closing the hbase Connection");
       conn.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
index e7e51a7..cbcff4c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
@@ -42,6 +42,8 @@ import org.apache.hadoop.util.GenericOptionsParser;
 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.entity.EntityTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
 
 /**
  * This creates the schema for a hbase based backend for storing application
@@ -199,7 +201,7 @@ public class TimelineSchemaCreator {
     return commandLine;
   }
 
-  private static void createAllTables(Configuration hbaseConf,
+  public static void createAllTables(Configuration hbaseConf,
       boolean skipExisting) throws IOException {
 
     Connection conn = null;
@@ -236,6 +238,24 @@ public class TimelineSchemaCreator {
           throw e;
         }
       }
+      try {
+        new FlowRunTable().createTable(admin, hbaseConf);
+      } catch (IOException e) {
+        if (skipExisting) {
+          LOG.warn("Skip and continue on: " + e.getMessage());
+        } else {
+          throw e;
+        }
+      }
+      try {
+        new FlowActivityTable().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/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java
index c028386..802626d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumn.java
@@ -26,6 +26,7 @@ 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.flow.Attribute;
 
 /**
  * Identifies fully qualified columns for the {@link ApplicationTable}.
@@ -76,9 +77,9 @@ public enum ApplicationColumn implements Column<ApplicationTable> {
 
   public void store(byte[] rowKey,
       TypedBufferedMutator<ApplicationTable> tableMutator, Long timestamp,
-      Object inputValue) throws IOException {
+      Object inputValue, Attribute... attributes) throws IOException {
     column.store(rowKey, tableMutator, columnQualifierBytes, timestamp,
-        inputValue);
+        inputValue, attributes);
   }
 
   public Object readResult(Result result) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
index ad1def6..d7b5773 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
@@ -28,6 +28,7 @@ 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.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
 /**
  * Identifies partially qualified columns for the application table.
@@ -112,7 +113,8 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
    */
   public void store(byte[] rowKey,
       TypedBufferedMutator<ApplicationTable> tableMutator, byte[] qualifier,
-      Long timestamp, Object inputValue) throws IOException {
+      Long timestamp, Object inputValue, Attribute... attributes)
+      throws IOException {
 
     // Null check
     if (qualifier == null) {
@@ -123,8 +125,9 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
     byte[] columnQualifier =
         ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
 
-    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue);
-  }
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
+        attributes);
+ }
 
   /*
    * (non-Javadoc)
@@ -137,7 +140,8 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
    */
   public void store(byte[] rowKey,
       TypedBufferedMutator<ApplicationTable> tableMutator, String qualifier,
-      Long timestamp, Object inputValue) throws IOException {
+      Long timestamp, Object inputValue, Attribute...attributes)
+      throws IOException {
 
     // Null check
     if (qualifier == null) {
@@ -148,7 +152,8 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
     byte[] columnQualifier =
         ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
 
-    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue);
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
+        attributes);
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/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
index 423037a..859fdca 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/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
@@ -25,8 +25,10 @@ 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.flow.Attribute;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * Identifies fully qualified columns for the {@link AppToFlowTable}.
@@ -67,9 +69,9 @@ public enum AppToFlowColumn implements Column<AppToFlowTable> {
 
   public void store(byte[] rowKey,
       TypedBufferedMutator<AppToFlowTable> tableMutator, Long timestamp,
-      Object inputValue) throws IOException {
+      Object inputValue, Attribute... attributes) throws IOException {
     column.store(rowKey, tableMutator, columnQualifierBytes, timestamp,
-        inputValue);
+        inputValue, attributes);
   }
 
   public Object readResult(Result result) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Column.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/Column.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/Column.java
index 3397d62..64c1cda 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/Column.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/Column.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
 /**
  * A Column represents the way to store a fully qualified column in a specific
@@ -38,12 +39,15 @@ public interface Column<T> {
    *          column.
    * @param timestamp version timestamp. When null the server timestamp will be
    *          used.
+   * @param attributes Map of attributes for this mutation. used in the coprocessor
+   *          to set/read the cell tags. Can be null.
    * @param inputValue the value to write to the rowKey and column qualifier.
    *          Nothing gets written when null.
    * @throws IOException
    */
   public void store(byte[] rowKey, TypedBufferedMutator<T> tableMutator,
-      Long timestamp, Object inputValue) throws IOException;
+      Long timestamp, Object inputValue, Attribute... attributes)
+      throws IOException;
 
   /**
    * Get the latest version of this specified column. Note: this call clones the

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/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 f1b7c58..3a2e088 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
@@ -31,7 +31,8 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
-
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 /**
  * This class is meant to be used only by explicit Columns, and not directly to
  * write by clients.
@@ -58,31 +59,66 @@ public class ColumnHelper<T> {
    * Sends a Mutation to the table. The mutations will be buffered and sent over
    * the wire as part of a batch.
    *
-   * @param rowKey identifying the row to write. Nothing gets written when null.
-   * @param tableMutator used to modify the underlying HBase table
-   * @param columnQualifier column qualifier. Nothing gets written when null.
-   * @param timestamp version timestamp. When null the server timestamp will be
-   *          used.
-   * @param inputValue the value to write to the rowKey and column qualifier.
-   *          Nothing gets written when null.
+   * @param rowKey
+   *          identifying the row to write. Nothing gets written when null.
+   * @param tableMutator
+   *          used to modify the underlying HBase table
+   * @param columnQualifier
+   *          column qualifier. Nothing gets written when null.
+   * @param timestamp
+   *          version timestamp. When null the current timestamp multiplied with
+   *          TimestampGenerator.TS_MULTIPLIER and added with last 3 digits of
+   *          app id will be used
+   * @param inputValue
+   *          the value to write to the rowKey and column qualifier. Nothing
+   *          gets written when null.
    * @throws IOException
    */
   public void store(byte[] rowKey, TypedBufferedMutator<?> tableMutator,
-      byte[] columnQualifier, Long timestamp, Object inputValue)
-      throws IOException {
+      byte[] columnQualifier, Long timestamp, Object inputValue,
+      Attribute... attributes) throws IOException {
     if ((rowKey == null) || (columnQualifier == null) || (inputValue == null)) {
       return;
     }
     Put p = new Put(rowKey);
+    timestamp = getPutTimestamp(timestamp, attributes);
+    p.addColumn(columnFamilyBytes, columnQualifier, timestamp,
+        GenericObjectMapper.write(inputValue));
+    if ((attributes != null) && (attributes.length > 0)) {
+      for (Attribute attribute : attributes) {
+        p.setAttribute(attribute.getName(), attribute.getValue());
+      }
+    }
+    tableMutator.mutate(p);
+  }
 
+  /*
+   * 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
+   */
+  private long getPutTimestamp(Long timestamp, Attribute[] attributes) {
     if (timestamp == null) {
-      p.addColumn(columnFamilyBytes, columnQualifier,
-          GenericObjectMapper.write(inputValue));
-    } else {
-      p.addColumn(columnFamilyBytes, columnQualifier, timestamp,
-          GenericObjectMapper.write(inputValue));
+      timestamp = System.currentTimeMillis();
     }
-    tableMutator.mutate(p);
+    String appId = getAppIdFromAttributes(attributes);
+    long supplementedTS = TimestampGenerator.getSupplementedTimestamp(
+        timestamp, appId);
+    return supplementedTS;
+  }
+
+  private String getAppIdFromAttributes(Attribute[] attributes) {
+    if (attributes == null) {
+      return null;
+    }
+    String appId = null;
+    for (Attribute attribute : attributes) {
+      if (AggregationCompactionDimension.APPLICATION_ID.toString().equals(
+          attribute.getName())) {
+        appId = Bytes.toString(attribute.getValue());
+      }
+    }
+    return appId;
   }
 
   /**
@@ -171,7 +207,9 @@ public class ColumnHelper<T> {
               for (Entry<Long, byte[]> cell : cells.entrySet()) {
                 V value =
                     (V) GenericObjectMapper.read(cell.getValue());
-                cellResults.put(cell.getKey(), value);
+                cellResults.put(
+                    TimestampGenerator.getTruncatedTimestamp(cell.getKey()),
+                    value);
               }
             }
             results.put(columnName, cellResults);
@@ -315,6 +353,27 @@ 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.
+   * @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.
+   */
+  public static byte[] getColumnQualifier(byte[] columnPrefixBytes,
+      long qualifier) {
+
+    if (columnPrefixBytes == null) {
+      return Bytes.toBytes(qualifier);
+    }
+
+    // Convert qualifier to lower case, strip of separators and tag on column
+    // prefix.
+    byte[] columnQualifier =
+        Separator.QUALIFIERS.join(columnPrefixBytes, Bytes.toBytes(qualifier));
+    return columnQualifier;
+  }
+  /**
+   * @param columnPrefixBytes The byte representation for the column prefix.
+   *          Should not contain {@link Separator#QUALIFIERS}.
    * @param qualifier the byte representation for the remainder of the column.
    * @return fully sanitized column qualifier that is a combination of prefix
    *         and qualifier. If prefix is null, the result is simply the encoded

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.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/ColumnPrefix.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/ColumnPrefix.java
index 509ff49..db49098 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/ColumnPrefix.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/ColumnPrefix.java
@@ -23,6 +23,7 @@ import java.util.NavigableMap;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
 /**
  * Used to represent a partially qualified column, where the actual column name
@@ -43,12 +44,36 @@ public interface ColumnPrefix<T> {
    * @param qualifier column qualifier. Nothing gets written when null.
    * @param timestamp version timestamp. When null the server timestamp will be
    *          used.
+   *@param attributes attributes for the mutation that are used by the coprocessor
+   *          to set/read the cell tags
    * @param inputValue the value to write to the rowKey and column qualifier.
    *          Nothing gets written when null.
    * @throws IOException
    */
   public void store(byte[] rowKey, TypedBufferedMutator<T> tableMutator,
-      String qualifier, Long timestamp, Object inputValue) throws IOException;
+      byte[] qualifier, Long timestamp, Object inputValue,
+      Attribute... attributes) throws IOException;
+
+  /**
+   * Sends a Mutation to the table. The mutations will be buffered and sent over
+   * the wire as part of a batch.
+   *
+   * @param rowKey identifying the row to write. Nothing gets written when null.
+   * @param tableMutator used to modify the underlying HBase table. Caller is
+   *          responsible to pass a mutator for the table that actually has this
+   *          column.
+   * @param qualifier column qualifier. Nothing gets written when null.
+   * @param timestamp version timestamp. When null the server timestamp will be
+   *          used.
+   *@param attributes attributes for the mutation that are used by the coprocessor
+   *          to set/read the cell tags
+   * @param inputValue the value to write to the rowKey and column qualifier.
+   *          Nothing gets written when null.
+   * @throws IOException
+   */
+  public void store(byte[] rowKey, TypedBufferedMutator<T> tableMutator,
+      String qualifier, Long timestamp, Object inputValue,
+      Attribute... attributes) throws IOException;
 
   /**
    * Get the latest version of this specified column. Note: this call clones the
@@ -81,4 +106,5 @@ public interface ColumnPrefix<T> {
    */
   public <V> NavigableMap<String, NavigableMap<Long, V>>
       readResultsWithTimestamps(Result result) throws IOException;
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineWriterUtils.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/TimelineWriterUtils.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/TimelineWriterUtils.java
index 58bdedc7e..371371a 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/TimelineWriterUtils.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/TimelineWriterUtils.java
@@ -19,9 +19,19 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.common;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.SortedSet;
+import java.util.Map.Entry;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Tag;
+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.TimelineEvent;
+import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
 /**
  * bunch of utility functions used across TimelineWriter classes
@@ -36,6 +46,9 @@ public class TimelineWriterUtils {
   /** indicator for no limits for splitting */
   public static final int NO_LIMIT_SPLIT = -1;
 
+  /** milliseconds in one day */
+  public static final long MILLIS_ONE_DAY = 86400000L;
+
   /**
    * Splits the source array into multiple array segments using the given
    * separator, up to a maximum of count items. This will naturally produce
@@ -140,4 +153,176 @@ public class TimelineWriterUtils {
     return Long.MAX_VALUE - key;
   }
 
+  /**
+   * returns the timestamp of that day's start (which is midnight 00:00:00 AM)
+   * for a given input timestamp
+   *
+   * @param ts
+   * @return timestamp of that day's beginning (midnight)
+   */
+  public static long getTopOfTheDayTimestamp(long ts) {
+    long dayTimestamp = ts - (ts % MILLIS_ONE_DAY);
+    return dayTimestamp;
+  }
+
+  /**
+   * Combines the input array of attributes and the input aggregation operation
+   * into a new array of attributes.
+   *
+   * @param attributes
+   * @param aggOp
+   * @return array of combined attributes
+   */
+  public static Attribute[] combineAttributes(Attribute[] attributes,
+      AggregationOperation aggOp) {
+    int newLength = getNewLengthCombinedAttributes(attributes, aggOp);
+    Attribute[] combinedAttributes = new Attribute[newLength];
+
+    if (attributes != null) {
+      System.arraycopy(attributes, 0, combinedAttributes, 0, attributes.length);
+    }
+
+    if (aggOp != null) {
+      Attribute a2 = aggOp.getAttribute();
+      combinedAttributes[newLength - 1] = a2;
+    }
+    return combinedAttributes;
+  }
+
+  /**
+   * Returns a number for the new array size. The new array is the combination
+   * of input array of attributes and the input aggregation operation.
+   *
+   * @param attributes
+   * @param aggOp
+   * @return the size for the new array
+   */
+  private static int getNewLengthCombinedAttributes(Attribute[] attributes,
+      AggregationOperation aggOp) {
+    int oldLength = getAttributesLength(attributes);
+    int aggLength = getAppOpLength(aggOp);
+    return oldLength + aggLength;
+  }
+
+  private static int getAppOpLength(AggregationOperation aggOp) {
+    if (aggOp != null) {
+      return 1;
+    }
+    return 0;
+  }
+
+  private static int getAttributesLength(Attribute[] attributes) {
+    if (attributes != null) {
+      return attributes.length;
+    }
+    return 0;
+  }
+
+  /**
+   * checks if an application has finished
+   *
+   * @param te
+   * @return true if application has finished else false
+   */
+  public static boolean isApplicationFinished(TimelineEntity te) {
+    SortedSet<TimelineEvent> allEvents = te.getEvents();
+    if ((allEvents != null) && (allEvents.size() > 0)) {
+      TimelineEvent event = allEvents.last();
+      if (event.getId().equals(ApplicationMetricsConstants.FINISHED_EVENT_TYPE)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * get the time at which an app finished
+   *
+   * @param te
+   * @return true if application has finished else false
+   */
+  public static long getApplicationFinishedTime(TimelineEntity te) {
+    SortedSet<TimelineEvent> allEvents = te.getEvents();
+    if ((allEvents != null) && (allEvents.size() > 0)) {
+      TimelineEvent event = allEvents.last();
+      if (event.getId().equals(ApplicationMetricsConstants.FINISHED_EVENT_TYPE)) {
+        return event.getTimestamp();
+      }
+    }
+    return 0l;
+  }
+
+  /**
+   * Checks if the input TimelineEntity object is an ApplicationEntity.
+   *
+   * @param te
+   * @return true if input is an ApplicationEntity, false otherwise
+   */
+  public static boolean isApplicationEntity(TimelineEntity te) {
+    return te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString());
+  }
+
+  /**
+   * Checks for the APPLICATION_CREATED event.
+   *
+   * @param te
+   * @return true is application event exists, false otherwise
+   */
+  public static boolean isApplicationCreated(TimelineEntity te) {
+    if (isApplicationEntity(te)) {
+      for (TimelineEvent event : te.getEvents()) {
+        if (event.getId()
+            .equals(ApplicationMetricsConstants.CREATED_EVENT_TYPE)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Returns the first seen aggregation operation as seen in the list of input
+   * tags or null otherwise
+   *
+   * @param tags
+   * @return AggregationOperation
+   */
+  public static AggregationOperation getAggregationOperationFromTagsList(
+      List<Tag> tags) {
+    for (AggregationOperation aggOp : AggregationOperation.values()) {
+      for (Tag tag : tags) {
+        if (tag.getType() == aggOp.getTagType()) {
+          return aggOp;
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Creates a {@link Tag} from the input attribute.
+   *
+   * @param attribute
+   * @return Tag
+   */
+  public static Tag getTagFromAttribute(Entry<String, byte[]> attribute) {
+    // attribute could be either an Aggregation Operation or
+    // an Aggregation Dimension
+    // Get the Tag type from either
+    AggregationOperation aggOp = AggregationOperation
+        .getAggregationOperation(attribute.getKey());
+    if (aggOp != null) {
+      Tag t = new Tag(aggOp.getTagType(), attribute.getValue());
+      return t;
+    }
+
+    AggregationCompactionDimension aggCompactDim = AggregationCompactionDimension
+        .getAggregationCompactionDimension(attribute.getKey());
+    if (aggCompactDim != null) {
+      Tag t = new Tag(aggCompactDim.getTagType(), attribute.getValue());
+      return t;
+    }
+    return null;
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.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/TimestampGenerator.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/TimestampGenerator.java
new file mode 100644
index 0000000..555b64e
--- /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/common/TimestampGenerator.java
@@ -0,0 +1,112 @@
+/**
+ * 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 java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+
+/**
+ * Utility class that allows HBase coprocessors to interact with unique
+ * timestamps.
+ */
+public class TimestampGenerator {
+
+  /*
+   * if this is changed, then reading cell timestamps written with older
+   * multiplier value will not work
+   */
+  public static final long TS_MULTIPLIER = 1000L;
+
+  private final AtomicLong lastTimestamp = new AtomicLong();
+
+  /**
+   * Returns the current wall clock time in milliseconds, multiplied by the
+   * required precision.
+   */
+  public long currentTime() {
+    // We want to align cell timestamps with current time.
+    // cell timestamps are not be less than
+    // System.currentTimeMillis() * TS_MULTIPLIER.
+    return System.currentTimeMillis() * TS_MULTIPLIER;
+  }
+
+  /**
+   * Returns a timestamp value unique within the scope of this
+   * {@code TimestampGenerator} instance. For usage by HBase
+   * {@code RegionObserver} coprocessors, this normally means unique within a
+   * given region.
+   *
+   * Unlikely scenario of generating a non-unique timestamp: if there is a
+   * sustained rate of more than 1M hbase writes per second AND if region fails
+   * over within that time range of timestamps being generated then there may be
+   * collisions writing to a cell version of the same column.
+   */
+  public long getUniqueTimestamp() {
+    long lastTs;
+    long nextTs;
+    do {
+      lastTs = lastTimestamp.get();
+      nextTs = Math.max(lastTs + 1, currentTime());
+    } while (!lastTimestamp.compareAndSet(lastTs, nextTs));
+    return nextTs;
+  }
+
+  /**
+   * returns a timestamp multiplied with TS_MULTIPLIER and last few digits of
+   * application id
+   *
+   * Unlikely scenario of generating a timestamp that is a duplicate: If more
+   * than a 1000 concurrent apps are running in one flow run AND write to same
+   * column at the same time, then say appId of 1001 will overlap with appId of
+   * 001 and there may be collisions for that flow run's specific column.
+   *
+   * @param incomingTS
+   * @param appId
+   * @return a timestamp multiplied with TS_MULTIPLIER and last few digits of
+   *         application id
+   */
+  public static long getSupplementedTimestamp(long incomingTS, String appId) {
+    long suffix = getAppIdSuffix(appId);
+    long outgoingTS = incomingTS * TS_MULTIPLIER + suffix;
+    return outgoingTS;
+
+  }
+
+  private static long getAppIdSuffix(String appIdStr) {
+    if (appIdStr == null) {
+      return 0L;
+    }
+    ApplicationId appId = ConverterUtils.toApplicationId(appIdStr);
+    long id = appId.getId() % TS_MULTIPLIER;
+    return id;
+  }
+
+  /**
+   * truncates the last few digits of the timestamp which were supplemented by
+   * the TimestampGenerator#getSupplementedTimestamp function
+   *
+   * @param incomingTS
+   * @return a truncated timestamp value
+   */
+  public static long getTruncatedTimestamp(long incomingTS) {
+    return incomingTS / TS_MULTIPLIER;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/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/storage/common/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/storage/common/package-info.java
deleted file mode 100644
index 32577fb..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java
+++ /dev/null
@@ -1,24 +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.
- */
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-package org.apache.hadoop.yarn.server.timelineservice.storage.common;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java
index 26e7748..8ae19b8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage.entity;
 
 import java.io.IOException;
+import java.util.Map;
 
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -26,6 +27,7 @@ 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.flow.Attribute;
 
 /**
  * Identifies fully qualified columns for the {@link EntityTable}.
@@ -81,9 +83,9 @@ public enum EntityColumn implements Column<EntityTable> {
 
   public void store(byte[] rowKey,
       TypedBufferedMutator<EntityTable> tableMutator, Long timestamp,
-      Object inputValue) throws IOException {
+      Object inputValue, Attribute... attributes) throws IOException {
     column.store(rowKey, tableMutator, columnQualifierBytes, timestamp,
-        inputValue);
+        inputValue, attributes);
   }
 
   public Object readResult(Result result) throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
index 75ff742..0d4e5a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
@@ -28,6 +28,7 @@ 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.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
 /**
  * Identifies partially qualified columns for the entity table.
@@ -108,11 +109,13 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    * 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)
+   * 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<EntityTable> tableMutator, String qualifier,
-      Long timestamp, Object inputValue) throws IOException {
+      Long timestamp, Object inputValue, Attribute... attributes)
+      throws IOException {
 
     // Null check
     if (qualifier == null) {
@@ -123,8 +126,9 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
     byte[] columnQualifier =
         ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
 
-    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue);
-  }
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
+        attributes);
+ }
 
   /*
    * (non-Javadoc)
@@ -137,7 +141,8 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    */
   public void store(byte[] rowKey,
       TypedBufferedMutator<EntityTable> tableMutator, byte[] qualifier,
-      Long timestamp, Object inputValue) throws IOException {
+      Long timestamp, Object inputValue, Attribute... attributes)
+      throws IOException {
 
     // Null check
     if (qualifier == null) {
@@ -148,8 +153,9 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
     byte[] columnQualifier =
         ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
 
-    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue);
-  }
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
+        attributes);
+ }
 
   /*
    * (non-Javadoc)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.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/flow/AggregationCompactionDimension.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationCompactionDimension.java
new file mode 100644
index 0000000..ff12c7b
--- /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/flow/AggregationCompactionDimension.java
@@ -0,0 +1,63 @@
+/**
+ * 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.flow;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Identifies the compaction dimensions for the data in the {@link FlowRunTable}
+ * .
+ */
+public enum AggregationCompactionDimension {
+
+  /**
+   * the application id
+   */
+  APPLICATION_ID((byte) 101);
+
+  private byte tagType;
+  private byte[] inBytes;
+
+  private AggregationCompactionDimension(byte tagType) {
+    this.tagType = tagType;
+    this.inBytes = Bytes.toBytes(this.name());
+  }
+
+  public Attribute getAttribute(String attributeValue) {
+    return new Attribute(this.name(), Bytes.toBytes(attributeValue));
+  }
+
+  public byte getTagType() {
+    return tagType;
+  }
+
+  public byte[] getInBytes() {
+    return this.inBytes.clone();
+  }
+
+  public static AggregationCompactionDimension getAggregationCompactionDimension(
+      String aggCompactDimStr) {
+    for (AggregationCompactionDimension aggDim : AggregationCompactionDimension
+        .values()) {
+      if (aggDim.name().equals(aggCompactDimStr)) {
+        return aggDim;
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.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/flow/AggregationOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/AggregationOperation.java
new file mode 100644
index 0000000..c635ce6
--- /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/flow/AggregationOperation.java
@@ -0,0 +1,87 @@
+/**
+ * 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.flow;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Identifies the attributes to be set for puts into the {@link FlowRunTable}.
+ * The numbers used for tagType are prime numbers
+ */
+public enum AggregationOperation {
+
+  /**
+   * When the flow was started.
+   */
+  MIN((byte) 71),
+
+  /**
+   * When it ended.
+   */
+  MAX((byte) 73),
+
+  /**
+   * The metrics of the flow
+   */
+  SUM((byte) 79),
+
+  /**
+   * application running
+   */
+  SUM_FINAL((byte) 83),
+
+  /**
+   * compact
+   */
+  COMPACT((byte) 89);
+
+  private byte tagType;
+  private byte[] inBytes;
+
+  private AggregationOperation(byte tagType) {
+    this.tagType = tagType;
+    this.inBytes = Bytes.toBytes(this.name());
+  }
+
+  public Attribute getAttribute() {
+    return new Attribute(this.name(), this.inBytes);
+  }
+
+  public byte getTagType() {
+    return tagType;
+  }
+
+  public byte[] getInBytes() {
+    return this.inBytes.clone();
+  }
+
+  /**
+   * returns the AggregationOperation enum that represents that string
+   * @param aggOpStr
+   * @return the AggregationOperation enum that represents that string
+   */
+  public static AggregationOperation getAggregationOperation(String aggOpStr) {
+    for (AggregationOperation aggOp : AggregationOperation.values()) {
+      if (aggOp.name().equals(aggOpStr)) {
+        return aggOp;
+      }
+    }
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.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/flow/Attribute.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/Attribute.java
new file mode 100644
index 0000000..d3de518
--- /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/flow/Attribute.java
@@ -0,0 +1,39 @@
+/**
+ * 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.flow;
+
+/**
+ * Defines the attribute tuple to be set for puts into the {@link FlowRunTable}.
+ */
+public class Attribute {
+  private final String name;
+  private final byte[] value;
+
+  public Attribute(String name, byte[] value) {
+    this.name = name;
+    this.value = value.clone();
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public byte[] getValue() {
+    return value.clone();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.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/flow/FlowActivityColumnFamily.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnFamily.java
new file mode 100644
index 0000000..d991b42
--- /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/flow/FlowActivityColumnFamily.java
@@ -0,0 +1,54 @@
+/**
+ * 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.flow;
+
+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 flow run table column families.
+ */
+public enum FlowActivityColumnFamily implements ColumnFamily<FlowActivityTable> {
+
+  /**
+   * Info column family houses known columns, specifically ones included in
+   * columnfamily filters.
+   */
+  INFO("i");
+
+  /**
+   * 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.
+   */
+  private FlowActivityColumnFamily(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);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java
new file mode 100644
index 0000000..b899e5c
--- /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/flow/FlowActivityColumnPrefix.java
@@ -0,0 +1,243 @@
+/**
+ * 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.flow;
+
+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.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+
+/**
+ * Identifies partially qualified columns for the {@link FlowActivityTable}
+ */
+public enum FlowActivityColumnPrefix implements ColumnPrefix<FlowActivityTable> {
+
+  /**
+   * To store run ids of the flows
+   */
+  RUN_ID(FlowActivityColumnFamily.INFO, "r", null);
+
+  private final ColumnHelper<FlowActivityTable> column;
+  private final ColumnFamily<FlowActivityTable> 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 final AggregationOperation aggOp;
+
+  /**
+   * Private constructor, meant to be used by the enum definition.
+   *
+   * @param columnFamily
+   *          that this column is stored in.
+   * @param columnPrefix
+   *          for this column.
+   */
+  private FlowActivityColumnPrefix(
+      ColumnFamily<FlowActivityTable> columnFamily, String columnPrefix,
+      AggregationOperation aggOp) {
+    column = new ColumnHelper<FlowActivityTable>(columnFamily);
+    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.aggOp = aggOp;
+  }
+
+  /**
+   * @return the column name value
+   */
+  public String getColumnPrefix() {
+    return columnPrefix;
+  }
+
+  public byte[] getColumnPrefixBytes() {
+    return columnPrefixBytes.clone();
+  }
+
+  public AggregationOperation getAttribute() {
+    return aggOp;
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
+   * #store(byte[],
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.
+   * TypedBufferedMutator, byte[], java.lang.Long, java.lang.Object,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[])
+   */
+  @Override
+  public void store(byte[] rowKey,
+      TypedBufferedMutator<FlowActivityTable> 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 = ColumnHelper.getColumnQualifier(
+        this.columnPrefixBytes, qualifier);
+    Attribute[] combinedAttributes = TimelineWriterUtils.combineAttributes(
+        attributes, this.aggOp);
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
+        combinedAttributes);
+  }
+
+  /*
+   * (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)
+   */
+  public Map<String, Object> readResults(Result result) throws IOException {
+    return column.readResults(result, columnPrefixBytes);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
+   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result)
+   */
+  public <T> NavigableMap<String, NavigableMap<Long, T>> readResultsWithTimestamps(
+      Result result) throws IOException {
+    return column.readResultsWithTimestamps(result, columnPrefixBytes);
+  }
+
+  /**
+   * 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)
+   *
+   * @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[])
+   */
+  @Override
+  public void store(byte[] rowKey,
+      TypedBufferedMutator<FlowActivityTable> 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 = ColumnHelper.getColumnQualifier(
+        this.columnPrefixBytes, qualifier);
+    Attribute[] combinedAttributes = TimelineWriterUtils.combineAttributes(
+        attributes, this.aggOp);
+    column.store(rowKey, tableMutator, columnQualifier, null, inputValue,
+        combinedAttributes);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
new file mode 100644
index 0000000..19e4e83
--- /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/flow/FlowActivityRowKey.java
@@ -0,0 +1,113 @@
+/**
+ * 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.flow;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+
+/**
+ * Represents a rowkey for the flow activity table.
+ */
+public class FlowActivityRowKey {
+
+  private final String clusterId;
+  private final long dayTs;
+  private final String userId;
+  private final String flowId;
+
+  public FlowActivityRowKey(String clusterId, long dayTs, String userId,
+      String flowId) {
+    this.clusterId = clusterId;
+    this.dayTs = dayTs;
+    this.userId = userId;
+    this.flowId = flowId;
+  }
+
+  public String getClusterId() {
+    return clusterId;
+  }
+
+  public long getDayTimestamp() {
+    return dayTs;
+  }
+
+  public String getUserId() {
+    return userId;
+  }
+
+  public String getFlowId() {
+    return flowId;
+  }
+
+  /**
+   * Constructs a row key for the flow activity table as follows:
+   * {@code clusterId!dayTimestamp!user!flowId}
+   *
+   * Will insert into current day's record in the table
+   * @param clusterId
+   * @param userId
+   * @param flowId
+   * @return byte array with the row key prefix
+   */
+  public static byte[] getRowKey(String clusterId, String userId, String flowId) {
+    long dayTs = TimelineWriterUtils.getTopOfTheDayTimestamp(System
+        .currentTimeMillis());
+    return getRowKey(clusterId, dayTs, userId, flowId);
+  }
+
+  /**
+   * Constructs a row key for the flow activity table as follows:
+   * {@code clusterId!dayTimestamp!user!flowId}
+   *
+   * @param clusterId
+   * @param dayTs
+   * @param userId
+   * @param flowId
+   * @return byte array for the row key
+   */
+  public static byte[] getRowKey(String clusterId, long dayTs, String userId,
+      String flowId) {
+    return Separator.QUALIFIERS.join(
+        Bytes.toBytes(Separator.QUALIFIERS.encode(clusterId)),
+        Bytes.toBytes(TimelineWriterUtils.invert(dayTs)),
+        Bytes.toBytes(Separator.QUALIFIERS.encode(userId)),
+        Bytes.toBytes(Separator.QUALIFIERS.encode(flowId)));
+  }
+
+  /**
+   * Given the raw row key as bytes, returns the row key as an object.
+   */
+  public static FlowActivityRowKey parseRowKey(byte[] rowKey) {
+    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
+
+    if (rowKeyComponents.length < 4) {
+      throw new IllegalArgumentException("the row key is not valid for "
+          + "a flow activity");
+    }
+
+    String clusterId = Separator.QUALIFIERS.decode(Bytes
+        .toString(rowKeyComponents[0]));
+    long dayTs = TimelineWriterUtils.invert(Bytes.toLong(rowKeyComponents[1]));
+    String userId = Separator.QUALIFIERS.decode(Bytes
+        .toString(rowKeyComponents[2]));
+    String flowId = Separator.QUALIFIERS.decode(Bytes
+        .toString(rowKeyComponents[3]));
+    return new FlowActivityRowKey(clusterId, dayTs, userId, flowId);
+  }
+}


[23/50] [abbrv] hadoop git commit: YARN-4203. Add request/response logging & timing for each REST endpoint call (Varun Saxena via vrushali)

Posted by gt...@apache.org.
YARN-4203. Add request/response logging & timing for each REST endpoint call (Varun Saxena via vrushali)


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

Branch: refs/heads/feature-YARN-2928
Commit: 008e27367c2c5f1d03fa4e11b7786d61fb49972c
Parents: 4552598
Author: Vrushali Channapattan <vr...@apache.org>
Authored: Wed Sep 30 12:56:07 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:58 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../reader/TimelineReaderWebServices.java       | 78 +++++++++++++++-----
 2 files changed, 62 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/008e2736/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 4846c5c..f9bb187 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -112,6 +112,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4075. [reader REST API] implement support for querying for flows
     and flow runs (Varun Saxena via vrushali)
 
+    YARN-4203. Add request/response logging & timing for each REST endpoint
+    call (Varun Saxena via vrushali)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/008e2736/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 f619c7d..a327099 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
@@ -44,6 +44,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
@@ -68,6 +69,7 @@ public class TimelineReaderWebServices {
 
   private static final String COMMA_DELIMITER = ",";
   private static final String COLON_DELIMITER = ":";
+  private static final String QUERY_STRING_SEP = "?";
 
   private void init(HttpServletResponse response) {
     response.setContentType(null);
@@ -185,14 +187,24 @@ public class TimelineReaderWebServices {
     return callerUGI;
   }
 
+  private static String getUserName(UserGroupInformation callerUGI) {
+    return ((callerUGI != null) ? callerUGI.getUserName().trim() : "");
+  }
+
   private TimelineReaderManager getTimelineReaderManager() {
     return (TimelineReaderManager)
         ctxt.getAttribute(TimelineReaderServer.TIMELINE_READER_MANAGER_ATTR);
   }
 
-  private static void handleException(Exception e) throws BadRequestException,
+  private static void handleException(Exception e, String url, long startTime,
+      String invalidNumMsg) throws BadRequestException,
       WebApplicationException {
-    if (e instanceof IllegalArgumentException) {
+    long endTime = Time.monotonicNow();
+    LOG.info("Processed URL " + url + " but encountered exception (Took " +
+        (endTime - startTime) + " ms.)");
+    if (e instanceof NumberFormatException) {
+      throw new BadRequestException(invalidNumMsg + " is not a numeric value.");
+    } else if (e instanceof IllegalArgumentException) {
       throw new BadRequestException("Requested Invalid Field.");
     } else {
       LOG.error("Error while processing REST request", e);
@@ -273,9 +285,14 @@ public class TimelineReaderWebServices {
       @QueryParam("metricfilters") String metricfilters,
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("fields") String fields) {
+    String url = req.getRequestURI() +
+        (null == req.getQueryString() ? "" :
+            QUERY_STRING_SEP + req.getQueryString());
+    UserGroupInformation callerUGI = getUser(req);
+    LOG.info("Received URL " + url + " from user " + getUserName(callerUGI));
+    long startTime = Time.monotonicNow();
     init(res);
     TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
-    UserGroupInformation callerUGI = getUser(req);
     Set<TimelineEntity> entities = null;
     try {
       entities = timelineReaderManager.getEntities(
@@ -291,16 +308,16 @@ public class TimelineReaderWebServices {
           parseValuesStr(metricfilters, COMMA_DELIMITER),
           parseValuesStr(eventfilters, COMMA_DELIMITER),
           parseFieldsStr(fields, COMMA_DELIMITER));
-    } catch (NumberFormatException e) {
-      throw new BadRequestException(
-          "createdTime or modifiedTime start/end or limit or flowId is not" +
-          " a numeric value.");
     } catch (Exception e) {
-      handleException(e);
+      handleException(e, url, startTime,
+          "createdTime or modifiedTime start/end or limit or flowId");
     }
+    long endTime = Time.monotonicNow();
     if (entities == null) {
       entities = Collections.emptySet();
     }
+    LOG.info("Processed URL " + url +
+        " (Took " + (endTime - startTime) + " ms.)");
     return entities;
   }
 
@@ -342,24 +359,32 @@ public class TimelineReaderWebServices {
       @QueryParam("flowid") String flowId,
       @QueryParam("flowrunid") String flowRunId,
       @QueryParam("fields") String fields) {
+    String url = req.getRequestURI() +
+        (null == req.getQueryString() ? "" :
+            QUERY_STRING_SEP + req.getQueryString());
+    UserGroupInformation callerUGI = getUser(req);
+    LOG.info("Received URL " + url + " from user " + getUserName(callerUGI));
+    long startTime = Time.monotonicNow();
     init(res);
     TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
-    UserGroupInformation callerUGI = getUser(req);
     TimelineEntity entity = null;
     try {
       entity = timelineReaderManager.getEntity(
           parseUser(callerUGI, userId), parseStr(clusterId), parseStr(flowId),
           parseLongStr(flowRunId), parseStr(appId), parseStr(entityType),
           parseStr(entityId), parseFieldsStr(fields, COMMA_DELIMITER));
-    } catch (NumberFormatException e) {
-      throw new BadRequestException("flowrunid is not a numeric value.");
     } catch (Exception e) {
-      handleException(e);
+      handleException(e, url, startTime, "flowrunid");
     }
+    long endTime = Time.monotonicNow();
     if (entity == null) {
+      LOG.info("Processed URL " + url + " but entity not found" + " (Took " +
+          (endTime - startTime) + " ms.)");
       throw new NotFoundException("Timeline entity {id: " + parseStr(entityId) +
           ", type: " + parseStr(entityType) + " } is not found");
     }
+    LOG.info("Processed URL " + url +
+        " (Took " + (endTime - startTime) + " ms.)");
     return entity;
   }
 
@@ -394,9 +419,14 @@ public class TimelineReaderWebServices {
       @PathParam("flowrunid") String flowRunId,
       @QueryParam("userid") String userId,
       @QueryParam("fields") String fields) {
+    String url = req.getRequestURI() +
+        (null == req.getQueryString() ? "" :
+            QUERY_STRING_SEP + req.getQueryString());
+    UserGroupInformation callerUGI = getUser(req);
+    LOG.info("Received URL " + url + " from user " + getUserName(callerUGI));
+    long startTime = Time.monotonicNow();
     init(res);
     TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
-    UserGroupInformation callerUGI = getUser(req);
     TimelineEntity entity = null;
     try {
       entity = timelineReaderManager.getEntity(
@@ -404,15 +434,18 @@ public class TimelineReaderWebServices {
           parseStr(flowId), parseLongStr(flowRunId), null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null,
           parseFieldsStr(fields, COMMA_DELIMITER));
-    } catch (NumberFormatException e) {
-      throw new BadRequestException("flowRunId is not a numeric value.");
     } catch (Exception e) {
-      handleException(e);
+      handleException(e, url, startTime, "flowrunid");
     }
+    long endTime = Time.monotonicNow();
     if (entity == null) {
+      LOG.info("Processed URL " + url + " but flowrun not found (Took " +
+          (endTime - startTime) + " ms.)");
       throw new NotFoundException("Flow run {flow id: " + parseStr(flowId) +
           ", run id: " + parseLongStr(flowRunId) + " } is not found");
     }
+    LOG.info("Processed URL " + url +
+        " (Took " + (endTime - startTime) + " ms.)");
     return entity;
   }
 
@@ -443,6 +476,12 @@ public class TimelineReaderWebServices {
       @PathParam("clusterid") String clusterId,
       @QueryParam("limit") String limit,
       @QueryParam("fields") String fields) {
+    String url = req.getRequestURI() +
+        (null == req.getQueryString() ? "" :
+            QUERY_STRING_SEP + req.getQueryString());
+    UserGroupInformation callerUGI = getUser(req);
+    LOG.info("Received URL " + url + " from user " + getUserName(callerUGI));
+    long startTime = Time.monotonicNow();
     init(res);
     TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
     Set<TimelineEntity> entities = null;
@@ -452,14 +491,15 @@ public class TimelineReaderWebServices {
           TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), parseLongStr(limit),
           null, null, null, null, null, null, null, null, null, null,
           parseFieldsStr(fields, COMMA_DELIMITER));
-    } catch (NumberFormatException e) {
-      throw new BadRequestException("limit is not a numeric value.");
     } catch (Exception e) {
-      handleException(e);
+      handleException(e, url, startTime, "limit");
     }
+    long endTime = Time.monotonicNow();
     if (entities == null) {
       entities = Collections.emptySet();
     }
+    LOG.info("Processed URL " + url +
+        " (Took " + (endTime - startTime) + " ms.)");
     return entities;
   }
 }
\ No newline at end of file


[47/50] [abbrv] hadoop git commit: YARN-3586. RM to only get back addresses of Collectors that NM needs to know. (Junping Du via Varun Saxena).

Posted by gt...@apache.org.
YARN-3586. RM to only get back addresses of Collectors that NM needs to know.
(Junping Du via Varun Saxena).


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

Branch: refs/heads/feature-YARN-2928
Commit: 0352b97430604fce39d06c578bd1d471b02556af
Parents: 41b9f27
Author: Varun Saxena <va...@apache.org>
Authored: Tue Dec 22 20:58:54 2015 +0530
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 18:03:31 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../resourcemanager/ResourceTrackerService.java | 30 +++----
 .../TestResourceTrackerService.java             | 82 ++++++++++++++++++++
 3 files changed, 100 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0352b974/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index fbd40ba..5bac262 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -155,6 +155,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4445. Unify the term flowId and flowName in timeline v2 codebase. 
     (Zhan Zhang via gtcarrera9). 
 
+    YARN-3586. RM to only get back addresses of Collectors that NM needs to know.
+    (Junping Du via varunsaxena)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0352b974/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 b386f0a..1d9433c 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
@@ -25,7 +25,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.commons.logging.Log;
@@ -477,16 +476,15 @@ public class ResourceTrackerService extends AbstractService implements
       nodeHeartBeatResponse.setSystemCredentialsForApps(systemCredentials);
     }
 
-    List<ApplicationId> keepAliveApps =
-        remoteNodeStatus.getKeepAliveApplications();
-    if (timelineV2Enabled && keepAliveApps != null) {
+    if (timelineV2Enabled) {
       // Return collectors' map that NM needs to know
-      // TODO we should optimize this to only include collector info that NM
-      // doesn't know yet.
-      setAppCollectorsMapToResponse(keepAliveApps, nodeHeartBeatResponse);
+      setAppCollectorsMapToResponse(rmNode.getRunningApps(),
+          nodeHeartBeatResponse);
     }
 
     // 4. Send status to RMNode, saving the latest response.
+    List<ApplicationId> keepAliveApps =
+        remoteNodeStatus.getKeepAliveApplications();
     RMNodeStatusEvent nodeStatusEvent =
         new RMNodeStatusEvent(nodeId, remoteNodeStatus, nodeHeartBeatResponse);
     if (request.getLogAggregationReportsForApps() != null
@@ -514,18 +512,20 @@ public class ResourceTrackerService extends AbstractService implements
   }
 
   private void setAppCollectorsMapToResponse(
-      List<ApplicationId> liveApps, NodeHeartbeatResponse response) {
+      List<ApplicationId> runningApps, NodeHeartbeatResponse response) {
     Map<ApplicationId, String> liveAppCollectorsMap = new
-        ConcurrentHashMap<ApplicationId, String>();
+        HashMap<ApplicationId, String>();
     Map<ApplicationId, RMApp> rmApps = rmContext.getRMApps();
-    // Set collectors for all apps now.
-    // TODO set collectors for only active apps running on NM (liveApps cannot be
-    // used for this case)
-    for (Map.Entry<ApplicationId, RMApp> rmApp : rmApps.entrySet()) {
-      ApplicationId appId = rmApp.getKey();
-      String appCollectorAddr = rmApp.getValue().getCollectorAddr();
+    // Set collectors for all running apps on this node.
+    for (ApplicationId appId : runningApps) {
+      String appCollectorAddr = rmApps.get(appId).getCollectorAddr();
       if (appCollectorAddr != null) {
         liveAppCollectorsMap.put(appId, appCollectorAddr);
+      } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Collector for applicaton: " + appId +
+              " hasn't registered yet!");
+        }
       }
     }
     response.setAppCollectorsMap(liveAppCollectorsMap);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0352b974/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.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/TestResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
index e42ed91..b0e6c55 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
@@ -30,6 +30,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
@@ -60,6 +61,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequ
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UnRegisterNodeManagerRequest;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
+import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NodeLabelsUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager;
@@ -67,8 +69,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsMana
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
+import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.YarnVersionInfo;
@@ -869,6 +874,83 @@ public class TestResourceTrackerService extends NodeLabelTestBase {
     checkRebootedNMCount(rm, ++initialMetricCount);
   }
 
+  @Test
+  public void testNodeHeartbeatForAppCollectorsMap() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    // set version to 2
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+    // enable aux-service based timeline collectors
+    conf.set(YarnConfiguration.NM_AUX_SERVICES, "timeline_collector");
+    conf.set(YarnConfiguration.NM_AUX_SERVICES + "."
+        + "timeline_collector" + ".class",
+        PerNodeTimelineCollectorsAuxService.class.getName());
+
+    rm = new MockRM(conf);
+    rm.start();
+
+    MockNM nm1 = rm.registerNode("host1:1234", 5120);
+    MockNM nm2 = rm.registerNode("host2:1234", 2048);
+
+    NodeHeartbeatResponse nodeHeartbeat1 = nm1.nodeHeartbeat(true);
+    NodeHeartbeatResponse nodeHeartbeat2 = nm2.nodeHeartbeat(true);
+
+    RMNodeImpl node1 =
+        (RMNodeImpl) rm.getRMContext().getRMNodes().get(nm1.getNodeId());
+
+    RMNodeImpl node2 =
+        (RMNodeImpl) rm.getRMContext().getRMNodes().get(nm2.getNodeId());
+
+    RMApp app1 = rm.submitApp(1024);
+    String collectorAddr1 = "1.2.3.4:5";
+    app1.setCollectorAddr(collectorAddr1);
+
+    String collectorAddr2 = "5.4.3.2:1";
+    RMApp app2 = rm.submitApp(1024);
+    app2.setCollectorAddr(collectorAddr2);
+
+    // Create a running container for app1 running on nm1
+    ContainerId runningContainerId1 = BuilderUtils.newContainerId(
+        BuilderUtils.newApplicationAttemptId(
+        app1.getApplicationId(), 0), 0);
+
+    ContainerStatus status1 = ContainerStatus.newInstance(runningContainerId1,
+        ContainerState.RUNNING, "", 0);
+    List<ContainerStatus> statusList = new ArrayList<ContainerStatus>();
+    statusList.add(status1);
+    NodeHealthStatus nodeHealth = NodeHealthStatus.newInstance(true,
+        "", System.currentTimeMillis());
+    node1.handle(new RMNodeStatusEvent(nm1.getNodeId(), nodeHealth,
+        statusList, null, nodeHeartbeat1));
+
+    Assert.assertEquals(1, node1.getRunningApps().size());
+    Assert.assertEquals(app1.getApplicationId(), node1.getRunningApps().get(0));
+
+    // Create a running container for app2 running on nm2
+    ContainerId runningContainerId2 = BuilderUtils.newContainerId(
+        BuilderUtils.newApplicationAttemptId(
+        app2.getApplicationId(), 0), 0);
+
+    ContainerStatus status2 = ContainerStatus.newInstance(runningContainerId2,
+        ContainerState.RUNNING, "", 0);
+    statusList = new ArrayList<ContainerStatus>();
+    statusList.add(status2);
+    node2.handle(new RMNodeStatusEvent(nm2.getNodeId(), nodeHealth,
+        statusList, null, nodeHeartbeat2));
+    Assert.assertEquals(1, node2.getRunningApps().size());
+    Assert.assertEquals(app2.getApplicationId(), node2.getRunningApps().get(0));
+
+    nodeHeartbeat1 = nm1.nodeHeartbeat(true);
+    Map<ApplicationId, String> map1 = nodeHeartbeat1.getAppCollectorsMap();
+    Assert.assertEquals(1, map1.size());
+    Assert.assertEquals(collectorAddr1, map1.get(app1.getApplicationId()));
+
+    nodeHeartbeat2 = nm2.nodeHeartbeat(true);
+    Map<ApplicationId, String> map2 = nodeHeartbeat2.getAppCollectorsMap();
+    Assert.assertEquals(1, map2.size());
+    Assert.assertEquals(collectorAddr2, map2.get(app2.getApplicationId()));
+  }
+
   private void checkRebootedNMCount(MockRM rm2, int count)
       throws InterruptedException {
     


[02/50] [abbrv] hadoop git commit: YARN-3949. Ensure timely flush of timeline writes. Contributed by Sangjin Lee.

Posted by gt...@apache.org.
YARN-3949. Ensure timely flush of timeline writes. Contributed by Sangjin Lee.

(cherry picked from commit 967bef7e0396d857913caa2574afb103a5f0b81b)


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

Branch: refs/heads/feature-YARN-2928
Commit: 7f5b3b94b62409089c0e5539db99a3d0f500de57
Parents: fce5409
Author: Junping Du <ju...@apache.org>
Authored: Sat Jul 25 10:30:29 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:36:53 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  9 +++
 .../src/main/resources/yarn-default.xml         | 17 ++++-
 .../collector/TimelineCollectorManager.java     | 65 ++++++++++++++++++--
 .../storage/FileSystemTimelineWriterImpl.java   |  5 ++
 .../storage/HBaseTimelineWriterImpl.java        |  6 ++
 .../storage/PhoenixTimelineWriterImpl.java      |  5 ++
 .../timelineservice/storage/TimelineWriter.java |  9 +++
 .../TestNMTimelineCollectorManager.java         |  5 ++
 9 files changed, 119 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f5b3b94/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e1f1898..fc381d6 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -79,6 +79,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3047. [Data Serving] Set up ATS reader with basic request serving
     structure and lifecycle (Varun Saxena via sjlee)
 
+    YARN-3949. Ensure timely flush of timeline writes. (Sangjin Lee via
+    junping_du)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f5b3b94/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 43f7367..b8281b0 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
@@ -1742,6 +1742,15 @@ public class YarnConfiguration extends Configuration {
   public static final String TIMELINE_SERVICE_READER_CLASS =
       TIMELINE_SERVICE_PREFIX + "reader.class";
 
+  /** The setting that controls how often the timeline collector flushes the
+   * timeline writer.
+   */
+  public static final String TIMELINE_SERVICE_WRITER_FLUSH_INTERVAL_SECONDS =
+      TIMELINE_SERVICE_PREFIX + "writer.flush-interval-seconds";
+
+  public static final int
+      DEFAULT_TIMELINE_SERVICE_WRITER_FLUSH_INTERVAL_SECONDS = 60;
+
   // 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/7f5b3b94/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 02aa56d..eb66475 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
@@ -782,7 +782,15 @@
     <name>yarn.system-metrics-publisher.enabled</name>
     <value>false</value>
   </property>
- 
+
+  <property>
+    <description>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.</description>
+    <name>yarn.rm.system-metrics-publisher.emit-container-events</name>
+    <value>false</value>
+  </property>
+
 
   <property>
     <description>Number of worker threads that send the yarn system metrics
@@ -2058,6 +2066,13 @@
     <value>604800</value>
   </property>
 
+  <property>
+    <description>The setting that controls how often the timeline collector
+    flushes the timeline writer.</description>
+    <name>yarn.timeline-service.writer.flush-interval-seconds</name>
+    <value>60</value>
+  </property>
+
   <!--  Shared Cache Configuration -->
 
   <property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f5b3b94/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 23ad4f4..e9f2085 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
@@ -19,6 +19,13 @@
 package org.apache.hadoop.yarn.server.timelineservice.collector;
 
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -32,9 +39,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
+import com.google.common.annotations.VisibleForTesting;
 
 /**
  * Class that manages adding and removing collectors and their lifecycle. It
@@ -47,7 +52,10 @@ public abstract class TimelineCollectorManager extends AbstractService {
   private static final Log LOG =
       LogFactory.getLog(TimelineCollectorManager.class);
 
-  protected TimelineWriter writer;
+  private TimelineWriter writer;
+  private ScheduledExecutorService writerFlusher;
+  private int flushInterval;
+  private boolean writerFlusherRunning;
 
   @Override
   public void serviceInit(Configuration conf) throws Exception {
@@ -56,6 +64,12 @@ public abstract class TimelineCollectorManager extends AbstractService {
         FileSystemTimelineWriterImpl.class,
         TimelineWriter.class), conf);
     writer.init(conf);
+    // create a single dedicated thread for flushing the writer on a periodic
+    // basis
+    writerFlusher = Executors.newSingleThreadScheduledExecutor();
+    flushInterval = conf.getInt(
+        YarnConfiguration.TIMELINE_SERVICE_WRITER_FLUSH_INTERVAL_SECONDS,
+        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WRITER_FLUSH_INTERVAL_SECONDS);
     super.serviceInit(conf);
   }
 
@@ -65,6 +79,10 @@ public abstract class TimelineCollectorManager extends AbstractService {
     if (writer != null) {
       writer.start();
     }
+    // schedule the flush task
+    writerFlusher.scheduleAtFixedRate(new WriterFlushTask(writer),
+        flushInterval, flushInterval, TimeUnit.SECONDS);
+    writerFlusherRunning = true;
   }
 
   // access to this map is synchronized with the map itself
@@ -161,9 +179,48 @@ public abstract class TimelineCollectorManager extends AbstractService {
         c.serviceStop();
       }
     }
+    // stop the flusher first
+    if (writerFlusher != null) {
+      writerFlusher.shutdown();
+      writerFlusherRunning = false;
+      if (!writerFlusher.awaitTermination(30, TimeUnit.SECONDS)) {
+        // in reality it should be ample time for the flusher task to finish
+        // even if it times out, writers may be able to handle closing in this
+        // situation fine
+        // proceed to close the writer
+        LOG.warn("failed to stop the flusher task in time. " +
+            "will still proceed to close the writer.");
+      }
+    }
     if (writer != null) {
       writer.close();
     }
     super.serviceStop();
   }
+
+  @VisibleForTesting
+  boolean writerFlusherRunning() {
+    return writerFlusherRunning;
+  }
+
+  /**
+   * Task that invokes the flush operation on the timeline writer.
+   */
+  private static class WriterFlushTask implements Runnable {
+    private final TimelineWriter writer;
+
+    public WriterFlushTask(TimelineWriter writer) {
+      this.writer = writer;
+    }
+
+    public void run() {
+      try {
+        writer.flush();
+      } catch (Throwable th) {
+        // we need to handle all exceptions or subsequent execution may be
+        // suppressed
+        LOG.error("exception during timeline writer flush!", th);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f5b3b94/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 34a6b7c..b22b39f 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
@@ -127,6 +127,11 @@ public class FileSystemTimelineWriterImpl extends AbstractService
     mkdirs(outputRoot, ENTITIES_DIR);
   }
 
+  @Override
+  public void flush() throws IOException {
+    // no op
+  }
+
   private static String mkdirs(String... dirStrs) throws IOException {
     StringBuilder path = new StringBuilder();
     for (String dirStr : dirStrs) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f5b3b94/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 e48ca60..876ad6a 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
@@ -214,6 +214,12 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     return null;
   }
 
+  @Override
+  public void flush() throws IOException {
+    // flush all buffered mutators
+    entityTable.flush();
+  }
+
   /**
    * close the hbase connections The close APIs perform flushing and release any
    * resources held

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f5b3b94/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixTimelineWriterImpl.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/PhoenixTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixTimelineWriterImpl.java
index 5b4442c..381ff17 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixTimelineWriterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixTimelineWriterImpl.java
@@ -187,6 +187,11 @@ public class PhoenixTimelineWriterImpl extends AbstractService
 
   }
 
+  @Override
+  public void flush() throws IOException {
+    // currently no-op
+  }
+
   // Utility functions
   @Private
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f5b3b94/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 494e8ad..50136de 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
@@ -70,4 +70,13 @@ public interface TimelineWriter extends Service {
    */
   TimelineWriteResponse aggregate(TimelineEntity data,
       TimelineAggregationTrack track) throws IOException;
+
+  /**
+   * Flushes the data to the backend storage. Whatever may be buffered will be
+   * written to the storage when the method returns. This may be a potentially
+   * time-consuming operation, and should be used judiciously.
+   *
+   * @throws IOException
+   */
+  void flush() throws IOException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f5b3b94/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 87343fd..0d69fbc 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
@@ -67,6 +67,11 @@ public class TestNMTimelineCollectorManager {
   }
 
   @Test
+  public void testStartingWriterFlusher() throws Exception {
+    assertTrue(collectorManager.writerFlusherRunning());
+  }
+
+  @Test
   public void testStartWebApp() throws Exception {
     assertNotNull(collectorManager.getRestServerBindAddress());
     String address = collectorManager.getRestServerBindAddress();


[29/50] [abbrv] hadoop git commit: YARN-4129. Refactor the SystemMetricPublisher in RM to better support newer events (Naganarasimha G R via sjlee)

Posted by gt...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/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 4b93165..d858a6b 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
@@ -30,17 +30,23 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
+import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
+import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 
-public class TimelineServiceV1Publisher extends
-    AbstractTimelineServicePublisher {
+public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
 
-  private static final Log LOG = LogFactory
-      .getLog(TimelineServiceV1Publisher.class);
+  private static final Log LOG =
+      LogFactory.getLog(TimelineServiceV1Publisher.class);
 
   public TimelineServiceV1Publisher() {
     super("TimelineserviceV1Publisher");
@@ -49,76 +55,69 @@ public class TimelineServiceV1Publisher extends
   private TimelineClient client;
 
   @Override
-  public void serviceInit(Configuration conf) throws Exception {
+  protected void serviceInit(Configuration conf) throws Exception {
     client = TimelineClient.createTimelineClient();
     addIfService(client);
     super.serviceInit(conf);
+    getDispatcher().register(SystemMetricsEventType.class,
+        new TimelineV1EventHandler());
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishApplicationCreatedEvent(ApplicationCreatedEvent event) {
-    TimelineEntity entity =
-        createApplicationEntity(event.getApplicationId());
+  public void appCreated(RMApp app, long createdTime) {
+    TimelineEntity entity = createApplicationEntity(app.getApplicationId());
     Map<String, Object> entityInfo = new HashMap<String, Object>();
-    entityInfo.put(ApplicationMetricsConstants.NAME_ENTITY_INFO,
-        event.getApplicationName());
+    entityInfo.put(ApplicationMetricsConstants.NAME_ENTITY_INFO, app.getName());
     entityInfo.put(ApplicationMetricsConstants.TYPE_ENTITY_INFO,
-        event.getApplicationType());
-    entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO,
-        event.getUser());
+        app.getApplicationType());
+    entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO, app.getUser());
     entityInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO,
-        event.getQueue());
+        app.getQueue());
     entityInfo.put(ApplicationMetricsConstants.SUBMITTED_TIME_ENTITY_INFO,
-        event.getSubmittedTime());
+        app.getSubmitTime());
     entityInfo.put(ApplicationMetricsConstants.APP_TAGS_INFO,
-        event.getAppTags());
+        app.getApplicationTags());
     entityInfo.put(
         ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO,
-        event.isUnmanagedApp());
+        app.getApplicationSubmissionContext().getUnmanagedAM());
     entityInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO,
-        event.getApplicationPriority().getPriority());
-    entityInfo.put(ApplicationMetricsConstants.APP_NODE_LABEL_EXPRESSION,
-        event.getAppNodeLabelsExpression());
+        app.getApplicationSubmissionContext().getPriority().getPriority());
     entityInfo.put(ApplicationMetricsConstants.AM_NODE_LABEL_EXPRESSION,
-        event.getAmNodeLabelsExpression());
-    if (event.getCallerContext() != null) {
-      if (event.getCallerContext().getContext() != null) {
-        entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_CONTEXT,
-            event.getCallerContext().getContext());
-      }
-      if (event.getCallerContext().getSignature() != null) {
-        entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_SIGNATURE,
-            event.getCallerContext().getSignature());
-      }
-    }
+        app.getAmNodeLabelExpression());
+    entityInfo.put(ApplicationMetricsConstants.APP_NODE_LABEL_EXPRESSION,
+        app.getAppNodeLabelExpression());
     entity.setOtherInfo(entityInfo);
     TimelineEvent tEvent = new TimelineEvent();
-    tEvent.setEventType(
-        ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setEventType(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    tEvent.setTimestamp(createdTime);
+
     entity.addEvent(tEvent);
-    putEntity(entity);
+    getDispatcher().getEventHandler().handle(new TimelineV1PublishEvent(
+        SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
   }
 
   @Override
-  void publishApplicationFinishedEvent(ApplicationFinishedEvent event) {
-    TimelineEntity entity = createApplicationEntity(event.getApplicationId());
+  public void appFinished(RMApp app, RMAppState state, long finishedTime) {
+    TimelineEntity entity = createApplicationEntity(app.getApplicationId());
 
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setEventType(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setTimestamp(finishedTime);
     Map<String, Object> eventInfo = new HashMap<String, Object>();
     eventInfo.put(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
-        event.getDiagnosticsInfo());
-    eventInfo.put(ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO, event
-        .getFinalApplicationStatus().toString());
-    eventInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO, event
-        .getYarnApplicationState().toString());
-    if (event.getLatestApplicationAttemptId() != null) {
+        app.getDiagnostics().toString());
+    eventInfo.put(ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO,
+        app.getFinalApplicationStatus().toString());
+    eventInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO,
+        RMServerUtils.createApplicationState(state).toString());
+    String latestApplicationAttemptId = app.getCurrentAppAttempt() == null
+        ? null : app.getCurrentAppAttempt().getAppAttemptId().toString();
+    if (latestApplicationAttemptId != null) {
       eventInfo.put(ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO,
-          event.getLatestApplicationAttemptId().toString());
+          latestApplicationAttemptId);
     }
-    RMAppMetrics appMetrics = event.getAppMetrics();
+    RMAppMetrics appMetrics = app.getRMAppMetrics();
     entity.addOtherInfo(ApplicationMetricsConstants.APP_CPU_METRICS,
         appMetrics.getVcoreSeconds());
     entity.addOtherInfo(ApplicationMetricsConstants.APP_MEM_METRICS,
@@ -126,161 +125,174 @@ public class TimelineServiceV1Publisher extends
     tEvent.setEventInfo(eventInfo);
 
     entity.addEvent(tEvent);
-    putEntity(entity);
+    // sync sending of finish event to avoid possibility of saving application
+    // finished state in RMStateStore save without publishing in ATS
+    putEntity(entity);// sync event so that ATS update is done without fail
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishApplicationUpdatedEvent(ApplicationUpdatedEvent event) {
-    TimelineEntity entity = createApplicationEntity(event.getApplicationId());
+  public void appUpdated(RMApp app, long updatedTime) {
+    TimelineEntity entity = createApplicationEntity(app.getApplicationId());
     Map<String, Object> eventInfo = new HashMap<String, Object>();
     eventInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO,
-        event.getQueue());
-    eventInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO, event
-        .getApplicationPriority().getPriority());
+        app.getQueue());
+    eventInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO,
+        app.getApplicationSubmissionContext().getPriority().getPriority());
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setEventType(ApplicationMetricsConstants.UPDATED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setTimestamp(updatedTime);
     tEvent.setEventInfo(eventInfo);
     entity.addEvent(tEvent);
-    putEntity(entity);
+    getDispatcher().getEventHandler().handle(new TimelineV1PublishEvent(
+        SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishApplicationACLsUpdatedEvent(ApplicationACLsUpdatedEvent event) {
-    TimelineEntity entity = createApplicationEntity(event.getApplicationId());
+  public void appACLsUpdated(RMApp app, String appViewACLs, long updatedTime) {
+    TimelineEntity entity = createApplicationEntity(app.getApplicationId());
 
     TimelineEvent tEvent = new TimelineEvent();
     Map<String, Object> entityInfo = new HashMap<String, Object>();
     entityInfo.put(ApplicationMetricsConstants.APP_VIEW_ACLS_ENTITY_INFO,
-        event.getViewAppACLs());
+        (appViewACLs == null) ? "" : appViewACLs);
     entity.setOtherInfo(entityInfo);
     tEvent.setEventType(ApplicationMetricsConstants.ACLS_UPDATED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setTimestamp(updatedTime);
 
     entity.addEvent(tEvent);
-    putEntity(entity);
-  }
-
-  private static TimelineEntity createApplicationEntity(
-      ApplicationId applicationId) {
-    TimelineEntity entity = new TimelineEntity();
-    entity.setEntityType(ApplicationMetricsConstants.ENTITY_TYPE);
-    entity.setEntityId(applicationId.toString());
-    return entity;
+    getDispatcher().getEventHandler().handle(new TimelineV1PublishEvent(
+        SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishAppAttemptRegisteredEvent(AppAttemptRegisteredEvent event) {
+  public void appAttemptRegistered(RMAppAttempt appAttempt,
+      long registeredTime) {
     TimelineEntity entity =
-        createAppAttemptEntity(event.getApplicationAttemptId());
+        createAppAttemptEntity(appAttempt.getAppAttemptId());
+
     TimelineEvent tEvent = new TimelineEvent();
-    tEvent.setEventType(
-        AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setEventType(AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE);
+    tEvent.setTimestamp(registeredTime);
     Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(
-        AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
-        event.getTrackingUrl());
-    eventInfo.put(
-        AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
-        event.getOriginalTrackingURL());
+    eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
+        appAttempt.getTrackingUrl());
+    eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
+        appAttempt.getOriginalTrackingUrl());
     eventInfo.put(AppAttemptMetricsConstants.HOST_EVENT_INFO,
-        event.getHost());
+        appAttempt.getHost());
     eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO,
-        event.getRpcPort());
-    if (event.getMasterContainerId() != null) {
-      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
-          event.getMasterContainerId().toString());
-    }
+        appAttempt.getRpcPort());
+    eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
+        appAttempt.getMasterContainer().getId().toString());
     tEvent.setEventInfo(eventInfo);
     entity.addEvent(tEvent);
-    putEntity(entity);
+    getDispatcher().getEventHandler().handle(
+        new TimelineV1PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY,
+            entity, appAttempt.getAppAttemptId().getApplicationId()));
+
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishAppAttemptFinishedEvent(AppAttemptFinishedEvent event) {
+  public void appAttemptFinished(RMAppAttempt appAttempt,
+      RMAppAttemptState appAttemtpState, RMApp app, long finishedTime) {
     TimelineEntity entity =
-        createAppAttemptEntity(event.getApplicationAttemptId());
+        createAppAttemptEntity(appAttempt.getAppAttemptId());
 
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setEventType(AppAttemptMetricsConstants.FINISHED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setTimestamp(finishedTime);
     Map<String, Object> eventInfo = new HashMap<String, Object>();
     eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
-        event.getTrackingUrl());
+        appAttempt.getTrackingUrl());
     eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
-        event.getOriginalTrackingURL());
+        appAttempt.getOriginalTrackingUrl());
     eventInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
-        event.getDiagnosticsInfo());
-    eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_EVENT_INFO, event
-        .getFinalApplicationStatus().toString());
-    eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO, event
-        .getYarnApplicationAttemptState().toString());
-    if (event.getMasterContainerId() != null) {
-      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
-          event.getMasterContainerId().toString());
-    }
+        appAttempt.getDiagnostics());
+    eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_EVENT_INFO,
+        app.getFinalApplicationStatus().toString());
+    eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO, RMServerUtils
+        .createApplicationAttemptState(appAttemtpState).toString());
     tEvent.setEventInfo(eventInfo);
 
     entity.addEvent(tEvent);
-    putEntity(entity);
-  }
-
-  private static TimelineEntity createAppAttemptEntity(
-      ApplicationAttemptId appAttemptId) {
-    TimelineEntity entity = new TimelineEntity();
-    entity.setEntityType(AppAttemptMetricsConstants.ENTITY_TYPE);
-    entity.setEntityId(appAttemptId.toString());
-    entity.addPrimaryFilter(AppAttemptMetricsConstants.PARENT_PRIMARY_FILTER,
-        appAttemptId.getApplicationId().toString());
-    return entity;
+    getDispatcher().getEventHandler().handle(
+        new TimelineV1PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY,
+            entity, appAttempt.getAppAttemptId().getApplicationId()));
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishContainerCreatedEvent(ContainerCreatedEvent event) {
-    TimelineEntity entity = createContainerEntity(event.getContainerId());
+  public void containerCreated(RMContainer container, long createdTime) {
+    TimelineEntity entity = createContainerEntity(container.getContainerId());
     Map<String, Object> entityInfo = new HashMap<String, Object>();
     entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO,
-        event.getAllocatedResource().getMemory());
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO, event
-        .getAllocatedResource().getVirtualCores());
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO, event
-        .getAllocatedNode().getHost());
-    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO, event
-        .getAllocatedNode().getPort());
+        container.getAllocatedResource().getMemory());
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO,
+        container.getAllocatedResource().getVirtualCores());
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
+        container.getAllocatedNode().getHost());
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
+        container.getAllocatedNode().getPort());
     entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO,
-        event.getAllocatedPriority().getPriority());
+        container.getAllocatedPriority().getPriority());
     entityInfo.put(
         ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO,
-        event.getNodeHttpAddress());
+        container.getNodeHttpAddress());
     entity.setOtherInfo(entityInfo);
 
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setEventType(ContainerMetricsConstants.CREATED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setTimestamp(createdTime);
 
     entity.addEvent(tEvent);
-    putEntity(entity);
+    getDispatcher().getEventHandler().handle(new TimelineV1PublishEvent(
+        SystemMetricsEventType.PUBLISH_ENTITY, entity, container
+            .getContainerId().getApplicationAttemptId().getApplicationId()));
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishContainerFinishedEvent(ContainerFinishedEvent event) {
-    TimelineEntity entity = createContainerEntity(event.getContainerId());
+  public void containerFinished(RMContainer container, long finishedTime) {
+    TimelineEntity entity = createContainerEntity(container.getContainerId());
 
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setEventType(ContainerMetricsConstants.FINISHED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setTimestamp(finishedTime);
     Map<String, Object> eventInfo = new HashMap<String, Object>();
     eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
-        event.getDiagnosticsInfo());
+        container.getDiagnosticsInfo());
     eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO,
-        event.getContainerExitStatus());
-    eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO, event
-        .getContainerState().toString());
+        container.getContainerExitStatus());
+    eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO,
+        container.getContainerState().toString());
     tEvent.setEventInfo(eventInfo);
 
     entity.addEvent(tEvent);
-    putEntity(entity);
+    getDispatcher().getEventHandler().handle(new TimelineV1PublishEvent(
+        SystemMetricsEventType.PUBLISH_ENTITY, entity, container
+            .getContainerId().getApplicationAttemptId().getApplicationId()));
+  }
+
+  private static TimelineEntity createApplicationEntity(
+      ApplicationId applicationId) {
+    TimelineEntity entity = new TimelineEntity();
+    entity.setEntityType(ApplicationMetricsConstants.ENTITY_TYPE);
+    entity.setEntityId(applicationId.toString());
+    return entity;
+  }
+
+  private static TimelineEntity createAppAttemptEntity(
+      ApplicationAttemptId appAttemptId) {
+    TimelineEntity entity = new TimelineEntity();
+    entity.setEntityType(AppAttemptMetricsConstants.ENTITY_TYPE);
+    entity.setEntityId(appAttemptId.toString());
+    entity.addPrimaryFilter(AppAttemptMetricsConstants.PARENT_PRIMARY_FILTER,
+        appAttemptId.getApplicationId().toString());
+    return entity;
   }
 
   private static TimelineEntity createContainerEntity(ContainerId containerId) {
@@ -305,4 +317,26 @@ public class TimelineServiceV1Publisher extends
           + entity.getEntityId() + "]", e);
     }
   }
+
+  private class TimelineV1PublishEvent extends TimelinePublishEvent {
+    private TimelineEntity entity;
+
+    public TimelineV1PublishEvent(SystemMetricsEventType type,
+        TimelineEntity entity, ApplicationId appId) {
+      super(type, appId);
+      this.entity = entity;
+    }
+
+    public TimelineEntity getEntity() {
+      return entity;
+    }
+  }
+
+  private class TimelineV1EventHandler
+      implements EventHandler<TimelineV1PublishEvent> {
+    @Override
+    public void handle(TimelineV1PublishEvent event) {
+      putEntity(event.getEntity());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/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 3a6c678..0105495 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
@@ -25,7 +25,6 @@ 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.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -39,93 +38,100 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.Identif
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
 import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * This class is responsible for posting application, appattempt & Container
  * lifecycle related events to timeline service V2
  */
 @Private
 @Unstable
-public class TimelineServiceV2Publisher extends
-    AbstractTimelineServicePublisher {
-  private static final Log LOG = LogFactory
-      .getLog(TimelineServiceV2Publisher.class);
+public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
+  private static final Log LOG =
+      LogFactory.getLog(TimelineServiceV2Publisher.class);
   protected RMTimelineCollectorManager rmTimelineCollectorManager;
+  private boolean publishContainerMetrics;
 
   public TimelineServiceV2Publisher(RMContext rmContext) {
     super("TimelineserviceV2Publisher");
     rmTimelineCollectorManager = rmContext.getRMTimelineCollectorManager();
   }
 
-  private boolean publishContainerMetrics;
-
   @Override
-  protected void serviceInit(Configuration conf) throws Exception {
-    publishContainerMetrics =
-        conf.getBoolean(YarnConfiguration.RM_PUBLISH_CONTAINER_METRICS_ENABLED,
-            YarnConfiguration.DEFAULT_RM_PUBLISH_CONTAINER_METRICS_ENABLED);
-    super.serviceInit(conf);
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    publishContainerMetrics = getConfig().getBoolean(
+        YarnConfiguration.RM_PUBLISH_CONTAINER_METRICS_ENABLED,
+        YarnConfiguration.DEFAULT_RM_PUBLISH_CONTAINER_METRICS_ENABLED);
+    getDispatcher().register(SystemMetricsEventType.class,
+        new TimelineV2EventHandler());
+  }
+
+  @VisibleForTesting
+  boolean isPublishContainerMetrics() {
+    return publishContainerMetrics;
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishApplicationCreatedEvent(ApplicationCreatedEvent event) {
-    TimelineEntity entity =
-        createApplicationEntity(event.getApplicationId());
+  public void appCreated(RMApp app, long createdTime) {
+    ApplicationEntity entity = createApplicationEntity(app.getApplicationId());
+    entity.setQueue(app.getQueue());
+    entity.setCreatedTime(createdTime);
+
     Map<String, Object> entityInfo = new HashMap<String, Object>();
-    entityInfo.put(ApplicationMetricsConstants.NAME_ENTITY_INFO,
-        event.getApplicationName());
+    entityInfo.put(ApplicationMetricsConstants.NAME_ENTITY_INFO, app.getName());
     entityInfo.put(ApplicationMetricsConstants.TYPE_ENTITY_INFO,
-        event.getApplicationType());
-    entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO,
-        event.getUser());
-    entityInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO,
-        event.getQueue());
+        app.getApplicationType());
+    entityInfo.put(ApplicationMetricsConstants.USER_ENTITY_INFO, app.getUser());
     entityInfo.put(ApplicationMetricsConstants.SUBMITTED_TIME_ENTITY_INFO,
-        event.getSubmittedTime());
+        app.getSubmitTime());
     entityInfo.put(ApplicationMetricsConstants.APP_TAGS_INFO,
-        event.getAppTags());
+        app.getApplicationTags());
     entityInfo.put(
         ApplicationMetricsConstants.UNMANAGED_APPLICATION_ENTITY_INFO,
-        event.isUnmanagedApp());
+        app.getApplicationSubmissionContext().getUnmanagedAM());
     entityInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO,
-        event.getApplicationPriority().getPriority());
-    entityInfo.put(ApplicationMetricsConstants.APP_NODE_LABEL_EXPRESSION,
-        event.getAppNodeLabelsExpression());
-    entityInfo.put(ApplicationMetricsConstants.AM_NODE_LABEL_EXPRESSION,
-        event.getAmNodeLabelsExpression());
-    if (event.getCallerContext() != null) {
-      if (event.getCallerContext().getContext() != null) {
-        entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_CONTEXT,
-            event.getCallerContext().getContext());
-      }
-      if (event.getCallerContext().getSignature() != null) {
-        entityInfo.put(ApplicationMetricsConstants.YARN_APP_CALLER_SIGNATURE,
-            event.getCallerContext().getSignature());
-      }
-    }
+        app.getApplicationSubmissionContext().getPriority().getPriority());
+    entity.getConfigs().put(
+        ApplicationMetricsConstants.AM_NODE_LABEL_EXPRESSION,
+        app.getAmNodeLabelExpression());
+    entity.getConfigs().put(
+        ApplicationMetricsConstants.APP_NODE_LABEL_EXPRESSION,
+        app.getAppNodeLabelExpression());
     entity.setInfo(entityInfo);
 
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setTimestamp(createdTime);
     entity.addEvent(tEvent);
 
-    putEntity(entity, event.getApplicationId());
+    getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
+        SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishApplicationFinishedEvent(ApplicationFinishedEvent event) {
-    ApplicationEntity entity =
-        createApplicationEntity(event.getApplicationId());
-    RMAppMetrics appMetrics = event.getAppMetrics();
+  public void appFinished(RMApp app, RMAppState state, long finishedTime) {
+    ApplicationEntity entity = createApplicationEntity(app.getApplicationId());
+    RMAppMetrics appMetrics = app.getRMAppMetrics();
     entity.addInfo(ApplicationMetricsConstants.APP_CPU_METRICS,
         appMetrics.getVcoreSeconds());
     entity.addInfo(ApplicationMetricsConstants.APP_MEM_METRICS,
@@ -133,54 +139,57 @@ public class TimelineServiceV2Publisher extends
 
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setTimestamp(finishedTime);
     Map<String, Object> eventInfo = new HashMap<String, Object>();
     eventInfo.put(ApplicationMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
-        event.getDiagnosticsInfo());
-    eventInfo.put(ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO, event
-        .getFinalApplicationStatus().toString());
-    eventInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO, event
-        .getYarnApplicationState().toString());
-    if (event.getLatestApplicationAttemptId() != null) {
+        app.getDiagnostics().toString());
+    eventInfo.put(ApplicationMetricsConstants.FINAL_STATUS_EVENT_INFO,
+        app.getFinalApplicationStatus().toString());
+    eventInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO,
+        RMServerUtils.createApplicationState(state).toString());
+    ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt() == null
+        ? null : app.getCurrentAppAttempt().getAppAttemptId();
+    if (appAttemptId != null) {
       eventInfo.put(ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO,
-          event.getLatestApplicationAttemptId().toString());
+          appAttemptId.toString());
     }
     tEvent.setInfo(eventInfo);
 
     entity.addEvent(tEvent);
-    putEntity(entity, event.getApplicationId());
 
-    //cleaning up the collector cached
-    event.getApp().stopTimelineCollector();
+    getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
+        SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public void appACLsUpdated(RMApp app, String appViewACLs, long updatedTime) {
+    ApplicationEntity entity = createApplicationEntity(app.getApplicationId());
+    Map<String, Object> entityInfo = new HashMap<String, Object>();
+    entityInfo.put(ApplicationMetricsConstants.APP_VIEW_ACLS_ENTITY_INFO,
+        appViewACLs);
+    entity.setInfo(entityInfo);
+
+    getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
+        SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishApplicationUpdatedEvent(ApplicationUpdatedEvent event) {
-    ApplicationEntity entity =
-        createApplicationEntity(event.getApplicationId());
+  public void appUpdated(RMApp app, long currentTimeMillis) {
+    ApplicationEntity entity = createApplicationEntity(app.getApplicationId());
     Map<String, Object> eventInfo = new HashMap<String, Object>();
     eventInfo.put(ApplicationMetricsConstants.QUEUE_ENTITY_INFO,
-        event.getQueue());
-    eventInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO, event
-        .getApplicationPriority().getPriority());
+        app.getQueue());
+    eventInfo.put(ApplicationMetricsConstants.APPLICATION_PRIORITY_INFO,
+        app.getApplicationSubmissionContext().getPriority().getPriority());
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setId(ApplicationMetricsConstants.UPDATED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setTimestamp(currentTimeMillis);
     tEvent.setInfo(eventInfo);
     entity.addEvent(tEvent);
-    putEntity(entity, event.getApplicationId());
-  }
-
-  @Override
-  void publishApplicationACLsUpdatedEvent(ApplicationACLsUpdatedEvent event) {
-    ApplicationEntity entity =
-        createApplicationEntity(event.getApplicationId());
-    Map<String, Object> entityInfo = new HashMap<String, Object>();
-    entityInfo.put(ApplicationMetricsConstants.APP_VIEW_ACLS_ENTITY_INFO,
-        event.getViewAppACLs());
-    entity.setInfo(entityInfo);
-
-    putEntity(entity, event.getApplicationId());
+    getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
+        SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
   }
 
   private static ApplicationEntity createApplicationEntity(
@@ -190,111 +199,134 @@ public class TimelineServiceV2Publisher extends
     return entity;
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishAppAttemptRegisteredEvent(AppAttemptRegisteredEvent event) {
+  public void appAttemptRegistered(RMAppAttempt appAttempt,
+      long registeredTime) {
     TimelineEntity entity =
-        createAppAttemptEntity(event.getApplicationAttemptId());
+        createAppAttemptEntity(appAttempt.getAppAttemptId());
+    entity.setCreatedTime(registeredTime);
+
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setId(AppAttemptMetricsConstants.REGISTERED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setTimestamp(registeredTime);
     Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(
-        AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
-        event.getTrackingUrl());
-    eventInfo.put(
-        AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
-        event.getOriginalTrackingURL());
+    eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
+        appAttempt.getTrackingUrl());
+    eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
+        appAttempt.getOriginalTrackingUrl());
     eventInfo.put(AppAttemptMetricsConstants.HOST_EVENT_INFO,
-        event.getHost());
+        appAttempt.getHost());
     eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO,
-        event.getRpcPort());
-    if (event.getMasterContainerId() != null) {
-      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
-          event.getMasterContainerId().toString());
-    }
+        appAttempt.getRpcPort());
+    eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
+        appAttempt.getMasterContainer().getId().toString());
     tEvent.setInfo(eventInfo);
     entity.addEvent(tEvent);
-    putEntity(entity, event.getApplicationAttemptId().getApplicationId());
+    getDispatcher().getEventHandler().handle(
+        new TimelineV2PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY,
+            entity, appAttempt.getAppAttemptId().getApplicationId()));
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishAppAttemptFinishedEvent(AppAttemptFinishedEvent event) {
+  public void appAttemptFinished(RMAppAttempt appAttempt,
+      RMAppAttemptState appAttemtpState, RMApp app, long finishedTime) {
+
     ApplicationAttemptEntity entity =
-        createAppAttemptEntity(event.getApplicationAttemptId());
+        createAppAttemptEntity(appAttempt.getAppAttemptId());
 
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setId(AppAttemptMetricsConstants.FINISHED_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
+    tEvent.setTimestamp(finishedTime);
     Map<String, Object> eventInfo = new HashMap<String, Object>();
     eventInfo.put(AppAttemptMetricsConstants.TRACKING_URL_EVENT_INFO,
-        event.getTrackingUrl());
+        appAttempt.getTrackingUrl());
     eventInfo.put(AppAttemptMetricsConstants.ORIGINAL_TRACKING_URL_EVENT_INFO,
-        event.getOriginalTrackingURL());
+        appAttempt.getOriginalTrackingUrl());
     eventInfo.put(AppAttemptMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
-        event.getDiagnosticsInfo());
-    eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_EVENT_INFO, event
-        .getFinalApplicationStatus().toString());
-    eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO, event
-        .getYarnApplicationAttemptState().toString());
-    if (event.getMasterContainerId() != null) {
-      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
-          event.getMasterContainerId().toString());
-    }
+        appAttempt.getDiagnostics());
+    // app will get the final status from app attempt, or create one
+    // based on app state if it doesn't exist
+    eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_EVENT_INFO,
+        app.getFinalApplicationStatus().toString());
+    eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO, RMServerUtils
+        .createApplicationAttemptState(appAttemtpState).toString());
     tEvent.setInfo(eventInfo);
 
     entity.addEvent(tEvent);
-    putEntity(entity, event.getApplicationAttemptId().getApplicationId());
+    getDispatcher().getEventHandler().handle(
+        new TimelineV2PublishEvent(SystemMetricsEventType.PUBLISH_ENTITY,
+            entity, appAttempt.getAppAttemptId().getApplicationId()));
   }
 
-  @Override
-  void publishContainerCreatedEvent(ContainerCreatedEvent event) {
-    TimelineEntity entity = createContainerEntity(event.getContainerId());
-
-    TimelineEvent tEvent = new TimelineEvent();
-    tEvent.setId(ContainerMetricsConstants.CREATED_IN_RM_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
-    // updated as event info instead of entity info, as entity info is updated
-    // by NM
-    Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO, event
-        .getAllocatedResource().getMemory());
-    eventInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO, event
-        .getAllocatedResource().getVirtualCores());
-    eventInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO, event
-        .getAllocatedNode().getHost());
-    eventInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO, event
-        .getAllocatedNode().getPort());
-    eventInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO,
-        event.getAllocatedPriority().getPriority());
-    eventInfo.put(
-        ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO,
-        event.getNodeHttpAddress());
-    tEvent.setInfo(eventInfo);
-
-    entity.addEvent(tEvent);
-    putEntity(entity, event.getContainerId().getApplicationAttemptId()
-        .getApplicationId());
+  private static ApplicationAttemptEntity createAppAttemptEntity(
+      ApplicationAttemptId appAttemptId) {
+    ApplicationAttemptEntity entity = new ApplicationAttemptEntity();
+    entity.setId(appAttemptId.toString());
+    entity.setParent(new Identifier(TimelineEntityType.YARN_APPLICATION.name(),
+        appAttemptId.getApplicationId().toString()));
+    return entity;
   }
 
+  @SuppressWarnings("unchecked")
   @Override
-  void publishContainerFinishedEvent(ContainerFinishedEvent event) {
-    TimelineEntity entity = createContainerEntity(event.getContainerId());
-
-    TimelineEvent tEvent = new TimelineEvent();
-    tEvent.setId(ContainerMetricsConstants.FINISHED_IN_RM_EVENT_TYPE);
-    tEvent.setTimestamp(event.getTimestamp());
-    Map<String, Object> eventInfo = new HashMap<String, Object>();
-    eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
-        event.getDiagnosticsInfo());
-    eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO,
-        event.getContainerExitStatus());
-    eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO, event
-        .getContainerState().toString());
-    tEvent.setInfo(eventInfo);
+  public void containerCreated(RMContainer container, long createdTime) {
+    if (publishContainerMetrics) {
+      TimelineEntity entity = createContainerEntity(container.getContainerId());
+      entity.setCreatedTime(createdTime);
+
+      TimelineEvent tEvent = new TimelineEvent();
+      tEvent.setId(ContainerMetricsConstants.CREATED_IN_RM_EVENT_TYPE);
+      tEvent.setTimestamp(createdTime);
+      // updated as event info instead of entity info, as entity info is updated
+      // by NM
+      Map<String, Object> eventInfo = new HashMap<String, Object>();
+      eventInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO,
+          container.getAllocatedResource().getMemory());
+      eventInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO,
+          container.getAllocatedResource().getVirtualCores());
+      eventInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
+          container.getAllocatedNode().getHost());
+      eventInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
+          container.getAllocatedNode().getPort());
+      eventInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO,
+          container.getAllocatedPriority().getPriority());
+      eventInfo.put(
+          ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO,
+          container.getNodeHttpAddress());
+      tEvent.setInfo(eventInfo);
+
+      entity.addEvent(tEvent);
+      getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
+          SystemMetricsEventType.PUBLISH_ENTITY, entity, container
+              .getContainerId().getApplicationAttemptId().getApplicationId()));
+    }
+  }
 
-    entity.addEvent(tEvent);
-    putEntity(entity, event.getContainerId().getApplicationAttemptId()
-        .getApplicationId());
+  @SuppressWarnings("unchecked")
+  @Override
+  public void containerFinished(RMContainer container, long finishedTime) {
+    if (publishContainerMetrics) {
+      TimelineEntity entity = createContainerEntity(container.getContainerId());
+
+      TimelineEvent tEvent = new TimelineEvent();
+      tEvent.setId(ContainerMetricsConstants.FINISHED_IN_RM_EVENT_TYPE);
+      tEvent.setTimestamp(finishedTime);
+      Map<String, Object> eventInfo = new HashMap<String, Object>();
+      eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+          container.getDiagnosticsInfo());
+      eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO,
+          container.getContainerExitStatus());
+      eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO,
+          container.getContainerState().toString());
+      tEvent.setInfo(eventInfo);
+
+      entity.addEvent(tEvent);
+      getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
+          SystemMetricsEventType.PUBLISH_ENTITY, entity, container
+              .getContainerId().getApplicationAttemptId().getApplicationId()));
+    }
   }
 
   private static ContainerEntity createContainerEntity(ContainerId containerId) {
@@ -322,17 +354,48 @@ public class TimelineServiceV2Publisher extends
     }
   }
 
-  private static ApplicationAttemptEntity createAppAttemptEntity(
-      ApplicationAttemptId appAttemptId) {
-    ApplicationAttemptEntity entity = new ApplicationAttemptEntity();
-    entity.setId(appAttemptId.toString());
-    entity.setParent(new Identifier(TimelineEntityType.YARN_APPLICATION.name(),
-        appAttemptId.getApplicationId().toString()));
-    return entity;
+  private class ApplicationFinishPublishEvent extends TimelineV2PublishEvent {
+    private RMAppImpl app;
+
+    public ApplicationFinishPublishEvent(SystemMetricsEventType type,
+        TimelineEntity entity, RMAppImpl app) {
+      super(type, entity, app.getApplicationId());
+      this.app = app;
+    }
+
+    public RMAppImpl getRMAppImpl() {
+      return app;
+    }
   }
 
-  @Override
-  public boolean publishRMContainerMetrics() {
-    return publishContainerMetrics;
+  private class TimelineV2EventHandler
+      implements EventHandler<TimelineV2PublishEvent> {
+    @Override
+    public void handle(TimelineV2PublishEvent event) {
+      switch (event.getType()) {
+      case PUBLISH_APPLICATION_FINISHED_ENTITY:
+        putEntity(event.getEntity(), event.getApplicationId());
+        ((ApplicationFinishPublishEvent) event).getRMAppImpl()
+            .stopTimelineCollector();
+        break;
+      default:
+        putEntity(event.getEntity(), event.getApplicationId());
+        break;
+      }
+    }
+  }
+
+  private class TimelineV2PublishEvent extends TimelinePublishEvent {
+    private TimelineEntity entity;
+
+    public TimelineV2PublishEvent(SystemMetricsEventType type,
+        TimelineEntity entity, ApplicationId appId) {
+      super(type, appId);
+      this.entity = entity;
+    }
+
+    public TimelineEntity getEntity() {
+      return entity;
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.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/TestSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
index ac5db49..2652acf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
@@ -48,7 +48,6 @@ import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistor
 import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
@@ -69,7 +68,7 @@ import org.junit.Test;
 public class TestSystemMetricsPublisher {
 
   private static ApplicationHistoryServer timelineServer;
-  private static SystemMetricsPublisher metricsPublisher;
+  private static TimelineServiceV1Publisher metricsPublisher;
   private static TimelineStore store;
 
   @BeforeClass
@@ -90,7 +89,7 @@ public class TestSystemMetricsPublisher {
     timelineServer.start();
     store = timelineServer.getTimelineStore();
 
-    metricsPublisher = new SystemMetricsPublisher(mock(RMContext.class));
+    metricsPublisher = new TimelineServiceV1Publisher();
     metricsPublisher.init(conf);
     metricsPublisher.start();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/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 ac20335..20a5b13 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
@@ -49,7 +49,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.metrics.AbstractTimelineServicePublisher.MultiThreadedDispatcher;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
@@ -75,7 +74,7 @@ public class TestSystemMetricsPublisherForV2 {
       TestSystemMetricsPublisherForV2.class.getName() + "-localDir")
       .getAbsoluteFile();
 
-  private static SystemMetricsPublisher metricsPublisher;
+  private static TimelineServiceV2Publisher metricsPublisher;
   private static DrainDispatcher dispatcher = new DrainDispatcher();
   private static final String DEFAULT_FLOW_VERSION = "1";
   private static final long DEFAULT_FLOW_RUN = 1;
@@ -103,10 +102,11 @@ public class TestSystemMetricsPublisherForV2 {
     rmTimelineCollectorManager.init(conf);
     rmTimelineCollectorManager.start();
 
-    metricsPublisher = new SystemMetricsPublisher(rmContext) {
+    dispatcher.init(conf);
+    dispatcher.start();
+    metricsPublisher = new TimelineServiceV2Publisher(rmContext) {
       @Override
-      Dispatcher createDispatcher(
-          TimelineServicePublisher timelineServicePublisher) {
+      protected Dispatcher getDispatcher() {
         return dispatcher;
       }
     };
@@ -150,8 +150,8 @@ public class TestSystemMetricsPublisherForV2 {
   @Test
   public void testSystemMetricPublisherInitialization() {
     @SuppressWarnings("resource")
-    SystemMetricsPublisher metricsPublisher =
-        new SystemMetricsPublisher(mock(RMContext.class));
+    TimelineServiceV2Publisher metricsPublisher =
+        new TimelineServiceV2Publisher(mock(RMContext.class));
     try {
       Configuration conf = getTimelineV2Conf();
       conf.setBoolean(YarnConfiguration.RM_PUBLISH_CONTAINER_METRICS_ENABLED,
@@ -163,20 +163,18 @@ public class TestSystemMetricsPublisherForV2 {
 
       metricsPublisher.stop();
 
-      metricsPublisher = new SystemMetricsPublisher(mock(RMContext.class));
+      metricsPublisher = new TimelineServiceV2Publisher(mock(RMContext.class));
       conf = getTimelineV2Conf();
       metricsPublisher.init(conf);
+      metricsPublisher.start();
       assertTrue("Expected to publish container Metrics from RM",
           metricsPublisher.isPublishContainerMetrics());
-      assertTrue(
-          "MultiThreadedDispatcher expected when container Metrics is not published",
-          metricsPublisher.getDispatcher() instanceof MultiThreadedDispatcher);
     } finally {
       metricsPublisher.stop();
     }
   }
 
-  @Test(timeout = 1000000)
+  @Test(timeout = 10000)
   public void testPublishApplicationMetrics() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     RMApp app = createAppAndRegister(appId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.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/scheduler/fair/TestFSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java
index 7637410..a642a78 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSLeafQueue.java
@@ -43,7 +43,6 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsEvent;
 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.event.NodeAddedSchedulerEvent;


[12/50] [abbrv] hadoop git commit: YARN-3901. Populate flow run data in the flow_run & flow activity tables (Vrushali C via sjlee)

Posted by gt...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
new file mode 100644
index 0000000..b4a0c74
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
@@ -0,0 +1,372 @@
+/**
+ * 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.flow;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+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.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+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.hbase.client.Get;
+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.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+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.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.timeline.GenericObjectMapper;
+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.application.ApplicationTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Tests the FlowRun and FlowActivity Tables
+ */
+public class TestHBaseStorageFlowActivity {
+
+  private static HBaseTestingUtility util;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    util = new HBaseTestingUtility();
+    Configuration conf = util.getConfiguration();
+    conf.setInt("hfile.format.version", 3);
+    util.startMiniCluster();
+    createSchema();
+  }
+
+  private static void createSchema() throws IOException {
+    TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
+  }
+
+  /**
+   * Writes 4 timeline entities belonging to one flow run through the
+   * {@link HBaseTimelineWriterImpl}
+   *
+   * Checks the flow run table contents
+   *
+   * The first entity has a created event, metrics and a finish event.
+   *
+   * The second entity has a created event and this is the entity with smallest
+   * start time. This should be the start time for the flow run.
+   *
+   * The third entity has a finish event and this is the entity with the max end
+   * time. This should be the end time for the flow run.
+   *
+   * The fourth entity has a created event which has a start time that is
+   * greater than min start time.
+   *
+   * The test also checks in the flow activity table that one entry has been
+   * made for all of these 4 application entities since they belong to the same
+   * flow run.
+   */
+  @Test
+  public void testWriteFlowRunMinMax() throws Exception {
+
+    TimelineEntities te = new TimelineEntities();
+    te.addEntity(TestFlowDataGenerator.getEntity1());
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    String cluster = "testWriteFlowRunMinMaxToHBase_cluster1";
+    String user = "testWriteFlowRunMinMaxToHBase_user1";
+    String flow = "testing_flowRun_flow_name";
+    String flowVersion = "CF7022C10F1354";
+    Long runid = 1002345678919L;
+    String appName = "application_100000000000_1111";
+    long endTs = 1439750690000L;
+    TimelineEntity entityMinStartTime = TestFlowDataGenerator
+        .getEntityMinStartTime();
+
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+
+      // 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);
+
+      // writer another entity for max end time
+      TimelineEntity entityMaxEndTime = TestFlowDataGenerator
+          .getEntityMaxEndTime(endTs);
+      te = new TimelineEntities();
+      te.addEntity(entityMaxEndTime);
+      appName = "application_100000000000_4444";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+
+      // writer another entity with greater start time
+      TimelineEntity entityGreaterStartTime = TestFlowDataGenerator
+          .getEntityGreaterStartTime();
+      te = new TimelineEntities();
+      te.addEntity(entityGreaterStartTime);
+      appName = "application_1000000000000000_2222";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+
+      // flush everything to hbase
+      hbi.flush();
+    } finally {
+      hbi.close();
+    }
+
+    Connection conn = ConnectionFactory.createConnection(c1);
+    // check in flow activity table
+    Table table1 = conn.getTable(TableName
+        .valueOf(FlowActivityTable.DEFAULT_TABLE_NAME));
+    byte[] startRow = FlowActivityRowKey.getRowKey(cluster, user, flow);
+    Get g = new Get(startRow);
+    Result r1 = table1.get(g);
+    assertNotNull(r1);
+    assertTrue(!r1.isEmpty());
+    Map<byte[], byte[]> values = r1.getFamilyMap(FlowActivityColumnFamily.INFO
+        .getBytes());
+    assertEquals(1, values.size());
+    byte[] row = r1.getRow();
+    FlowActivityRowKey flowActivityRowKey = FlowActivityRowKey.parseRowKey(row);
+    assertNotNull(flowActivityRowKey);
+    assertEquals(cluster, flowActivityRowKey.getClusterId());
+    assertEquals(user, flowActivityRowKey.getUserId());
+    assertEquals(flow, flowActivityRowKey.getFlowId());
+    long dayTs = TimelineWriterUtils.getTopOfTheDayTimestamp(System
+        .currentTimeMillis());
+    assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
+    assertEquals(1, values.size());
+    checkFlowActivityRunId(runid, flowVersion, values);
+  }
+
+  /**
+   * Write 1 application entity and checks the record for today in the flow
+   * activity table
+   */
+  @Test
+  public void testWriteFlowActivityOneFlow() throws Exception {
+    String cluster = "testWriteFlowActivityOneFlow_cluster1";
+    String user = "testWriteFlowActivityOneFlow_user1";
+    String flow = "flow_activity_test_flow_name";
+    String flowVersion = "A122110F135BC4";
+    Long runid = 1001111178919L;
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entityApp1 = TestFlowDataGenerator.getFlowApp1();
+    te.addEntity(entityApp1);
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      String appName = "application_1111999999_1234";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.flush();
+    } finally {
+      hbi.close();
+    }
+    // check flow activity
+    checkFlowActivityTable(cluster, user, flow, flowVersion, runid, c1);
+  }
+
+  private void checkFlowActivityTable(String cluster, String user, String flow,
+      String flowVersion, Long runid, Configuration c1) throws IOException {
+    Scan s = new Scan();
+    s.addFamily(FlowActivityColumnFamily.INFO.getBytes());
+    byte[] startRow = FlowActivityRowKey.getRowKey(cluster, user, flow);
+    s.setStartRow(startRow);
+    String clusterStop = cluster + "1";
+    byte[] stopRow = FlowActivityRowKey.getRowKey(clusterStop, user, flow);
+    s.setStopRow(stopRow);
+    Connection conn = ConnectionFactory.createConnection(c1);
+    Table table1 = conn.getTable(TableName
+        .valueOf(FlowActivityTable.DEFAULT_TABLE_NAME));
+    ResultScanner scanner = table1.getScanner(s);
+    int rowCount = 0;
+    for (Result result : scanner) {
+      assertNotNull(result);
+      assertTrue(!result.isEmpty());
+      Map<byte[], byte[]> values = result
+          .getFamilyMap(FlowActivityColumnFamily.INFO.getBytes());
+      rowCount++;
+      byte[] row = result.getRow();
+      FlowActivityRowKey flowActivityRowKey = FlowActivityRowKey
+          .parseRowKey(row);
+      assertNotNull(flowActivityRowKey);
+      assertEquals(cluster, flowActivityRowKey.getClusterId());
+      assertEquals(user, flowActivityRowKey.getUserId());
+      assertEquals(flow, flowActivityRowKey.getFlowId());
+      long dayTs = TimelineWriterUtils.getTopOfTheDayTimestamp(System
+          .currentTimeMillis());
+      assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
+      assertEquals(1, values.size());
+      checkFlowActivityRunId(runid, flowVersion, values);
+    }
+    assertEquals(1, rowCount);
+  }
+
+  /**
+   * Writes 3 applications each with a different run id and version for the same
+   * {cluster, user, flow}
+   *
+   * They should be getting inserted into one record in the flow activity table
+   * with 3 columns, one per run id
+   */
+  @Test
+  public void testFlowActivityTableOneFlowMultipleRunIds() throws IOException {
+    String cluster = "testManyRunsFlowActivity_cluster1";
+    String user = "testManyRunsFlowActivity_c_user1";
+    String flow = "flow_activity_test_flow_name";
+    String flowVersion1 = "A122110F135BC4";
+    Long runid1 = 11111111111L;
+
+    String flowVersion2 = "A12222222222C4";
+    long runid2 = 2222222222222L;
+
+    String flowVersion3 = "A1333333333C4";
+    long runid3 = 3333333333333L;
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entityApp1 = TestFlowDataGenerator.getFlowApp1();
+    te.addEntity(entityApp1);
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      String appName = "application_11888888888_1111";
+      hbi.write(cluster, user, flow, flowVersion1, runid1, appName, te);
+
+      // 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);
+
+      // 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.flush();
+    } finally {
+      hbi.close();
+    }
+    // check flow activity
+    checkFlowActivityTableSeveralRuns(cluster, user, flow, c1, flowVersion1,
+        runid1, flowVersion2, runid2, flowVersion3, runid3);
+
+  }
+
+  private void checkFlowActivityTableSeveralRuns(String cluster, String user,
+      String flow, Configuration c1, String flowVersion1, Long runid1,
+      String flowVersion2, Long runid2, String flowVersion3, Long runid3)
+      throws IOException {
+    Scan s = new Scan();
+    s.addFamily(FlowActivityColumnFamily.INFO.getBytes());
+    byte[] startRow = FlowActivityRowKey.getRowKey(cluster, user, flow);
+    s.setStartRow(startRow);
+    String clusterStop = cluster + "1";
+    byte[] stopRow = FlowActivityRowKey.getRowKey(clusterStop, user, flow);
+    s.setStopRow(stopRow);
+    Connection conn = ConnectionFactory.createConnection(c1);
+    Table table1 = conn.getTable(TableName
+        .valueOf(FlowActivityTable.DEFAULT_TABLE_NAME));
+    ResultScanner scanner = table1.getScanner(s);
+    int rowCount = 0;
+    for (Result result : scanner) {
+      assertNotNull(result);
+      assertTrue(!result.isEmpty());
+      byte[] row = result.getRow();
+      FlowActivityRowKey flowActivityRowKey = FlowActivityRowKey
+          .parseRowKey(row);
+      assertNotNull(flowActivityRowKey);
+      assertEquals(cluster, flowActivityRowKey.getClusterId());
+      assertEquals(user, flowActivityRowKey.getUserId());
+      assertEquals(flow, flowActivityRowKey.getFlowId());
+      long dayTs = TimelineWriterUtils.getTopOfTheDayTimestamp(System
+          .currentTimeMillis());
+      assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
+
+      Map<byte[], byte[]> values = result
+          .getFamilyMap(FlowActivityColumnFamily.INFO.getBytes());
+      rowCount++;
+      assertEquals(3, values.size());
+      checkFlowActivityRunId(runid1, flowVersion1, values);
+      checkFlowActivityRunId(runid2, flowVersion2, values);
+      checkFlowActivityRunId(runid3, flowVersion3, values);
+    }
+    // the flow activity table is such that it will insert
+    // into current day's record
+    // hence, if this test runs across the midnight boundary,
+    // it may fail since it would insert into two records
+    // one for each day
+    assertEquals(1, rowCount);
+  }
+
+  private void checkFlowActivityRunId(Long runid, String flowVersion,
+      Map<byte[], byte[]> values) throws IOException {
+    byte[] rq = ColumnHelper.getColumnQualifier(
+        FlowActivityColumnPrefix.RUN_ID.getColumnPrefixBytes(),
+        GenericObjectMapper.write(runid));
+    for (Map.Entry<byte[], byte[]> k : values.entrySet()) {
+      String actualQ = Bytes.toString(k.getKey());
+      if (Bytes.toString(rq).equals(actualQ)) {
+        String actualV = (String) GenericObjectMapper.read(k.getValue());
+        assertEquals(flowVersion, actualV);
+      }
+    }
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    util.shutdownMiniCluster();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
new file mode 100644
index 0000000..bf524ea
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.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.flow;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+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.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+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.hbase.client.Get;
+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.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+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.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.timeline.GenericObjectMapper;
+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.application.ApplicationTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
+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.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnFamily;
+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.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Tests the FlowRun and FlowActivity Tables
+ */
+public class TestHBaseStorageFlowRun {
+
+  private static HBaseTestingUtility util;
+
+  private final String metric1 = "MAP_SLOT_MILLIS";
+  private final String metric2 = "HDFS_BYTES_READ";
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    util = new HBaseTestingUtility();
+    Configuration conf = util.getConfiguration();
+    conf.setInt("hfile.format.version", 3);
+    util.startMiniCluster();
+    createSchema();
+  }
+
+  private static void createSchema() throws IOException {
+    TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
+  }
+
+  /**
+   * Writes 4 timeline entities belonging to one flow run through the
+   * {@link HBaseTimelineWriterImpl}
+   *
+   * Checks the flow run table contents
+   *
+   * The first entity has a created event, metrics and a finish event.
+   *
+   * The second entity has a created event and this is the entity with smallest
+   * start time. This should be the start time for the flow run.
+   *
+   * The third entity has a finish event and this is the entity with the max end
+   * time. This should be the end time for the flow run.
+   *
+   * The fourth entity has a created event which has a start time that is
+   * greater than min start time.
+   *
+   */
+  @Test
+  public void testWriteFlowRunMinMax() throws Exception {
+
+    TimelineEntities te = new TimelineEntities();
+    te.addEntity(TestFlowDataGenerator.getEntity1());
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    String cluster = "testWriteFlowRunMinMaxToHBase_cluster1";
+    String user = "testWriteFlowRunMinMaxToHBase_user1";
+    String flow = "testing_flowRun_flow_name";
+    String flowVersion = "CF7022C10F1354";
+    Long runid = 1002345678919L;
+    String appName = "application_100000000000_1111";
+    long endTs = 1439750690000L;
+    TimelineEntity entityMinStartTime = TestFlowDataGenerator
+        .getEntityMinStartTime();
+
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+
+      // 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);
+
+      // writer another entity for max end time
+      TimelineEntity entityMaxEndTime = TestFlowDataGenerator
+          .getEntityMaxEndTime(endTs);
+      te = new TimelineEntities();
+      te.addEntity(entityMaxEndTime);
+      appName = "application_100000000000_4444";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+
+      // writer another entity with greater start time
+      TimelineEntity entityGreaterStartTime = TestFlowDataGenerator
+          .getEntityGreaterStartTime();
+      te = new TimelineEntities();
+      te.addEntity(entityGreaterStartTime);
+      appName = "application_1000000000000000_2222";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+
+      // flush everything to hbase
+      hbi.flush();
+    } finally {
+      hbi.close();
+    }
+
+    Connection conn = ConnectionFactory.createConnection(c1);
+    // check in flow run table
+    Table table1 = conn.getTable(TableName
+        .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
+    // scan the table and see that we get back the right min and max
+    // timestamps
+    byte[] startRow = FlowRunRowKey.getRowKey(cluster, user, flow, runid);
+    Get g = new Get(startRow);
+    g.addColumn(FlowRunColumnFamily.INFO.getBytes(),
+        FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes());
+    g.addColumn(FlowRunColumnFamily.INFO.getBytes(),
+        FlowRunColumn.MAX_END_TIME.getColumnQualifierBytes());
+    Result r1 = table1.get(g);
+    assertNotNull(r1);
+    assertTrue(!r1.isEmpty());
+    Map<byte[], byte[]> values = r1.getFamilyMap(FlowRunColumnFamily.INFO
+        .getBytes());
+
+    assertEquals(2, r1.size());
+    Long starttime = (Long) GenericObjectMapper.read(values
+        .get(FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes()));
+    Long expmin = entityMinStartTime.getCreatedTime();
+    assertEquals(expmin, starttime);
+    assertEquals(endTs, GenericObjectMapper.read(values
+        .get(FlowRunColumn.MAX_END_TIME.getColumnQualifierBytes())));
+  }
+
+  boolean isFlowRunRowKeyCorrect(byte[] rowKey, String cluster, String user,
+      String flow, Long runid) {
+    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey, -1);
+    assertTrue(rowKeyComponents.length == 4);
+    assertEquals(cluster, Bytes.toString(rowKeyComponents[0]));
+    assertEquals(user, Bytes.toString(rowKeyComponents[1]));
+    assertEquals(flow, Bytes.toString(rowKeyComponents[2]));
+    assertEquals(TimelineWriterUtils.invert(runid),
+        Bytes.toLong(rowKeyComponents[3]));
+    return true;
+  }
+
+  /**
+   * Writes two application entities of the same flow run. Each application has
+   * two metrics: slot millis and hdfs bytes read. Each metric has values at two
+   * timestamps.
+   *
+   * Checks the metric values of the flow in the flow run table. Flow metric
+   * values should be the sum of individual metric values that belong to the
+   * latest timestamp for that metric
+   */
+  @Test
+  public void testWriteFlowRunMetricsOneFlow() throws Exception {
+    String cluster = "testWriteFlowRunMetricsOneFlow_cluster1";
+    String user = "testWriteFlowRunMetricsOneFlow_user1";
+    String flow = "testing_flowRun_metrics_flow_name";
+    String flowVersion = "CF7022C10F1354";
+    Long runid = 1002345678919L;
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1();
+    te.addEntity(entityApp1);
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      String appName = "application_11111111111111_1111";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      // write another application with same metric to this flow
+      te = new TimelineEntities();
+      TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2();
+      te.addEntity(entityApp2);
+      appName = "application_11111111111111_2222";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.flush();
+    } finally {
+      hbi.close();
+    }
+
+    // check flow run
+    checkFlowRunTable(cluster, user, flow, runid, c1);
+  }
+
+  private void checkFlowRunTable(String cluster, String user, String flow,
+      long runid, Configuration c1) throws IOException {
+    Scan s = new Scan();
+    s.addFamily(FlowRunColumnFamily.INFO.getBytes());
+    byte[] startRow = FlowRunRowKey.getRowKey(cluster, user, flow, runid);
+    s.setStartRow(startRow);
+    String clusterStop = cluster + "1";
+    byte[] stopRow = FlowRunRowKey.getRowKey(clusterStop, user, flow, runid);
+    s.setStopRow(stopRow);
+    Connection conn = ConnectionFactory.createConnection(c1);
+    Table table1 = conn.getTable(TableName
+        .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
+    ResultScanner scanner = table1.getScanner(s);
+
+    int rowCount = 0;
+    for (Result result : scanner) {
+      assertNotNull(result);
+      assertTrue(!result.isEmpty());
+      Map<byte[], byte[]> values = result.getFamilyMap(FlowRunColumnFamily.INFO
+          .getBytes());
+      rowCount++;
+      // check metric1
+      byte[] q = ColumnHelper.getColumnQualifier(
+          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric1);
+      assertTrue(values.containsKey(q));
+      assertEquals(141, GenericObjectMapper.read(values.get(q)));
+
+      // check metric2
+      assertEquals(2, values.size());
+      q = ColumnHelper.getColumnQualifier(
+          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric2);
+      assertTrue(values.containsKey(q));
+      assertEquals(57, GenericObjectMapper.read(values.get(q)));
+    }
+    assertEquals(1, rowCount);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    util.shutdownMiniCluster();
+  }
+}


[06/50] [abbrv] hadoop git commit: MAPREDUCE-6370. Made the timeline service v2 test driver write event ID. Contributed by Li Lu.

Posted by gt...@apache.org.
MAPREDUCE-6370. Made the timeline service v2 test driver write event ID. Contributed by Li Lu.

(cherry picked from commit 827633ee9fee26e3e15343cbccb0b7905ae02170)


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

Branch: refs/heads/feature-YARN-2928
Commit: 557fd5e8527a51b99474db09e1f729ce3cc29906
Parents: a7ac8b2
Author: Zhijie Shen <zj...@apache.org>
Authored: Fri May 22 00:00:05 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:37:47 2016 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../test/java/org/apache/hadoop/mapred/SimpleEntityWriter.java    | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/557fd5e8/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index b5b6104..923751f 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -21,6 +21,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
 
   BUG FIXES
 
+    MAPREDUCE-6370. Made the timeline service v2 test driver write event ID.
+    (Li Lu via zjshen)
+
 Trunk (Unreleased)
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/557fd5e8/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/SimpleEntityWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/SimpleEntityWriter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/SimpleEntityWriter.java
index 4ef0a14..625c32a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/SimpleEntityWriter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/SimpleEntityWriter.java
@@ -97,6 +97,7 @@ class SimpleEntityWriter extends EntityWriter {
         entity.addInfo("PERF_TEST", payLoad);
         // add an event
         TimelineEvent event = new TimelineEvent();
+        event.setId("foo_event_id");
         event.setTimestamp(System.currentTimeMillis());
         event.addInfo("foo_event", "test");
         entity.addEvent(event);


[25/50] [abbrv] hadoop git commit: YARN-3864. Implement support for querying single app and all apps for a flow run (Varun Saxena via sjlee)

Posted by gt...@apache.org.
YARN-3864. Implement support for querying single app and all apps for a flow run (Varun Saxena via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: 2b6784ab9585dc8dea97b0c99b51a64a4fee5c8d
Parents: 0839cd1
Author: Sangjin Lee <sj...@apache.org>
Authored: Mon Oct 5 13:14:11 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:58 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../reader/TimelineReaderWebServices.java       | 206 +++++-
 .../storage/ApplicationEntityReader.java        |  64 +-
 .../storage/FlowActivityEntityReader.java       |  33 +-
 .../storage/FlowRunEntityReader.java            |   2 +-
 .../storage/GenericEntityReader.java            |  16 +-
 .../storage/TimelineEntityReader.java           |  21 +-
 .../storage/TimelineEntityReaderFactory.java    |   2 +-
 .../storage/application/ApplicationRowKey.java  |  34 +
 .../TestTimelineReaderWebServicesFlowRun.java   | 405 -----------
 ...stTimelineReaderWebServicesHBaseStorage.java | 673 +++++++++++++++++++
 11 files changed, 1000 insertions(+), 459 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b6784ab/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0d2537e..902d05e 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -118,6 +118,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4210. HBase reader throws NPE if Get returns no rows (Varun Saxena
     via vrushali)
 
+    YARN-3864. Implement support for querying single app and all apps for a
+    flow run (Varun Saxena via sjlee)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b6784ab/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 a327099..610f74c 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
@@ -286,7 +286,7 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
-        (null == req.getQueryString() ? "" :
+        (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
     UserGroupInformation callerUGI = getUser(req);
     LOG.info("Received URL " + url + " from user " + getUserName(callerUGI));
@@ -310,7 +310,7 @@ public class TimelineReaderWebServices {
           parseFieldsStr(fields, COMMA_DELIMITER));
     } catch (Exception e) {
       handleException(e, url, startTime,
-          "createdTime or modifiedTime start/end or limit or flowId");
+          "createdTime or modifiedTime start/end or limit or flowrunid");
     }
     long endTime = Time.monotonicNow();
     if (entities == null) {
@@ -360,7 +360,7 @@ public class TimelineReaderWebServices {
       @QueryParam("flowrunid") String flowRunId,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
-        (null == req.getQueryString() ? "" :
+        (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
     UserGroupInformation callerUGI = getUser(req);
     LOG.info("Received URL " + url + " from user " + getUserName(callerUGI));
@@ -420,7 +420,7 @@ public class TimelineReaderWebServices {
       @QueryParam("userid") String userId,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
-        (null == req.getQueryString() ? "" :
+        (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
     UserGroupInformation callerUGI = getUser(req);
     LOG.info("Received URL " + url + " from user " + getUserName(callerUGI));
@@ -477,7 +477,7 @@ public class TimelineReaderWebServices {
       @QueryParam("limit") String limit,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
-        (null == req.getQueryString() ? "" :
+        (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
     UserGroupInformation callerUGI = getUser(req);
     LOG.info("Received URL " + url + " from user " + getUserName(callerUGI));
@@ -502,4 +502,200 @@ public class TimelineReaderWebServices {
         " (Took " + (endTime - startTime) + " ms.)");
     return entities;
   }
+
+  /**
+   * Return a single app for given app id. Cluster ID is not provided by
+   * client so default cluster ID has to be taken.
+   */
+  @GET
+  @Path("/app/{appid}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public TimelineEntity getApp(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("appid") String appId,
+      @QueryParam("flowid") String flowId,
+      @QueryParam("flowrunid") String flowRunId,
+      @QueryParam("userid") String userId,
+      @QueryParam("fields") String fields) {
+    return getApp(req, res, null, appId, flowId, flowRunId, userId, fields);
+  }
+
+  /**
+   * Return a single app for given cluster id and app id.
+   */
+  @GET
+  @Path("/app/{clusterid}/{appid}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public TimelineEntity getApp(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("clusterid") String clusterId,
+      @PathParam("appid") String appId,
+      @QueryParam("flowid") String flowId,
+      @QueryParam("flowrunid") String flowRunId,
+      @QueryParam("userid") String userId,
+      @QueryParam("fields") String fields) {
+    String url = req.getRequestURI() +
+        (req.getQueryString() == null ? "" :
+            QUERY_STRING_SEP + req.getQueryString());
+    UserGroupInformation callerUGI = getUser(req);
+    LOG.info("Received URL " + url + " from user " + getUserName(callerUGI));
+    long startTime = Time.monotonicNow();
+    init(res);
+    TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
+    TimelineEntity entity = null;
+    try {
+      entity = timelineReaderManager.getEntity(
+          parseUser(callerUGI, userId), parseStr(clusterId),
+          parseStr(flowId), parseLongStr(flowRunId), parseStr(appId),
+          TimelineEntityType.YARN_APPLICATION.toString(), null,
+          parseFieldsStr(fields, COMMA_DELIMITER));
+    } catch (Exception e) {
+      handleException(e, url, startTime, "flowrunid");
+    }
+    long endTime = Time.monotonicNow();
+    if (entity == null) {
+      LOG.info("Processed URL " + url + " but app not found" + " (Took " +
+          (endTime - startTime) + " ms.)");
+      throw new NotFoundException("App " + appId + " not found");
+    }
+    LOG.info("Processed URL " + url +
+        " (Took " + (endTime - startTime) + " ms.)");
+    return entity;
+  }
+
+  /**
+   * Return a list of apps for given flow id and flow run id. Cluster ID is not
+   * provided by client so default cluster ID has to be taken. If number of
+   * matching apps are more than the limit, most recent apps till the limit is
+   * reached, will be returned.
+   */
+  @GET
+  @Path("/flowrunapps/{flowid}/{flowrunid}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Set<TimelineEntity> getFlowRunApps(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("flowid") String flowId,
+      @PathParam("flowrunid") String flowRunId,
+      @QueryParam("userid") String userId,
+      @QueryParam("limit") String limit,
+      @QueryParam("createdtimestart") String createdTimeStart,
+      @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("modifiedtimestart") String modifiedTimeStart,
+      @QueryParam("modifiedtimeend") String modifiedTimeEnd,
+      @QueryParam("relatesto") String relatesTo,
+      @QueryParam("isrelatedto") String isRelatedTo,
+      @QueryParam("infofilters") String infofilters,
+      @QueryParam("conffilters") String conffilters,
+      @QueryParam("metricfilters") String metricfilters,
+      @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("fields") String fields) {
+    return getEntities(req, res, null, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), userId, flowId,
+        flowRunId, limit, createdTimeStart, createdTimeEnd, modifiedTimeStart,
+        modifiedTimeEnd, relatesTo, isRelatedTo, infofilters, conffilters,
+        metricfilters, eventfilters, fields);
+  }
+
+  /**
+   * Return a list of apps for a given cluster id, flow id and flow run id. If
+   * number of matching apps are more than the limit, most recent apps till the
+   * limit is reached, will be returned.
+   */
+  @GET
+  @Path("/flowrunapps/{clusterid}/{flowid}/{flowrunid}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Set<TimelineEntity> getFlowRunApps(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("clusterid") String clusterId,
+      @PathParam("flowid") String flowId,
+      @PathParam("flowrunid") String flowRunId,
+      @QueryParam("userid") String userId,
+      @QueryParam("limit") String limit,
+      @QueryParam("createdtimestart") String createdTimeStart,
+      @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("modifiedtimestart") String modifiedTimeStart,
+      @QueryParam("modifiedtimeend") String modifiedTimeEnd,
+      @QueryParam("relatesto") String relatesTo,
+      @QueryParam("isrelatedto") String isRelatedTo,
+      @QueryParam("infofilters") String infofilters,
+      @QueryParam("conffilters") String conffilters,
+      @QueryParam("metricfilters") String metricfilters,
+      @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("fields") String fields) {
+    return getEntities(req, res, clusterId, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), userId, flowId,
+        flowRunId, limit, createdTimeStart, createdTimeEnd, modifiedTimeStart,
+        modifiedTimeEnd, relatesTo, isRelatedTo, infofilters, conffilters,
+        metricfilters, eventfilters, fields);
+  }
+
+  /**
+   * Return a list of apps for given flow id. Cluster ID is not provided by
+   * client so default cluster ID has to be taken. If number of matching apps
+   * are more than the limit, most recent apps till the limit is reached, will
+   * be returned.
+   */
+  @GET
+  @Path("/flowapps/{flowid}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Set<TimelineEntity> getFlowApps(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("flowid") String flowId,
+      @QueryParam("userid") String userId,
+      @QueryParam("limit") String limit,
+      @QueryParam("createdtimestart") String createdTimeStart,
+      @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("modifiedtimestart") String modifiedTimeStart,
+      @QueryParam("modifiedtimeend") String modifiedTimeEnd,
+      @QueryParam("relatesto") String relatesTo,
+      @QueryParam("isrelatedto") String isRelatedTo,
+      @QueryParam("infofilters") String infofilters,
+      @QueryParam("conffilters") String conffilters,
+      @QueryParam("metricfilters") String metricfilters,
+      @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("fields") String fields) {
+    return getEntities(req, res, null, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), userId, flowId,
+        null, limit, createdTimeStart, createdTimeEnd, modifiedTimeStart,
+        modifiedTimeEnd, relatesTo, isRelatedTo, infofilters, conffilters,
+        metricfilters, eventfilters, fields);
+  }
+
+  /**
+   * Return a list of apps for a given cluster id and flow id. If number of
+   * matching apps are more than the limit, most recent apps till the limit is
+   * reached, will be returned.
+   */
+  @GET
+  @Path("/flowapps/{clusterid}/{flowid}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Set<TimelineEntity> getFlowApps(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("clusterid") String clusterId,
+      @PathParam("flowid") String flowId,
+      @QueryParam("userid") String userId,
+      @QueryParam("limit") String limit,
+      @QueryParam("createdtimestart") String createdTimeStart,
+      @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("modifiedtimestart") String modifiedTimeStart,
+      @QueryParam("modifiedtimeend") String modifiedTimeEnd,
+      @QueryParam("relatesto") String relatesTo,
+      @QueryParam("isrelatedto") String isRelatedTo,
+      @QueryParam("infofilters") String infofilters,
+      @QueryParam("conffilters") String conffilters,
+      @QueryParam("metricfilters") String metricfilters,
+      @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("fields") String fields) {
+    return getEntities(req, res, clusterId, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), userId, flowId,
+        null, limit, createdTimeStart, createdTimeEnd, modifiedTimeStart,
+        modifiedTimeEnd, relatesTo, isRelatedTo, infofilters, conffilters,
+        metricfilters, eventfilters, fields);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b6784ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/ApplicationEntityReader.java
index d5b5d63..61954e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/ApplicationEntityReader.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage;
 
 import java.io.IOException;
-import java.util.Collections;
 import java.util.EnumSet;
 import java.util.Map;
 import java.util.Set;
@@ -28,6 +27,8 @@ 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.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.PageFilter;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
@@ -38,6 +39,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineReaderUtils;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Timeline entity reader for application entities that are stored in the
  * application table.
@@ -57,7 +60,7 @@ class ApplicationEntityReader extends GenericEntityReader {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, fieldsToRetrieve);
+        eventFilters, fieldsToRetrieve, true);
   }
 
   public ApplicationEntityReader(String userId, String clusterId,
@@ -86,10 +89,63 @@ class ApplicationEntityReader extends GenericEntityReader {
   }
 
   @Override
+  protected void validateParams() {
+    Preconditions.checkNotNull(userId, "userId shouldn't be null");
+    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
+    Preconditions.checkNotNull(entityType, "entityType shouldn't be null");
+    if (singleEntityRead) {
+      Preconditions.checkNotNull(appId, "appId shouldn't be null");
+    } else {
+      Preconditions.checkNotNull(flowId, "flowId shouldn't be null");
+    }
+  }
+
+  @Override
+  protected void augmentParams(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    if (singleEntityRead) {
+      if (flowId == null || flowRunId == null) {
+        FlowContext context =
+            lookupFlowContext(clusterId, appId, hbaseConf, conn);
+        flowId = context.flowId;
+        flowRunId = context.flowRunId;
+      }
+    }
+    if (fieldsToRetrieve == null) {
+      fieldsToRetrieve = EnumSet.noneOf(Field.class);
+    }
+    if (!singleEntityRead) {
+      if (limit == null || limit < 0) {
+        limit = TimelineReader.DEFAULT_LIMIT;
+      }
+      if (createdTimeBegin == null) {
+        createdTimeBegin = DEFAULT_BEGIN_TIME;
+      }
+      if (createdTimeEnd == null) {
+        createdTimeEnd = DEFAULT_END_TIME;
+      }
+      if (modifiedTimeBegin == null) {
+        modifiedTimeBegin = DEFAULT_BEGIN_TIME;
+      }
+      if (modifiedTimeEnd == null) {
+        modifiedTimeEnd = DEFAULT_END_TIME;
+      }
+    }
+  }
+
+  @Override
   protected ResultScanner getResults(Configuration hbaseConf,
       Connection conn) throws IOException {
-    throw new UnsupportedOperationException(
-        "we don't support multiple apps query");
+    Scan scan = new Scan();
+    if (flowRunId != null) {
+      scan.setRowPrefixFilter(ApplicationRowKey.
+          getRowKeyPrefix(clusterId, userId, flowId, flowRunId));
+    } else {
+      scan.setRowPrefixFilter(ApplicationRowKey.
+          getRowKeyPrefix(clusterId, userId, flowId));
+    }
+    scan.setFilter(new PageFilter(limit));
+    return table.getResultScanner(hbaseConf, conn, scan);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b6784ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.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/FlowActivityEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
index e68ca17..70a0915 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
@@ -20,9 +20,7 @@ package org.apache.hadoop.yarn.server.timelineservice.storage;
 import java.io.IOException;
 import java.util.EnumSet;
 import java.util.Map;
-import java.util.NavigableSet;
 import java.util.Set;
-import java.util.TreeSet;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
@@ -60,7 +58,7 @@ class FlowActivityEntityReader extends TimelineEntityReader {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, fieldsToRetrieve);
+        eventFilters, fieldsToRetrieve, true);
   }
 
   public FlowActivityEntityReader(String userId, String clusterId,
@@ -78,35 +76,6 @@ class FlowActivityEntityReader extends TimelineEntityReader {
     return FLOW_ACTIVITY_TABLE;
   }
 
-  /**
-   * Since this is strictly sorted by the row key, it is sufficient to collect
-   * the first results as specified by the limit.
-   */
-  @Override
-  public Set<TimelineEntity> readEntities(Configuration hbaseConf,
-      Connection conn) throws IOException {
-    validateParams();
-    augmentParams(hbaseConf, conn);
-
-    NavigableSet<TimelineEntity> entities = new TreeSet<>();
-    ResultScanner results = getResults(hbaseConf, conn);
-    try {
-      for (Result result : results) {
-        TimelineEntity entity = parseEntity(result);
-        if (entity == null) {
-          continue;
-        }
-        entities.add(entity);
-        if (entities.size() == limit) {
-          break;
-        }
-      }
-      return entities;
-    } finally {
-      results.close();
-    }
-  }
-
   @Override
   protected void validateParams() {
     Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b6784ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/FlowRunEntityReader.java
index b5d7ae5..90ce28f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/FlowRunEntityReader.java
@@ -56,7 +56,7 @@ class FlowRunEntityReader extends TimelineEntityReader {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, fieldsToRetrieve);
+        eventFilters, fieldsToRetrieve, false);
   }
 
   public FlowRunEntityReader(String userId, String clusterId,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b6784ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.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/GenericEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
index 396a02b..42079d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
@@ -61,8 +61,8 @@ class GenericEntityReader extends TimelineEntityReader {
   private static final EntityTable ENTITY_TABLE = new EntityTable();
   private static final Log LOG = LogFactory.getLog(GenericEntityReader.class);
 
-  private static final long DEFAULT_BEGIN_TIME = 0L;
-  private static final long DEFAULT_END_TIME = Long.MAX_VALUE;
+  protected static final long DEFAULT_BEGIN_TIME = 0L;
+  protected static final long DEFAULT_END_TIME = Long.MAX_VALUE;
 
   /**
    * Used to look up the flow context.
@@ -76,11 +76,11 @@ class GenericEntityReader extends TimelineEntityReader {
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String> metricFilters, Set<String> eventFilters,
-      EnumSet<Field> fieldsToRetrieve) {
+      EnumSet<Field> fieldsToRetrieve, boolean sortedKeys) {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, fieldsToRetrieve);
+        eventFilters, fieldsToRetrieve, sortedKeys);
   }
 
   public GenericEntityReader(String userId, String clusterId,
@@ -97,7 +97,7 @@ class GenericEntityReader extends TimelineEntityReader {
     return ENTITY_TABLE;
   }
 
-  private FlowContext lookupFlowContext(String clusterId, String appId,
+  protected FlowContext lookupFlowContext(String clusterId, String appId,
       Configuration hbaseConf, Connection conn) throws IOException {
     byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
     Get get = new Get(rowKey);
@@ -113,9 +113,9 @@ class GenericEntityReader extends TimelineEntityReader {
     }
   }
 
-  private static class FlowContext {
-    private final String flowId;
-    private final Long flowRunId;
+  protected static class FlowContext {
+    protected final String flowId;
+    protected final Long flowRunId;
     public FlowContext(String flowId, Long flowRunId) {
       this.flowId = flowId;
       this.flowRunId = flowRunId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b6784ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.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/TimelineEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
index 93be2db..d4a659c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
@@ -74,6 +74,14 @@ abstract class TimelineEntityReader {
   protected BaseTable<?> table;
 
   /**
+   * Specifies whether keys for this table are sorted in a manner where entities
+   * can be retrieved by created time. If true, it will be sufficient to collect
+   * the first results as specified by the limit. Otherwise all matched entities
+   * will be fetched and then limit applied.
+   */
+  private boolean sortedKeys = false;
+
+  /**
    * Instantiates a reader for multiple-entity reads.
    */
   protected TimelineEntityReader(String userId, String clusterId,
@@ -83,8 +91,9 @@ abstract class TimelineEntityReader {
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String> metricFilters, Set<String> eventFilters,
-      EnumSet<Field> fieldsToRetrieve) {
+      EnumSet<Field> fieldsToRetrieve, boolean sortedKeys) {
     this.singleEntityRead = false;
+    this.sortedKeys = sortedKeys;
     this.userId = userId;
     this.clusterId = clusterId;
     this.flowId = flowId;
@@ -162,8 +171,14 @@ abstract class TimelineEntityReader {
           continue;
         }
         entities.add(entity);
-        if (entities.size() > limit) {
-          entities.pollLast();
+        if (!sortedKeys) {
+          if (entities.size() > limit) {
+            entities.pollLast();
+          }
+        } else {
+          if (entities.size() == limit) {
+            break;
+          }
         }
       }
       return entities;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b6784ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.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/TimelineEntityReaderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.java
index 4fdef40..f5341c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.java
@@ -91,7 +91,7 @@ class TimelineEntityReaderFactory {
           appId, entityType, limit, createdTimeBegin, createdTimeEnd,
           modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
           infoFilters, configFilters, metricFilters, eventFilters,
-          fieldsToRetrieve);
+          fieldsToRetrieve, false);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b6784ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java
index e3b5a87..10e3c2e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java
@@ -61,6 +61,40 @@ public class ApplicationRowKey {
   }
 
   /**
+   * Constructs a row key prefix for the application table as follows:
+   * {@code clusterId!userName!flowId!}
+   *
+   * @param clusterId
+   * @param userId
+   * @param flowId
+   * @return byte array with the row key prefix
+   */
+  public static byte[] getRowKeyPrefix(String clusterId, String userId,
+      String flowId) {
+    byte[] first = Bytes.toBytes(
+        Separator.QUALIFIERS.joinEncoded(clusterId, userId, flowId));
+    return Separator.QUALIFIERS.join(first, new byte[0]);
+  }
+
+  /**
+   * Constructs a row key prefix for the application table as follows:
+   * {@code clusterId!userName!flowId!flowRunId!}
+   *
+   * @param clusterId
+   * @param userId
+   * @param flowId
+   * @param flowRunId
+   * @return byte array with the row key prefix
+   */
+  public static byte[] getRowKeyPrefix(String clusterId, String userId,
+      String flowId, Long flowRunId) {
+    byte[] first = Bytes.toBytes(
+        Separator.QUALIFIERS.joinEncoded(clusterId, userId, flowId));
+    byte[] second = Bytes.toBytes(TimelineWriterUtils.invert(flowRunId));
+    return Separator.QUALIFIERS.join(first, second, new byte[0]);
+  }
+
+  /**
    * Constructs a row key for the application table as follows:
    * {@code clusterId!userName!flowId!flowRunId!AppId}
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b6784ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesFlowRun.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/TestTimelineReaderWebServicesFlowRun.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesFlowRun.java
deleted file mode 100644
index e359f78..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesFlowRun.java
+++ /dev/null
@@ -1,405 +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.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.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-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.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.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.HBaseTimelineWriterImpl;
-import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator;
-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;
-
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.GenericType;
-import com.sun.jersey.api.client.ClientResponse.Status;
-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 TestTimelineReaderWebServicesFlowRun {
-  private int serverPort;
-  private TimelineReaderServer server;
-  private static HBaseTestingUtility util;
-  private static long ts = System.currentTimeMillis();
-
-  @BeforeClass
-  public static void setup() throws Exception {
-    util = new HBaseTestingUtility();
-    Configuration conf = util.getConfiguration();
-    conf.setInt("hfile.format.version", 3);
-    util.startMiniCluster();
-    TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
-    loadData();
-  }
-
-  private static void loadData() throws Exception {
-    String cluster = "cluster1";
-    String user = "user1";
-    String flow = "flow_name";
-    String flowVersion = "CF7022C10F1354";
-    Long runid = 1002345678919L;
-    Long runid1 = 1002345678920L;
-
-    TimelineEntities te = new TimelineEntities();
-    TimelineEntity entity = new TimelineEntity();
-    String id = "flowRunMetrics_test";
-    String type = TimelineEntityType.YARN_APPLICATION.toString();
-    entity.setId(id);
-    entity.setType(type);
-    Long cTime = 1425016501000L;
-    entity.setCreatedTime(cTime);
-
-    // add metrics
-    Set<TimelineMetric> metrics = new HashSet<>();
-    TimelineMetric m1 = new TimelineMetric();
-    m1.setId("MAP_SLOT_MILLIS");
-    Map<Long, Number> metricValues = new HashMap<Long, Number>();
-    metricValues.put(ts - 100000, 2);
-    metricValues.put(ts - 80000, 40);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-
-    m1 = new TimelineMetric();
-    m1.setId("HDFS_BYTES_READ");
-    metricValues = new HashMap<Long, Number>();
-    metricValues.put(ts - 100000, 31);
-    metricValues.put(ts - 80000, 57);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    entity.addMetrics(metrics);
-
-    TimelineEvent event = new TimelineEvent();
-    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    Long expTs = 1436512802000L;
-    event.setTimestamp(expTs);
-    String expKey = "foo_event";
-    Object expVal = "test";
-    event.addInfo(expKey, expVal);
-    entity.addEvent(event);
-    te.addEntity(entity);
-
-    // write another application with same metric to this flow
-    TimelineEntities te1 = new TimelineEntities();
-    TimelineEntity entity1 = new TimelineEntity();
-    id = "flowRunMetrics_test";
-    type = TimelineEntityType.YARN_APPLICATION.toString();
-    entity1.setId(id);
-    entity1.setType(type);
-    cTime = 1425016501000L;
-    entity1.setCreatedTime(cTime);
-    // add metrics
-    metrics.clear();
-    TimelineMetric m2 = new TimelineMetric();
-    m2.setId("MAP_SLOT_MILLIS");
-    metricValues = new HashMap<Long, Number>();
-    metricValues.put(ts - 100000, 5L);
-    metricValues.put(ts - 80000, 101L);
-    m2.setType(Type.TIME_SERIES);
-    m2.setValues(metricValues);
-    metrics.add(m2);
-    entity1.addMetrics(metrics);
-    te1.addEntity(entity1);
-
-    String flow2 = "flow_name2";
-    String flowVersion2 = "CF7022C10F1454";
-    Long runid2 = 2102356789046L;
-    TimelineEntities te3 = new TimelineEntities();
-    TimelineEntity entity3 = new TimelineEntity();
-    id = "flowRunMetrics_test1";
-    entity3.setId(id);
-    entity3.setType(type);
-    cTime = 1425016501030L;
-    entity3.setCreatedTime(cTime);
-    TimelineEvent event2 = new TimelineEvent();
-    event2.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    event2.setTimestamp(1436512802030L);
-    event2.addInfo("foo_event", "test");
-    entity3.addEvent(event2);
-    te3.addEntity(entity3);
-
-    HBaseTimelineWriterImpl hbi = null;
-    Configuration c1 = util.getConfiguration();
-    try {
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      String appName = "application_11111111111111_1111";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
-      appName = "application_11111111111111_2222";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
-      hbi.write(cluster, user, flow, flowVersion, runid1, appName, te);
-      appName = "application_11111111111111_2223";
-      hbi.write(cluster, user, flow2, flowVersion2, runid2, appName, te3);
-      hbi.flush();
-    } finally {
-      hbi.close();
-    }
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    util.shutdownMiniCluster();
-  }
-
-  @Before
-  public void init() throws Exception {
-    try {
-      Configuration config = util.getConfiguration();
-      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 = new String();
-      if (resp != null) {
-        msg = resp.getClientResponseStatus().toString();
-      }
-      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 TimelineMetric newMetric(String id, long ts, Number value) {
-    TimelineMetric metric = new TimelineMetric();
-    metric.setId(id);
-    metric.addValue(ts, value);
-    return metric;
-  }
-
-  private static boolean verifyMetricValues(Map<Long, Number> m1,
-      Map<Long, Number> m2) {
-    for (Map.Entry<Long, Number> entry : m1.entrySet()) {
-      if (!m2.containsKey(entry.getKey())) {
-        return false;
-      }
-      if (m2.get(entry.getKey()).equals(entry.getValue())) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  private static boolean verifyMetrics(
-      TimelineMetric m, TimelineMetric... metrics) {
-    for (TimelineMetric metric : metrics) {
-      if (!metric.equals(m)) {
-        continue;
-      }
-      if (!verifyMetricValues(metric.getValues(), m.getValues())) {
-        continue;
-      }
-      return true;
-    }
-    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().equals(status));
-  }
-
-  @Test
-  public void testGetFlowRun() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowrun/cluster1/flow_name/1002345678919?userid=user1");
-      ClientResponse resp = getResponse(client, uri);
-      FlowRunEntity entity = resp.getEntity(FlowRunEntity.class);
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
-      assertNotNull(entity);
-      assertEquals("user1@flow_name/1002345678919", entity.getId());
-      assertEquals(2, entity.getMetrics().size());
-      TimelineMetric m1 = newMetric("HDFS_BYTES_READ", ts - 80000, 57L);
-      TimelineMetric m2 = newMetric("MAP_SLOT_MILLIS", ts - 80000, 141L);
-      for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue(verifyMetrics(metric, m1, m2));
-      }
-
-      // Query without specifying cluster ID.
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowrun/flow_name/1002345678919?userid=user1");
-      resp = getResponse(client, uri);
-      entity = resp.getEntity(FlowRunEntity.class);
-      assertNotNull(entity);
-      assertEquals("user1@flow_name/1002345678919", entity.getId());
-      assertEquals(2, entity.getMetrics().size());
-      m1 = newMetric("HDFS_BYTES_READ", ts - 80000, 57L);
-      m2 = newMetric("MAP_SLOT_MILLIS", ts - 80000, 141L);
-      for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue(verifyMetrics(metric, m1, m2));
-      }
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetFlows() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flows/cluster1");
-      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));
-      }
-
-      // 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());
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-              "timeline/flows/cluster1?limit=1");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetFlowRunNotPresent() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flowrun/cluster1/flow_name/1002345678929?userid=user1");
-      verifyHttpResponse(client, uri, Status.NOT_FOUND);
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetFlowsNotPresent() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flows/cluster2");
-      ClientResponse resp = getResponse(client, uri);
-      Set<FlowActivityEntity> entities =
-          resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
-      assertNotNull(entities);
-      assertEquals(0, entities.size());
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @After
-  public void stop() throws Exception {
-    if (server != null) {
-      server.stop();
-      server = null;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b6784ab/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
new file mode 100644
index 0000000..a89d2fc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -0,0 +1,673 @@
+/**
+ * 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.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+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.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.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.HBaseTimelineWriterImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator;
+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;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.GenericType;
+import com.sun.jersey.api.client.ClientResponse.Status;
+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;
+  private static long ts = System.currentTimeMillis();
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    util = new HBaseTestingUtility();
+    Configuration conf = util.getConfiguration();
+    conf.setInt("hfile.format.version", 3);
+    util.startMiniCluster();
+    TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
+    loadData();
+  }
+
+  private static void loadData() throws Exception {
+    String cluster = "cluster1";
+    String user = "user1";
+    String flow = "flow_name";
+    String flowVersion = "CF7022C10F1354";
+    Long runid = 1002345678919L;
+    Long runid1 = 1002345678920L;
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entity = new TimelineEntity();
+    String id = "application_1111111111_1111";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    Long cTime = 1425016501000L;
+    entity.setCreatedTime(cTime);
+    entity.addConfig("cfg2", "value1");
+
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    TimelineMetric m1 = new TimelineMetric();
+    m1.setId("MAP_SLOT_MILLIS");
+    Map<Long, Number> metricValues = new HashMap<Long, Number>();
+    metricValues.put(ts - 100000, 2);
+    metricValues.put(ts - 80000, 40);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+
+    m1 = new TimelineMetric();
+    m1.setId("HDFS_BYTES_READ");
+    metricValues = new HashMap<Long, Number>();
+    metricValues.put(ts - 100000, 31);
+    metricValues.put(ts - 80000, 57);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    entity.addMetrics(metrics);
+
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    Long expTs = 1436512802000L;
+    event.setTimestamp(expTs);
+    String expKey = "foo_event";
+    Object expVal = "test";
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+    TimelineEvent event11 = new TimelineEvent();
+    event11.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+    expTs = 1436512802010L;
+    event11.setTimestamp(expTs);
+    entity.addEvent(event11);
+
+    te.addEntity(entity);
+
+    // write another application with same metric to this flow
+    TimelineEntities te1 = new TimelineEntities();
+    TimelineEntity entity1 = new TimelineEntity();
+    id = "application_1111111111_2222";
+    type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity1.setId(id);
+    entity1.setType(type);
+    cTime = 1425016501000L;
+    entity1.setCreatedTime(cTime);
+    entity1.addConfig("cfg1", "value1");
+    // add metrics
+    metrics.clear();
+    TimelineMetric m2 = new TimelineMetric();
+    m2.setId("MAP_SLOT_MILLIS");
+    metricValues = new HashMap<Long, Number>();
+    metricValues.put(ts - 100000, 5L);
+    metricValues.put(ts - 80000, 101L);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues);
+    metrics.add(m2);
+    entity1.addMetrics(metrics);
+    TimelineEvent event1 = new TimelineEvent();
+    event1.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    event1.setTimestamp(expTs);
+    event1.addInfo(expKey, expVal);
+    entity1.addEvent(event1);
+    te1.addEntity(entity1);
+
+    String flow2 = "flow_name2";
+    String flowVersion2 = "CF7022C10F1454";
+    Long runid2 = 2102356789046L;
+    TimelineEntities te3 = new TimelineEntities();
+    TimelineEntity entity3 = new TimelineEntity();
+    id = "application_11111111111111_2223";
+    entity3.setId(id);
+    entity3.setType(type);
+    cTime = 1425016501030L;
+    entity3.setCreatedTime(cTime);
+    TimelineEvent event2 = new TimelineEvent();
+    event2.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    event2.setTimestamp(1436512802030L);
+    event2.addInfo("foo_event", "test");
+    entity3.addEvent(event2);
+    te3.addEntity(entity3);
+
+    TimelineEntities te4 = new TimelineEntities();
+    TimelineEntity entity4 = new TimelineEntity();
+    id = "application_1111111111_2224";
+    entity4.setId(id);
+    entity4.setType(type);
+    cTime = 1425016501034L;
+    entity4.setCreatedTime(cTime);
+    TimelineEvent event4 = new TimelineEvent();
+    event4.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    event4.setTimestamp(1436512802037L);
+    event4.addInfo("foo_event", "test");
+    entity4.addEvent(event4);
+    te4.addEntity(entity4);
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      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.flush();
+    } finally {
+      hbi.close();
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    util.shutdownMiniCluster();
+  }
+
+  @Before
+  public void init() throws Exception {
+    try {
+      Configuration config = util.getConfiguration();
+      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 = new String();
+      if (resp != null) {
+        msg = resp.getClientResponseStatus().toString();
+      }
+      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));
+    return entity;
+  }
+
+  private static TimelineMetric newMetric(TimelineMetric.Type type,
+      String id, long ts, Number value) {
+    TimelineMetric metric = new TimelineMetric(type);
+    metric.setId(id);
+    metric.addValue(ts, value);
+    return metric;
+  }
+
+  private static boolean verifyMetricValues(Map<Long, Number> m1,
+      Map<Long, Number> m2) {
+    for (Map.Entry<Long, Number> entry : m1.entrySet()) {
+      if (!m2.containsKey(entry.getKey())) {
+        return false;
+      }
+      if (m2.get(entry.getKey()).equals(entry.getValue())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static boolean verifyMetrics(
+      TimelineMetric m, TimelineMetric... metrics) {
+    for (TimelineMetric metric : metrics) {
+      if (!metric.equals(m)) {
+        continue;
+      }
+      if (!verifyMetricValues(metric.getValues(), m.getValues())) {
+        continue;
+      }
+      return true;
+    }
+    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().equals(status));
+  }
+
+  @Test
+  public void testGetFlowRun() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowrun/cluster1/flow_name/1002345678919?userid=user1");
+      ClientResponse resp = getResponse(client, uri);
+      FlowRunEntity entity = resp.getEntity(FlowRunEntity.class);
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entity);
+      assertEquals("user1@flow_name/1002345678919", entity.getId());
+      assertEquals(2, entity.getMetrics().size());
+      TimelineMetric m1 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+          "HDFS_BYTES_READ", ts - 80000, 57L);
+      TimelineMetric m2 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+          "MAP_SLOT_MILLIS", ts - 80000, 141L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m1, m2));
+      }
+
+      // Query without specifying cluster ID.
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowrun/flow_name/1002345678919?userid=user1");
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(FlowRunEntity.class);
+      assertNotNull(entity);
+      assertEquals("user1@flow_name/1002345678919", entity.getId());
+      assertEquals(2, entity.getMetrics().size());
+      m1 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+          "HDFS_BYTES_READ", ts - 80000, 57L);
+      m2 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+          "MAP_SLOT_MILLIS", ts - 80000, 141L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m1, m2));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlows() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows/cluster1");
+      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));
+      }
+
+      // 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());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+              "timeline/flows/cluster1?limit=1");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetApp() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/app/cluster1/application_1111111111_1111?" +
+          "userid=user1&fields=ALL&flowid=flow_name&flowrunid=1002345678919");
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("application_1111111111_1111", entity.getId());
+      assertEquals(2, entity.getMetrics().size());
+      TimelineMetric m1 = newMetric(TimelineMetric.Type.TIME_SERIES,
+          "HDFS_BYTES_READ", ts - 100000, 31L);
+      m1.addValue(ts - 80000, 57L);
+      TimelineMetric m2 = newMetric(TimelineMetric.Type.TIME_SERIES,
+          "MAP_SLOT_MILLIS", ts - 100000, 2L);
+      m2.addValue(ts - 80000, 40L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m1, m2));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+              "timeline/app/application_1111111111_2222?userid=user1" +
+              "&fields=metrics&flowid=flow_name&flowrunid=1002345678919");
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("application_1111111111_2222", entity.getId());
+      assertEquals(1, entity.getMetrics().size());
+      TimelineMetric m3 = newMetric(TimelineMetric.Type.TIME_SERIES,
+         "MAP_SLOT_MILLIS", ts - 100000, 5L);
+      m2.addValue(ts - 80000, 101L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m3));
+      }
+    } finally {
+        client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetAppWithoutFlowInfo() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/app/cluster1/application_1111111111_1111?" +
+          "userid=user1&fields=ALL");
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("application_1111111111_1111", entity.getId());
+      assertEquals(2, entity.getMetrics().size());
+      TimelineMetric m1 = newMetric(TimelineMetric.Type.TIME_SERIES,
+          "HDFS_BYTES_READ", ts - 100000, 31L);
+      m1.addValue(ts - 80000, 57L);
+      TimelineMetric m2 = newMetric(TimelineMetric.Type.TIME_SERIES,
+          "MAP_SLOT_MILLIS", ts - 100000, 2L);
+      m2.addValue(ts - 80000, 40L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m1, m2));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlowRunApps() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowrunapps/cluster1/flow_name/1002345678919?" +
+          "userid=user1&fields=ALL");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue("Unexpected app in result",
+            (entity.getId().equals("application_1111111111_1111") &&
+            entity.getMetrics().size() == 2) ||
+            (entity.getId().equals("application_1111111111_2222") &&
+            entity.getMetrics().size() == 1));
+      }
+
+      // Query without specifying cluster ID.
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowrunapps/flow_name/1002345678919?userid=user1");
+      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/" +
+          "timeline/flowrunapps/flow_name/1002345678919?userid=user1&limit=1");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlowApps() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowapps/cluster1/flow_name?userid=user1&fields=ALL");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(3, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue("Unexpected app in result",
+            (entity.getId().equals("application_1111111111_1111") &&
+            entity.getMetrics().size() == 2) ||
+            (entity.getId().equals("application_1111111111_2222") &&
+            entity.getMetrics().size() == 1) ||
+            (entity.getId().equals("application_1111111111_2224") &&
+            entity.getMetrics().size() == 0));
+      }
+
+      // Query without specifying cluster ID.
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowapps/flow_name?userid=user1");
+      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/" +
+          "timeline/flowapps/flow_name?userid=user1&limit=1");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlowAppsFilters() throws Exception {
+    Client client = createClient();
+    try {
+      String entityType = TimelineEntityType.YARN_APPLICATION.toString();
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowapps/cluster1/flow_name?userid=user1&eventfilters=" +
+          ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      assertTrue("Unexpected app in result", entities.contains(
+          newEntity(entityType, "application_1111111111_1111")));
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowapps/cluster1/flow_name?userid=user1&metricfilters=" +
+          "HDFS_BYTES_READ");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      assertTrue("Unexpected app in result", entities.contains(
+          newEntity(entityType, "application_1111111111_1111")));
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowapps/cluster1/flow_name?userid=user1&conffilters=" +
+          "cfg1:value1");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      assertTrue("Unexpected app in result", entities.contains(
+          newEntity(entityType, "application_1111111111_2222")));
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlowRunNotPresent() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowrun/cluster1/flow_name/1002345678929?userid=user1");
+      verifyHttpResponse(client, uri, Status.NOT_FOUND);
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlowsNotPresent() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows/cluster2");
+      ClientResponse resp = getResponse(client, uri);
+      Set<FlowActivityEntity> entities =
+          resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetAppNotPresent() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/app/cluster1/flow_name/1002345678919/" +
+          "application_1111111111_1378?userid=user1");
+      verifyHttpResponse(client, uri, Status.NOT_FOUND);
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlowRunAppsNotPresent() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowrunapps/cluster2/flow_name/1002345678919");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlowAppsNotPresent() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowapps/cluster2/flow_name55");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @After
+  public void stop() throws Exception {
+    if (server != null) {
+      server.stop();
+      server = null;
+    }
+  }
+}


[10/50] [abbrv] hadoop git commit: YARN-4025. Deal with byte representations of Longs in writer code. Contributed by Sangjin Lee and Vrushali C.

Posted by gt...@apache.org.
YARN-4025. Deal with byte representations of Longs in writer code. Contributed by Sangjin Lee and Vrushali C.


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

Branch: refs/heads/feature-YARN-2928
Commit: 233bfc9690d1cd6cc9a872e65d9a7a9b17a3946b
Parents: acbf140
Author: Junping Du <ju...@apache.org>
Authored: Wed Aug 19 10:00:33 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:56 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../storage/HBaseTimelineReaderImpl.java        |  68 +++---
 .../storage/HBaseTimelineWriterImpl.java        |  20 +-
 .../application/ApplicationColumnPrefix.java    |  40 ++++
 .../storage/application/ApplicationTable.java   |   6 +-
 .../storage/common/ColumnHelper.java            |  99 ++++++++-
 .../storage/common/Separator.java               |  16 +-
 .../storage/common/TimelineWriterUtils.java     |   9 +-
 .../storage/entity/EntityColumnPrefix.java      |  40 ++++
 .../storage/entity/EntityTable.java             |   6 +-
 .../storage/TestHBaseTimelineWriterImpl.java    | 207 +++++++++++--------
 11 files changed, 373 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/233bfc96/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a36c288..b72c4a6 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -94,6 +94,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3045. Implement NM writing container lifecycle events to Timeline
     Service v2. (Naganarasimha G R via junping_du)
 
+    YARN-4025. Deal with byte representations of Longs in writer code.
+    (Sangjin Lee and Vrushali C via junping_du)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/233bfc96/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.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/HBaseTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
index 094f868..c514c20 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
@@ -19,12 +19,9 @@ package org.apache.hadoop.yarn.server.timelineservice.storage;
 
 
 import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.NavigableSet;
@@ -431,44 +428,51 @@ public class HBaseTimelineReaderImpl
     Map<String, Object> columns = prefix.readResults(result);
     if (isConfig) {
       for (Map.Entry<String, Object> column : columns.entrySet()) {
-        entity.addConfig(column.getKey(), column.getKey().toString());
+        entity.addConfig(column.getKey(), column.getValue().toString());
       }
     } else {
       entity.addInfo(columns);
     }
   }
 
+  /**
+   * Read events from the entity table or the application table. The column name
+   * is of the form "eventId=timestamp=infoKey" where "infoKey" may be omitted
+   * if there is no info associated with the event.
+   *
+   * See {@link EntityTable} and {@link ApplicationTable} for a more detailed
+   * schema description.
+   */
   private static void readEvents(TimelineEntity entity, Result result,
       boolean isApplication) throws IOException {
     Map<String, TimelineEvent> eventsMap = new HashMap<>();
-    Map<String, Object> eventsResult = isApplication ?
-        ApplicationColumnPrefix.EVENT.readResults(result) :
-        EntityColumnPrefix.EVENT.readResults(result);
-    for (Map.Entry<String,Object> eventResult : eventsResult.entrySet()) {
-      Collection<String> tokens =
-          Separator.VALUES.splitEncoded(eventResult.getKey());
-      if (tokens.size() != 2 && tokens.size() != 3) {
-        throw new IOException(
-            "Invalid event column name: " + eventResult.getKey());
-      }
-      Iterator<String> idItr = tokens.iterator();
-      String id = idItr.next();
-      String tsStr = idItr.next();
-      // TODO: timestamp is not correct via ser/des through UTF-8 string
-      Long ts =
-          TimelineWriterUtils.invert(Bytes.toLong(tsStr.getBytes(
-              StandardCharsets.UTF_8)));
-      String key = Separator.VALUES.joinEncoded(id, ts.toString());
-      TimelineEvent event = eventsMap.get(key);
-      if (event == null) {
-        event = new TimelineEvent();
-        event.setId(id);
-        event.setTimestamp(ts);
-        eventsMap.put(key, event);
-      }
-      if (tokens.size() == 3) {
-        String infoKey = idItr.next();
-        event.addInfo(infoKey, eventResult.getValue());
+    Map<?, Object> eventsResult = isApplication ?
+        ApplicationColumnPrefix.EVENT.
+            readResultsHavingCompoundColumnQualifiers(result) :
+        EntityColumnPrefix.EVENT.
+            readResultsHavingCompoundColumnQualifiers(result);
+    for (Map.Entry<?, Object> eventResult : eventsResult.entrySet()) {
+      byte[][] karr = (byte[][])eventResult.getKey();
+      // the column name is of the form "eventId=timestamp=infoKey"
+      if (karr.length == 3) {
+        String id = Bytes.toString(karr[0]);
+        long ts = TimelineWriterUtils.invert(Bytes.toLong(karr[1]));
+        String key = Separator.VALUES.joinEncoded(id, Long.toString(ts));
+        TimelineEvent event = eventsMap.get(key);
+        if (event == null) {
+          event = new TimelineEvent();
+          event.setId(id);
+          event.setTimestamp(ts);
+          eventsMap.put(key, event);
+        }
+        // handle empty info
+        String infoKey = karr[2].length == 0 ? null : Bytes.toString(karr[2]);
+        if (infoKey != null) {
+          event.addInfo(infoKey, eventResult.getValue());
+        }
+      } else {
+        LOG.warn("incorrectly formatted column name: it will be discarded");
+        continue;
       }
     }
     Set<TimelineEvent> eventsSet = new HashSet<>(eventsMap.values());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/233bfc96/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 96192cc..772002d 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
@@ -300,25 +300,27 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
               byte[] compoundColumnQualifierBytes =
                   Separator.VALUES.join(columnQualifierWithTsBytes,
                       null);
-              String compoundColumnQualifier =
-                  Bytes.toString(compoundColumnQualifierBytes);
-              EntityColumnPrefix.EVENT.store(rowKey, entityTable,
-                  compoundColumnQualifier, null, TimelineWriterUtils.EMPTY_BYTES);
+              if (isApplication) {
+                ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
+                    compoundColumnQualifierBytes, null,
+                      TimelineWriterUtils.EMPTY_BYTES);
+              } else {
+                EntityColumnPrefix.EVENT.store(rowKey, entityTable,
+                    compoundColumnQualifierBytes, null,
+                    TimelineWriterUtils.EMPTY_BYTES);
+              }
             } else {
               for (Map.Entry<String, Object> info : eventInfo.entrySet()) {
                 // eventId?infoKey
                 byte[] compoundColumnQualifierBytes =
                     Separator.VALUES.join(columnQualifierWithTsBytes,
                         Bytes.toBytes(info.getKey()));
-                // convert back to string to avoid additional API on store.
-                String compoundColumnQualifier =
-                    Bytes.toString(compoundColumnQualifierBytes);
                 if (isApplication) {
                   ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
-                    compoundColumnQualifier, null, info.getValue());
+                    compoundColumnQualifierBytes, null, info.getValue());
                 } else {
                   EntityColumnPrefix.EVENT.store(rowKey, entityTable,
-                    compoundColumnQualifier, null, info.getValue());
+                    compoundColumnQualifierBytes, null, info.getValue());
                 }
               } // for info: eventInfo
             }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/233bfc96/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
index cd9e845..ad1def6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
@@ -111,6 +111,31 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
    * TypedBufferedMutator, java.lang.String, java.lang.Long, java.lang.Object)
    */
   public void store(byte[] rowKey,
+      TypedBufferedMutator<ApplicationTable> tableMutator, byte[] qualifier,
+      Long timestamp, Object inputValue) throws IOException {
+
+    // Null check
+    if (qualifier == null) {
+      throw new IOException("Cannot store column with null qualifier in "
+          + tableMutator.getName().getNameAsString());
+    }
+
+    byte[] columnQualifier =
+        ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
+
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue);
+  }
+
+  /*
+   * (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<ApplicationTable> tableMutator, String qualifier,
       Long timestamp, Object inputValue) throws IOException {
 
@@ -150,6 +175,21 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
     return column.readResults(result, columnPrefixBytes);
   }
 
+  /**
+   * @param result from which to read columns
+   * @return the latest values of columns in the column family. The column
+   *         qualifier is returned as a list of parts, each part a byte[]. This
+   *         is to facilitate returning byte arrays of values that were not
+   *         Strings. If they can be treated as Strings, you should use
+   *         {@link #readResults(Result)} instead.
+   * @throws IOException
+   */
+  public Map<?, Object> readResultsHavingCompoundColumnQualifiers(Result result)
+      throws IOException {
+    return column.readResultsHavingCompoundColumnQualifiers(result,
+        columnPrefixBytes);
+  }
+
   /*
    * (non-Javadoc)
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/233bfc96/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
index d2a2cb9..a997997 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
@@ -57,12 +57,12 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBas
  * |            | infoValue                    | metricValue1 |              |
  * |            |                              | @timestamp2  |              |
  * |            | r!relatesToKey:              |              |              |
- * |            | id3?id4?id5                  |              |              |
+ * |            | id3=id4=id5                  |              |              |
  * |            |                              |              |              |
  * |            | s!isRelatedToKey:            |              |              |
- * |            | id7?id9?id6                  |              |              |
+ * |            | id7=id9=id6                  |              |              |
  * |            |                              |              |              |
- * |            | e!eventId?timestamp?infoKey: |              |              |
+ * |            | e!eventId=timestamp=infoKey: |              |              |
  * |            | eventInfoValue               |              |              |
  * |            |                              |              |              |
  * |            | flowVersion:                 |              |              |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/233bfc96/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 a902924..f1b7c58 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
@@ -24,6 +24,8 @@ import java.util.Map.Entry;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
@@ -37,6 +39,7 @@ import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
  * @param <T> refers to the table.
  */
 public class ColumnHelper<T> {
+  private static final Log LOG = LogFactory.getLog(ColumnHelper.class);
 
   private final ColumnFamily<T> columnFamily;
 
@@ -143,6 +146,7 @@ public class ColumnHelper<T> {
             .entrySet()) {
           String columnName = null;
           if (columnPrefixBytes == null) {
+            LOG.info("null prefix was specified; returning all columns");
             // Decode the spaces we encoded in the column name.
             columnName = Separator.decode(entry.getKey(), Separator.SPACE);
           } else {
@@ -181,32 +185,43 @@ public class ColumnHelper<T> {
   /**
    * @param result from which to read columns
    * @param columnPrefixBytes optional prefix to limit columns. If null all
-   *          columns are returned.
-   * @return the latest values of columns in the column family.
+   *        columns are returned.
+   * @return the latest values of columns in the column family. This assumes
+   *         that the column name parts are all Strings by default. If the
+   *         column name parts should be treated natively and not be converted
+   *         back and forth from Strings, you should use
+   *         {@link #readResultsHavingCompoundColumnQualifiers(Result, byte[])}
+   *         instead.
    * @throws IOException
    */
-  public Map<String, Object> readResults(Result result, byte[] columnPrefixBytes)
-      throws IOException {
+  public Map<String, Object> readResults(Result result,
+      byte[] columnPrefixBytes) throws IOException {
     Map<String, Object> results = new HashMap<String, Object>();
 
     if (result != null) {
       Map<byte[], byte[]> columns = result.getFamilyMap(columnFamilyBytes);
       for (Entry<byte[], byte[]> entry : columns.entrySet()) {
-        if (entry.getKey() != null && entry.getKey().length > 0) {
+        byte[] columnKey = entry.getKey();
+        if (columnKey != null && columnKey.length > 0) {
 
           String columnName = null;
           if (columnPrefixBytes == null) {
+            LOG.info("null prefix was specified; returning all columns");
             // Decode the spaces we encoded in the column name.
-            columnName = Separator.decode(entry.getKey(), Separator.SPACE);
+            columnName = Separator.decode(columnKey, Separator.SPACE);
           } else {
             // A non-null prefix means columns are actually of the form
             // prefix!columnNameRemainder
             byte[][] columnNameParts =
-                Separator.QUALIFIERS.split(entry.getKey(), 2);
+                Separator.QUALIFIERS.split(columnKey, 2);
             byte[] actualColumnPrefixBytes = columnNameParts[0];
             if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes)
                 && columnNameParts.length == 2) {
               // This is the prefix that we want
+              // if the column name is a compound qualifier
+              // with non string datatypes, the following decode will not
+              // work correctly since it considers all components to be String
+              // invoke the readResultsHavingCompoundColumnQualifiers function
               columnName = Separator.decode(columnNameParts[1]);
             }
           }
@@ -223,6 +238,56 @@ public class ColumnHelper<T> {
   }
 
   /**
+   * @param result from which to read columns
+   * @param columnPrefixBytes optional prefix to limit columns. If null all
+   *        columns are returned.
+   * @return the latest values of columns in the column family. If the column
+   *         prefix is null, the column qualifier is returned as Strings. For a
+   *         non-null column prefix bytes, the column qualifier is returned as
+   *         a list of parts, each part a byte[]. This is to facilitate
+   *         returning byte arrays of values that were not Strings.
+   * @throws IOException
+   */
+  public Map<?, Object> readResultsHavingCompoundColumnQualifiers(Result result,
+      byte[] columnPrefixBytes) throws IOException {
+    // handle the case where the column prefix is null
+    // it is the same as readResults() so simply delegate to that implementation
+    if (columnPrefixBytes == null) {
+      return readResults(result, null);
+    }
+
+    Map<byte[][], Object> results = new HashMap<byte[][], Object>();
+
+    if (result != null) {
+      Map<byte[], byte[]> columns = result.getFamilyMap(columnFamilyBytes);
+      for (Entry<byte[], byte[]> entry : columns.entrySet()) {
+        byte[] columnKey = entry.getKey();
+        if (columnKey != null && columnKey.length > 0) {
+          // A non-null prefix means columns are actually of the form
+          // prefix!columnNameRemainder
+          // with a compound column qualifier, we are presuming existence of a
+          // prefix
+          byte[][] columnNameParts = Separator.QUALIFIERS.split(columnKey, 2);
+          if (columnNameParts.length > 0) {
+            byte[] actualColumnPrefixBytes = columnNameParts[0];
+            if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes)
+                && columnNameParts.length == 2) {
+              // This is the prefix that we want
+              byte[][] columnQualifierParts =
+                  Separator.VALUES.split(columnNameParts[1]);
+              Object value = GenericObjectMapper.read(entry.getValue());
+              // we return the columnQualifier in parts since we don't know
+              // which part is of which data type
+              results.put(columnQualifierParts, value);
+            }
+          }
+        }
+      } // for entry
+    }
+    return results;
+  }
+
+  /**
    * @param columnPrefixBytes The byte representation for the column prefix.
    *          Should not contain {@link Separator#QUALIFIERS}.
    * @param qualifier for the remainder of the column. Any
@@ -247,4 +312,24 @@ public class ColumnHelper<T> {
     return columnQualifier;
   }
 
+  /**
+   * @param columnPrefixBytes The byte representation for the column prefix.
+   *          Should not contain {@link Separator#QUALIFIERS}.
+   * @param qualifier the byte representation for the remainder of the column.
+   * @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.
+   */
+  public static byte[] getColumnQualifier(byte[] columnPrefixBytes,
+      byte[] qualifier) {
+
+    if (columnPrefixBytes == null) {
+      return qualifier;
+    }
+
+    byte[] columnQualifier =
+        Separator.QUALIFIERS.join(columnPrefixBytes, qualifier);
+    return columnQualifier;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/233bfc96/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.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/Separator.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/Separator.java
index 3319419..9f91af8 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/Separator.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/Separator.java
@@ -37,7 +37,7 @@ public enum Separator {
   /**
    * separator in values, and/or compound key/column qualifier fields.
    */
-  VALUES("?", "%1$"),
+  VALUES("=", "%1$"),
 
   /**
    * separator in values, often used to avoid having these in qualifiers and
@@ -299,12 +299,22 @@ public enum Separator {
    * up to a maximum of count items. This will naturally produce copied byte
    * arrays for each of the split segments.
    * @param source to be split
-   * @param limit on how many segments are supposed to be returned. Negative
-   *          value indicates no limit on number of segments.
+   * @param limit on how many segments are supposed to be returned. A
+   *          non-positive value indicates no limit on number of segments.
    * @return source split by this separator.
    */
   public byte[][] split(byte[] source, int limit) {
     return TimelineWriterUtils.split(source, this.bytes, limit);
   }
 
+  /**
+   * Splits the source array into multiple array segments using this separator,
+   * as many times as splits are found. This will naturally produce copied byte
+   * arrays for each of the split segments.
+   * @param source to be split
+   * @return source split by this separator.
+   */
+  public byte[][] split(byte[] source) {
+    return TimelineWriterUtils.split(source, this.bytes);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/233bfc96/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineWriterUtils.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/TimelineWriterUtils.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/TimelineWriterUtils.java
index c957bf5..58bdedc7e 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/TimelineWriterUtils.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/TimelineWriterUtils.java
@@ -33,6 +33,9 @@ public class TimelineWriterUtils {
   /** empty bytes */
   public static final byte[] EMPTY_BYTES = new byte[0];
 
+  /** indicator for no limits for splitting */
+  public static final int NO_LIMIT_SPLIT = -1;
+
   /**
    * Splits the source array into multiple array segments using the given
    * separator, up to a maximum of count items. This will naturally produce
@@ -45,7 +48,7 @@ public class TimelineWriterUtils {
    * @return byte[] array after splitting the source
    */
   public static byte[][] split(byte[] source, byte[] separator) {
-    return split(source, separator, -1);
+    return split(source, separator, NO_LIMIT_SPLIT);
   }
 
   /**
@@ -57,7 +60,7 @@ public class TimelineWriterUtils {
    *
    * @param source
    * @param separator
-   * @param limit a negative value indicates no limit on number of segments.
+   * @param limit a non-positive value indicates no limit on number of segments.
    * @return byte[][] after splitting the input source
    */
   public static byte[][] split(byte[] source, byte[] separator, int limit) {
@@ -81,7 +84,7 @@ public class TimelineWriterUtils {
    * separator byte array.
    */
   public static List<Range> splitRanges(byte[] source, byte[] separator) {
-    return splitRanges(source, separator, -1);
+    return splitRanges(source, separator, NO_LIMIT_SPLIT);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/233bfc96/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
index c8485c0..75ff742 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
@@ -131,6 +131,31 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    *
    * @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<EntityTable> tableMutator, byte[] qualifier,
+      Long timestamp, Object inputValue) throws IOException {
+
+    // Null check
+    if (qualifier == null) {
+      throw new IOException("Cannot store column with null qualifier in "
+          + tableMutator.getName().getNameAsString());
+    }
+
+    byte[] columnQualifier =
+        ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
+
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue);
+  }
+
+  /*
+   * (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 {
@@ -150,6 +175,21 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
     return column.readResults(result, columnPrefixBytes);
   }
 
+  /**
+   * @param result from which to read columns
+   * @return the latest values of columns in the column family. The column
+   *         qualifier is returned as a list of parts, each part a byte[]. This
+   *         is to facilitate returning byte arrays of values that were not
+   *         Strings. If they can be treated as Strings, you should use
+   *         {@link #readResults(Result)} instead.
+   * @throws IOException
+   */
+  public Map<?, Object> readResultsHavingCompoundColumnQualifiers(Result result)
+          throws IOException {
+    return column.readResultsHavingCompoundColumnQualifiers(result,
+        columnPrefixBytes);
+  }
+
   /*
    * (non-Javadoc)
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/233bfc96/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.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/entity/EntityTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
index 9a8bd8c..96773b2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
@@ -58,12 +58,12 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBas
  * |            | infoValue                    |              |              |
  * |            |                              |              |              |
  * |            | r!relatesToKey:              |              |              |
- * |            | id3?id4?id5                  |              |              |
+ * |            | id3=id4=id5                  |              |              |
  * |            |                              |              |              |
  * |            | s!isRelatedToKey             |              |              |
- * |            | id7?id9?id6                  |              |              |
+ * |            | id7=id9=id6                  |              |              |
  * |            |                              |              |              |
- * |            | e!eventId?timestamp?infoKey: |              |              |
+ * |            | e!eventId=timestamp=infoKey: |              |              |
  * |            | eventInfoValue               |              |              |
  * |            |                              |              |              |
  * |            | flowVersion:                 |              |              |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/233bfc96/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.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/TestHBaseTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
index 95f88d1..2875e01 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
@@ -27,8 +27,8 @@ import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.NavigableMap;
+import java.util.NavigableSet;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
@@ -44,6 +44,7 @@ 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;
 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.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
@@ -75,7 +76,7 @@ import org.junit.Test;
  * even if other records exist in the table. Use a different cluster name if
  * you add a new test.
  */
-public class TestHBaseTimelineWriterImpl {
+public class TestHBaseTimelineStorage {
 
   private static HBaseTestingUtility util;
 
@@ -101,8 +102,8 @@ public class TestHBaseTimelineWriterImpl {
     ApplicationEntity entity = new ApplicationEntity();
     String id = "hello";
     entity.setId(id);
-    Long cTime = 1425016501000L;
-    Long mTime = 1425026901000L;
+    long cTime = 1425016501000L;
+    long mTime = 1425026901000L;
     entity.setCreatedTime(cTime);
     entity.setModifiedTime(mTime);
 
@@ -197,19 +198,16 @@ public class TestHBaseTimelineWriterImpl {
 
       Number val =
           (Number) ApplicationColumn.CREATED_TIME.readResult(result);
-      Long cTime1 = val.longValue();
+      long cTime1 = val.longValue();
       assertEquals(cTime1, cTime);
 
       val = (Number) ApplicationColumn.MODIFIED_TIME.readResult(result);
-      Long mTime1 = val.longValue();
+      long mTime1 = val.longValue();
       assertEquals(mTime1, mTime);
 
       Map<String, Object> infoColumns =
           ApplicationColumnPrefix.INFO.readResults(result);
-      assertEquals(infoMap.size(), infoColumns.size());
-      for (String infoItem : infoMap.keySet()) {
-        assertEquals(infoMap.get(infoItem), infoColumns.get(infoItem));
-      }
+      assertEquals(infoMap, infoColumns);
 
       // Remember isRelatedTo is of type Map<String, Set<String>>
       for (String isRelatedToKey : isRelatedTo.keySet()) {
@@ -245,27 +243,15 @@ public class TestHBaseTimelineWriterImpl {
       // Configuration
       Map<String, Object> configColumns =
           ApplicationColumnPrefix.CONFIG.readResults(result);
-      assertEquals(conf.size(), configColumns.size());
-      for (String configItem : conf.keySet()) {
-        assertEquals(conf.get(configItem), configColumns.get(configItem));
-      }
+      assertEquals(conf, configColumns);
 
       NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
           ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result);
 
       NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
-      // We got metrics back
-      assertNotNull(metricMap);
-      // Same number of metrics as we wrote
-      assertEquals(metricValues.entrySet().size(), metricMap.entrySet().size());
-
-      // Iterate over original metrics and confirm that they are present
-      // here.
-      for (Entry<Long, Number> metricEntry : metricValues.entrySet()) {
-        assertEquals(metricEntry.getValue(),
-            metricMap.get(metricEntry.getKey()));
-      }
+      assertEquals(metricValues, metricMap);
 
+      // read the timeline entity using the reader this time
       TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, id,
           entity.getType(), entity.getId(),
           EnumSet.of(TimelineReader.Field.ALL));
@@ -274,6 +260,31 @@ public class TestHBaseTimelineWriterImpl {
           null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
       assertNotNull(e1);
       assertEquals(1, es1.size());
+
+      // verify attributes
+      assertEquals(id, e1.getId());
+      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
+          e1.getType());
+      assertEquals(cTime, e1.getCreatedTime());
+      assertEquals(mTime, e1.getModifiedTime());
+      Map<String, Object> infoMap2 = e1.getInfo();
+      assertEquals(infoMap, infoMap2);
+
+      Map<String, Set<String>> isRelatedTo2 = e1.getIsRelatedToEntities();
+      assertEquals(isRelatedTo, isRelatedTo2);
+
+      Map<String, Set<String>> relatesTo2 = e1.getRelatesToEntities();
+      assertEquals(relatesTo, relatesTo2);
+
+      Map<String, String> conf2 = e1.getConfigs();
+      assertEquals(conf, conf2);
+
+      Set<TimelineMetric> metrics2 = e1.getMetrics();
+      assertEquals(metrics, metrics2);
+      for (TimelineMetric metric2 : metrics2) {
+        Map<Long, Number> metricValues2 = metric2.getValues();
+        assertEquals(metricValues, metricValues2);
+      }
     } finally {
       if (hbi != null) {
         hbi.stop();
@@ -294,8 +305,8 @@ public class TestHBaseTimelineWriterImpl {
     String type = "world";
     entity.setId(id);
     entity.setType(type);
-    Long cTime = 1425016501000L;
-    Long mTime = 1425026901000L;
+    long cTime = 1425016501000L;
+    long mTime = 1425026901000L;
     entity.setCreatedTime(cTime);
     entity.setModifiedTime(mTime);
 
@@ -396,20 +407,16 @@ public class TestHBaseTimelineWriterImpl {
           assertEquals(type, type1);
 
           Number val = (Number) EntityColumn.CREATED_TIME.readResult(result);
-          Long cTime1 = val.longValue();
+          long cTime1 = val.longValue();
           assertEquals(cTime1, cTime);
 
           val = (Number) EntityColumn.MODIFIED_TIME.readResult(result);
-          Long mTime1 = val.longValue();
+          long mTime1 = val.longValue();
           assertEquals(mTime1, mTime);
 
           Map<String, Object> infoColumns =
               EntityColumnPrefix.INFO.readResults(result);
-          assertEquals(infoMap.size(), infoColumns.size());
-          for (String infoItem : infoMap.keySet()) {
-            assertEquals(infoMap.get(infoItem),
-                infoColumns.get(infoItem));
-          }
+          assertEquals(infoMap, infoColumns);
 
           // Remember isRelatedTo is of type Map<String, Set<String>>
           for (String isRelatedToKey : isRelatedTo.keySet()) {
@@ -447,32 +454,19 @@ public class TestHBaseTimelineWriterImpl {
           // Configuration
           Map<String, Object> configColumns =
               EntityColumnPrefix.CONFIG.readResults(result);
-          assertEquals(conf.size(), configColumns.size());
-          for (String configItem : conf.keySet()) {
-            assertEquals(conf.get(configItem), configColumns.get(configItem));
-          }
+          assertEquals(conf, configColumns);
 
           NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
               EntityColumnPrefix.METRIC.readResultsWithTimestamps(result);
 
           NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
-          // We got metrics back
-          assertNotNull(metricMap);
-          // Same number of metrics as we wrote
-          assertEquals(metricValues.entrySet().size(), metricMap.entrySet()
-              .size());
-
-          // Iterate over original metrics and confirm that they are present
-          // here.
-          for (Entry<Long, Number> metricEntry : metricValues.entrySet()) {
-            assertEquals(metricEntry.getValue(),
-                metricMap.get(metricEntry.getKey()));
-          }
+          assertEquals(metricValues, metricMap);
         }
       }
       assertEquals(1, rowCount);
       assertEquals(17, colCount);
 
+      // read the timeline entity using the reader this time
       TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
           entity.getType(), entity.getId(),
           EnumSet.of(TimelineReader.Field.ALL));
@@ -481,6 +475,30 @@ public class TestHBaseTimelineWriterImpl {
           null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
       assertNotNull(e1);
       assertEquals(1, es1.size());
+
+      // verify attributes
+      assertEquals(id, e1.getId());
+      assertEquals(type, e1.getType());
+      assertEquals(cTime, e1.getCreatedTime());
+      assertEquals(mTime, e1.getModifiedTime());
+      Map<String, Object> infoMap2 = e1.getInfo();
+      assertEquals(infoMap, infoMap2);
+
+      Map<String, Set<String>> isRelatedTo2 = e1.getIsRelatedToEntities();
+      assertEquals(isRelatedTo, isRelatedTo2);
+
+      Map<String, Set<String>> relatesTo2 = e1.getRelatesToEntities();
+      assertEquals(relatesTo, relatesTo2);
+
+      Map<String, String> conf2 = e1.getConfigs();
+      assertEquals(conf, conf2);
+
+      Set<TimelineMetric> metrics2 = e1.getMetrics();
+      assertEquals(metrics, metrics2);
+      for (TimelineMetric metric2 : metrics2) {
+        Map<Long, Number> metricValues2 = metric2.getValues();
+        assertEquals(metricValues, metricValues2);
+      }
     } finally {
       if (hbi != null) {
         hbi.stop();
@@ -494,9 +512,9 @@ public class TestHBaseTimelineWriterImpl {
   }
 
   private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user,
-      String flow, Long runid, String appName, TimelineEntity te) {
+      String flow, long runid, String appName, TimelineEntity te) {
 
-    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey, -1);
+    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
 
     assertTrue(rowKeyComponents.length == 7);
     assertEquals(user, Bytes.toString(rowKeyComponents[0]));
@@ -511,9 +529,9 @@ public class TestHBaseTimelineWriterImpl {
   }
 
   private boolean isApplicationRowKeyCorrect(byte[] rowKey, String cluster,
-      String user, String flow, Long runid, String appName) {
+      String user, String flow, long runid, String appName) {
 
-    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey, -1);
+    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
 
     assertTrue(rowKeyComponents.length == 5);
     assertEquals(cluster, Bytes.toString(rowKeyComponents[0]));
@@ -530,7 +548,7 @@ public class TestHBaseTimelineWriterImpl {
     TimelineEvent event = new TimelineEvent();
     String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE;
     event.setId(eventId);
-    Long expTs = 1436512802000L;
+    long expTs = 1436512802000L;
     event.setTimestamp(expTs);
     String expKey = "foo_event";
     Object expVal = "test";
@@ -577,24 +595,25 @@ public class TestHBaseTimelineWriterImpl {
       assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
           appName));
 
-      Map<String, Object> eventsResult =
-          ApplicationColumnPrefix.EVENT.readResults(result);
+      Map<?, Object> eventsResult =
+          ApplicationColumnPrefix.EVENT.
+              readResultsHavingCompoundColumnQualifiers(result);
       // there should be only one event
       assertEquals(1, eventsResult.size());
-      // key name for the event
-      byte[] compoundColumnQualifierBytes =
-          Separator.VALUES.join(Bytes.toBytes(eventId),
-              Bytes.toBytes(TimelineWriterUtils.invert(expTs)),
-              Bytes.toBytes(expKey));
-      String valueKey = Bytes.toString(compoundColumnQualifierBytes);
-      for (Map.Entry<String, Object> e : eventsResult.entrySet()) {
-        // the value key must match
-        assertEquals(valueKey, e.getKey());
+      for (Map.Entry<?, Object> e : eventsResult.entrySet()) {
+        // the qualifier is a compound key
+        // hence match individual values
+        byte[][] karr = (byte[][])e.getKey();
+        assertEquals(3, karr.length);
+        assertEquals(eventId, Bytes.toString(karr[0]));
+        assertEquals(TimelineWriterUtils.invert(expTs), Bytes.toLong(karr[1]));
+        assertEquals(expKey, Bytes.toString(karr[2]));
         Object value = e.getValue();
         // there should be only one timestamp and value
         assertEquals(expVal, value.toString());
       }
 
+      // read the timeline entity using the reader this time
       TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
           entity.getType(), entity.getId(),
           EnumSet.of(TimelineReader.Field.ALL));
@@ -613,6 +632,21 @@ public class TestHBaseTimelineWriterImpl {
       assertEquals(1, es1.size());
       assertEquals(1, es2.size());
       assertEquals(es1, es2);
+
+      // check the events
+      NavigableSet<TimelineEvent> events = e1.getEvents();
+      // there should be only one event
+      assertEquals(1, events.size());
+      for (TimelineEvent e : events) {
+        assertEquals(eventId, e.getId());
+        assertEquals(expTs, e.getTimestamp());
+        Map<String,Object> info = e.getInfo();
+        assertEquals(1, info.size());
+        for (Map.Entry<String, Object> infoEntry : info.entrySet()) {
+          assertEquals(expKey, infoEntry.getKey());
+          assertEquals(expVal, infoEntry.getValue());
+        }
+      }
     } finally {
       if (hbi != null) {
         hbi.stop();
@@ -630,7 +664,7 @@ public class TestHBaseTimelineWriterImpl {
     TimelineEvent event = new TimelineEvent();
     String eventId = "foo_event_id";
     event.setId(eventId);
-    Long expTs = 1436512802000L;
+    long expTs = 1436512802000L;
     event.setTimestamp(expTs);
 
     final TimelineEntity entity = new TimelineEntity();
@@ -678,22 +712,21 @@ public class TestHBaseTimelineWriterImpl {
           assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
               entity));
 
-          Map<String, Object> eventsResult =
-              EntityColumnPrefix.EVENT.readResults(result);
+          Map<?, Object> eventsResult =
+              EntityColumnPrefix.EVENT.
+                  readResultsHavingCompoundColumnQualifiers(result);
           // there should be only one event
           assertEquals(1, eventsResult.size());
-          // key name for the event
-          byte[] compoundColumnQualifierWithTsBytes =
-              Separator.VALUES.join(Bytes.toBytes(eventId),
-                  Bytes.toBytes(TimelineWriterUtils.invert(expTs)));
-          byte[] compoundColumnQualifierBytes =
-              Separator.VALUES.join(compoundColumnQualifierWithTsBytes,
-                  null);
-          String valueKey = Bytes.toString(compoundColumnQualifierBytes);
-          for (Map.Entry<String, Object> e :
-              eventsResult.entrySet()) {
-            // the column qualifier key must match
-            assertEquals(valueKey, e.getKey());
+          for (Map.Entry<?, Object> e : eventsResult.entrySet()) {
+            // the qualifier is a compound key
+            // hence match individual values
+            byte[][] karr = (byte[][])e.getKey();
+            assertEquals(3, karr.length);
+            assertEquals(eventId, Bytes.toString(karr[0]));
+            assertEquals(TimelineWriterUtils.invert(expTs),
+                Bytes.toLong(karr[1]));
+            // key must be empty
+            assertEquals(0, karr[2].length);
             Object value = e.getValue();
             // value should be empty
             assertEquals("", value.toString());
@@ -702,6 +735,7 @@ public class TestHBaseTimelineWriterImpl {
       }
       assertEquals(1, rowCount);
 
+      // read the timeline entity using the reader this time
       TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
           entity.getType(), entity.getId(),
           EnumSet.of(TimelineReader.Field.ALL));
@@ -710,6 +744,17 @@ public class TestHBaseTimelineWriterImpl {
           null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
       assertNotNull(e1);
       assertEquals(1, es1.size());
+
+      // check the events
+      NavigableSet<TimelineEvent> events = e1.getEvents();
+      // there should be only one event
+      assertEquals(1, events.size());
+      for (TimelineEvent e : events) {
+        assertEquals(eventId, e.getId());
+        assertEquals(expTs, e.getTimestamp());
+        Map<String,Object> info = e.getInfo();
+        assertTrue(info == null || info.isEmpty());
+      }
     } finally {
       hbi.stop();
       hbi.close();


[11/50] [abbrv] hadoop git commit: YARN-3045. Implement NM writing container lifecycle events to Timeline Service v2. Contributed by Naganarasimha G R.

Posted by gt...@apache.org.
YARN-3045. Implement NM writing container lifecycle events to Timeline Service v2. Contributed by Naganarasimha G R.


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

Branch: refs/heads/feature-YARN-2928
Commit: acbf140a929de268d8adb42696ba1a7fc4b0ed49
Parents: a87a00e
Author: Junping Du <ju...@apache.org>
Authored: Tue Aug 18 04:31:45 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:56 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../dev-support/findbugs-exclude.xml            |  10 +
 .../distributedshell/TestDistributedShell.java  |  28 +-
 .../hadoop/yarn/server/nodemanager/Context.java |   5 +
 .../yarn/server/nodemanager/NodeManager.java    |  13 +
 .../containermanager/ContainerManagerImpl.java  |  46 ++-
 .../ApplicationContainerFinishedEvent.java      |  17 +-
 .../containermanager/container/Container.java   |   3 +
 .../container/ContainerImpl.java                |  30 +-
 .../monitor/ContainersMonitorImpl.java          | 107 +-----
 .../timelineservice/NMTimelineEvent.java        |  31 ++
 .../timelineservice/NMTimelineEventType.java    |  24 ++
 .../timelineservice/NMTimelinePublisher.java    | 376 +++++++++++++++++++
 .../nodemanager/TestNodeStatusUpdater.java      |  24 +-
 .../amrmproxy/BaseAMRMProxyTest.java            |   9 +
 .../containermanager/TestAuxServices.java       |   4 +-
 .../TestContainerManagerRecovery.java           |   8 +
 .../application/TestApplication.java            |   7 +-
 .../container/TestContainer.java                |   2 +-
 .../nodemanager/webapp/MockContainer.java       |   6 +
 .../nodemanager/webapp/TestNMWebServer.java     |   3 +-
 .../PerNodeTimelineCollectorsAuxService.java    |  16 +-
 ...TestPerNodeTimelineCollectorsAuxService.java |   9 +
 23 files changed, 646 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 42da97b..a36c288 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -91,6 +91,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3904. Refactor timelineservice.storage to add support to online and
     offline aggregation writers (Li Lu via sjlee)
 
+    YARN-3045. Implement NM writing container lifecycle events to Timeline
+    Service v2. (Naganarasimha G R via junping_du)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index d36f245..68ed723 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -115,6 +115,16 @@
     <Bug pattern="BC_UNCONFIRMED_CAST" />
   </Match>
 
+  <!-- Object cast is based on the event type -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.metrics.AbstractTimelineServicePublisher" />
+     <Bug pattern="BC_UNCONFIRMED_CAST" />
+  </Match>
+
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher$ApplicationEventHandler" />
+     <Bug pattern="BC_UNCONFIRMED_CAST" />
+  </Match>
 
   <!-- Ignore intentional switch fallthroughs -->
   <Match>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index 8eac50f..fe817c3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
@@ -211,7 +212,7 @@ public class TestDistributedShell {
     testDSShell(false, "v2", false);
   }
 
-  public void testDSShell(boolean haveDomain, String timelineVersion,
+  private void testDSShell(boolean haveDomain, String timelineVersion,
       boolean defaultFlow)
       throws Exception {
     String[] args = {
@@ -412,9 +413,32 @@ public class TestDistributedShell {
           "container_" + appId.getClusterTimestamp() + "_000" + appId.getId()
               + "_01_000001"
               + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
-      verifyEntityTypeFileExists(basePath,
+      File containerEntityFile = verifyEntityTypeFileExists(basePath,
           TimelineEntityType.YARN_CONTAINER.toString(),
           containerMetricsTimestampFileName);
+      Assert.assertEquals(
+          "Container created event needs to be published atleast once",
+          1,
+          getNumOfStringOccurences(containerEntityFile,
+              ContainerMetricsConstants.CREATED_EVENT_TYPE));
+
+      // to avoid race condition of testcase, atleast check 4 times with sleep
+      // of 500ms
+      long numOfContainerFinishedOccurences = 0;
+      for (int i = 0; i < 4; i++) {
+        numOfContainerFinishedOccurences =
+            getNumOfStringOccurences(containerEntityFile,
+                ContainerMetricsConstants.FINISHED_EVENT_TYPE);
+        if (numOfContainerFinishedOccurences > 0) {
+          break;
+        } else {
+          Thread.sleep(500l);
+        }
+      }
+      Assert.assertEquals(
+          "Container finished event needs to be published atleast once",
+          1,
+          numOfContainerFinishedOccurences);
 
       // Verify RM posting Application life cycle Events are getting published
       String appMetricsTimestampFileName =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.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/Context.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
index 924860b..0b378a1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 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;
 
 /**
@@ -96,4 +97,8 @@ public interface Context {
 
   ConcurrentLinkedQueue<LogAggregationReport>
       getLogAggregationStatusForApps();
+
+  void setNMTimelinePublisher(NMTimelinePublisher nmMetricsPublisher);
+
+  NMTimelinePublisher getNMTimelinePublisher();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.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/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index e871d73..57e649e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -72,6 +72,7 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreServic
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
+import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
 import org.apache.hadoop.yarn.server.nodemanager.webapp.WebServer;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 
@@ -474,6 +475,8 @@ public class NodeManager extends CompositeService
     private final ConcurrentLinkedQueue<LogAggregationReport>
         logAggregationReportForApps;
 
+    private NMTimelinePublisher nmTimelinePublisher;
+
     public NMContext(NMContainerTokenSecretManager containerTokenSecretManager,
         NMTokenSecretManagerInNM nmTokenSecretManager,
         LocalDirsHandlerService dirsHandler, ApplicationACLsManager aclsManager,
@@ -616,6 +619,16 @@ public class NodeManager extends CompositeService
         Map<ApplicationId, String> newRegisteredCollectors) {
       this.registeredCollectors.putAll(newRegisteredCollectors);
     }
+
+    @Override
+    public void setNMTimelinePublisher(NMTimelinePublisher nmMetricsPublisher) {
+      this.nmTimelinePublisher = nmMetricsPublisher;
+    }
+
+    @Override
+    public NMTimelinePublisher getNMTimelinePublisher() {
+      return nmTimelinePublisher;
+    }
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/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 da79446..12aab79 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
@@ -125,6 +125,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.SignalContainersLauncherEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.sharedcache.SharedCacheUploadEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.sharedcache.SharedCacheUploadService;
@@ -142,6 +143,7 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.Re
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerStatus;
 import org.apache.hadoop.yarn.server.nodemanager.security.authorize.NMPolicyProvider;
+import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
 import org.apache.hadoop.yarn.util.resource.Resources;
@@ -188,6 +190,8 @@ public class ContainerManagerImpl extends CompositeService implements
 
   private long waitForContainersOnShutdownMillis;
 
+  private final NMTimelinePublisher nmMetricsPublisher;
+
   public ContainerManagerImpl(Context context, ContainerExecutor exec,
       DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater,
       NodeManagerMetrics metrics, LocalDirsHandlerService dirsHandler) {
@@ -214,6 +218,8 @@ public class ContainerManagerImpl extends CompositeService implements
     auxiliaryServices.registerServiceListener(this);
     addService(auxiliaryServices);
 
+    nmMetricsPublisher = createNMTimelinePublisher(context);
+    context.setNMTimelinePublisher(nmMetricsPublisher);
     this.containersMonitor =
         new ContainersMonitorImpl(exec, dispatcher, this.context);
     addService(this.containersMonitor);
@@ -222,13 +228,16 @@ public class ContainerManagerImpl extends CompositeService implements
         new ContainerEventDispatcher());
     dispatcher.register(ApplicationEventType.class,
         new ApplicationEventDispatcher());
-    dispatcher.register(LocalizationEventType.class, rsrcLocalizationSrvc);
+    dispatcher.register(LocalizationEventType.class,
+        new LocalizationEventHandlerWrapper(rsrcLocalizationSrvc,
+            nmMetricsPublisher));
     dispatcher.register(AuxServicesEventType.class, auxiliaryServices);
     dispatcher.register(ContainersMonitorEventType.class, containersMonitor);
     dispatcher.register(ContainersLauncherEventType.class, containersLauncher);
     
     addService(dispatcher);
 
+
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     this.readLock = lock.readLock();
     this.writeLock = lock.writeLock();
@@ -351,7 +360,7 @@ public class ContainerManagerImpl extends CompositeService implements
       Container container = new ContainerImpl(getConfig(), dispatcher,
           context.getNMStateStore(), req.getContainerLaunchContext(),
           credentials, metrics, token, rcs.getStatus(), rcs.getExitCode(),
-          rcs.getDiagnostics(), rcs.getKilled(), rcs.getCapability());
+          rcs.getDiagnostics(), rcs.getKilled(), rcs.getCapability(), context);
       context.getContainers().put(containerId, container);
       dispatcher.getEventHandler().handle(
           new ApplicationContainerInitEvent(container));
@@ -413,6 +422,13 @@ public class ContainerManagerImpl extends CompositeService implements
     return new SharedCacheUploadService();
   }
 
+  @VisibleForTesting
+  protected NMTimelinePublisher createNMTimelinePublisher(Context context) {
+    NMTimelinePublisher nmTimelinePublisherLocal = new NMTimelinePublisher(context);
+    addIfService(nmTimelinePublisherLocal);
+    return nmTimelinePublisherLocal;
+  }
+
   protected ContainersLauncher createContainersLauncher(Context context,
       ContainerExecutor exec) {
     return new ContainersLauncher(context, this.dispatcher, exec, dirsHandler, this);
@@ -910,7 +926,7 @@ public class ContainerManagerImpl extends CompositeService implements
     Container container =
         new ContainerImpl(getConfig(), this.dispatcher,
             context.getNMStateStore(), launchContext,
-          credentials, metrics, containerTokenIdentifier);
+          credentials, metrics, containerTokenIdentifier, context);
     ApplicationId applicationID =
         containerId.getApplicationAttemptId().getApplicationId();
     if (context.getContainers().putIfAbsent(containerId, container) != null) {
@@ -952,9 +968,9 @@ public class ContainerManagerImpl extends CompositeService implements
               logAggregationContext));
         }
 
-        this.context.getNMStateStore().storeContainer(containerId, request);
         dispatcher.getEventHandler().handle(
           new ApplicationContainerInitEvent(container));
+        this.context.getNMStateStore().storeContainer(containerId, request);
 
         this.context.getContainerTokenSecretManager().startContainerSuccessful(
           containerTokenIdentifier);
@@ -1291,6 +1307,7 @@ public class ContainerManagerImpl extends CompositeService implements
       Container c = containers.get(event.getContainerID());
       if (c != null) {
         c.handle(event);
+        nmMetricsPublisher.publishContainerEvent(event);
       } else {
         LOG.warn("Event " + event + " sent to absent container " +
             event.getContainerID());
@@ -1299,7 +1316,6 @@ public class ContainerManagerImpl extends CompositeService implements
   }
 
   class ApplicationEventDispatcher implements EventHandler<ApplicationEvent> {
-
     @Override
     public void handle(ApplicationEvent event) {
       Application app =
@@ -1307,6 +1323,7 @@ public class ContainerManagerImpl extends CompositeService implements
               event.getApplicationID());
       if (app != null) {
         app.handle(event);
+        nmMetricsPublisher.publishApplicationEvent(event);
       } else {
         LOG.warn("Event " + event + " sent to absent application "
             + event.getApplicationID());
@@ -1314,6 +1331,25 @@ public class ContainerManagerImpl extends CompositeService implements
     }
   }
 
+  private static final class LocalizationEventHandlerWrapper implements
+      EventHandler<LocalizationEvent> {
+
+    private EventHandler<LocalizationEvent> origLocalizationEventHandler;
+    private NMTimelinePublisher timelinePublisher;
+
+    LocalizationEventHandlerWrapper(EventHandler<LocalizationEvent> handler,
+        NMTimelinePublisher publisher) {
+      this.origLocalizationEventHandler = handler;
+      this.timelinePublisher = publisher;
+    }
+
+    @Override
+    public void handle(LocalizationEvent event) {
+      origLocalizationEventHandler.handle(event);
+      timelinePublisher.publishLocalizationEvent(event);
+    }
+  }
+
   @SuppressWarnings("unchecked")
   @Override
   public void handle(ContainerManagerEvent event) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/ApplicationContainerFinishedEvent.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/ApplicationContainerFinishedEvent.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/ApplicationContainerFinishedEvent.java
index 6b8007f..9cd34cc 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/ApplicationContainerFinishedEvent.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/ApplicationContainerFinishedEvent.java
@@ -19,18 +19,23 @@
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.application;
 
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 
 public class ApplicationContainerFinishedEvent extends ApplicationEvent {
-  private ContainerId containerID;
+  private ContainerStatus containerStatus;
 
-  public ApplicationContainerFinishedEvent(
-      ContainerId containerID) {
-    super(containerID.getApplicationAttemptId().getApplicationId(),
+  public ApplicationContainerFinishedEvent(ContainerStatus containerStatus) {
+    super(containerStatus.getContainerId().getApplicationAttemptId().getApplicationId(),
         ApplicationEventType.APPLICATION_CONTAINER_FINISHED);
-    this.containerID = containerID;
+    this.containerStatus = containerStatus;
   }
 
   public ContainerId getContainerID() {
-    return this.containerID;
+    return containerStatus.getContainerId();
   }
+
+  public ContainerStatus getContainerStatus() {
+    return containerStatus;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.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/container/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
index 1d2ec56..aac88fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
@@ -57,4 +58,6 @@ public interface Container extends EventHandler<ContainerEvent> {
 
   String toString();
 
+  Priority getPriority();
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.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/container/ContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
index 3c49489..0640350 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
@@ -44,12 +44,14 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger;
 import org.apache.hadoop.yarn.server.nodemanager.NMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent;
@@ -70,6 +72,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.Contai
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredContainerStatus;
+import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
@@ -125,11 +128,12 @@ public class ContainerImpl implements Container {
       RecoveredContainerStatus.REQUESTED;
   // whether container was marked as killed after recovery
   private boolean recoveredAsKilled = false;
+  private Context context;
 
   public ContainerImpl(Configuration conf, Dispatcher dispatcher,
       NMStateStoreService stateStore, ContainerLaunchContext launchContext,
       Credentials creds, NodeManagerMetrics metrics,
-      ContainerTokenIdentifier containerTokenIdentifier) {
+      ContainerTokenIdentifier containerTokenIdentifier, Context context) {
     this.daemonConf = conf;
     this.dispatcher = dispatcher;
     this.stateStore = stateStore;
@@ -144,8 +148,8 @@ public class ContainerImpl implements Container {
     ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
     this.readLock = readWriteLock.readLock();
     this.writeLock = readWriteLock.writeLock();
-
     stateMachine = stateMachineFactory.make(this);
+    this.context = context;
   }
 
   // constructor for a recovered container
@@ -154,9 +158,10 @@ public class ContainerImpl implements Container {
       Credentials creds, NodeManagerMetrics metrics,
       ContainerTokenIdentifier containerTokenIdentifier,
       RecoveredContainerStatus recoveredStatus, int exitCode,
-      String diagnostics, boolean wasKilled, Resource recoveredCapability) {
+      String diagnostics, boolean wasKilled, Resource recoveredCapability,
+      Context context) {
     this(conf, dispatcher, stateStore, launchContext, creds, metrics,
-        containerTokenIdentifier);
+        containerTokenIdentifier, context);
     this.recoveredStatus = recoveredStatus;
     this.exitCode = exitCode;
     this.recoveredAsKilled = wasKilled;
@@ -372,6 +377,10 @@ public class ContainerImpl implements Container {
     }
   }
 
+  public NMTimelinePublisher getNMTimelinePublisher() {
+    return context.getNMTimelinePublisher();
+  }
+
   @Override
   public String getUser() {
     this.readLock.lock();
@@ -483,7 +492,10 @@ public class ContainerImpl implements Container {
     // Inform the application
     @SuppressWarnings("rawtypes")
     EventHandler eventHandler = dispatcher.getEventHandler();
-    eventHandler.handle(new ApplicationContainerFinishedEvent(containerId));
+
+    ContainerStatus containerStatus = cloneAndGetContainerStatus();
+    eventHandler.handle(new ApplicationContainerFinishedEvent(containerStatus));
+
     // Remove the container from the resource-monitor
     eventHandler.handle(new ContainerStopMonitoringEvent(containerId));
     // Tell the logService too
@@ -985,7 +997,8 @@ public class ContainerImpl implements Container {
     public void transition(ContainerImpl container, ContainerEvent event) {
       container.metrics.releaseContainer(container.resource);
       container.sendFinishedEvents();
-      //if the current state is NEW it means the CONTAINER_INIT was never 
+
+      // if the current state is NEW it means the CONTAINER_INIT was never
       // sent for the event, thus no need to send the CONTAINER_STOP
       if (container.getCurrentState() 
           != org.apache.hadoop.yarn.api.records.ContainerState.NEW) {
@@ -1176,4 +1189,9 @@ public class ContainerImpl implements Container {
       LocalResourceRequest resource) {
     return container.resourcesUploadPolicies.get(resource);
   }
+
+  @Override
+  public Priority getPriority() {
+    return containerTokenIdentifier.getPriority();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.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/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index 51b80bd..3b6b6c4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -18,13 +18,9 @@
 
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
 
-import java.io.IOException;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
@@ -36,10 +32,6 @@ import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.timelineservice.ContainerEntity;
-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.TimelineClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -48,13 +40,13 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
 
 import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 public class ContainersMonitorImpl extends AbstractService implements
     ContainersMonitor {
@@ -86,17 +78,11 @@ public class ContainersMonitorImpl extends AbstractService implements
   private boolean pmemCheckEnabled;
   private boolean vmemCheckEnabled;
   private boolean containersMonitorEnabled;
-  
-  private boolean publishContainerMetricsToTimelineService;
 
   private long maxVCoresAllottedForContainers;
 
   private static final long UNKNOWN_MEMORY_LIMIT = -1L;
   private int nodeCpuPercentageForYARN;
-  
-  // For posting entities in new timeline service in a non-blocking way
-  // TODO replace with event loop in TimelineClient.
-  private static ExecutorService threadPool;
 
   @Private
   public static enum ContainerMetric {
@@ -210,22 +196,6 @@ public class ContainersMonitorImpl extends AbstractService implements
                 1) + "). Thrashing might happen.");
       }
     }
-    
-    publishContainerMetricsToTimelineService =
-        YarnConfiguration.systemMetricsPublisherEnabled(conf);
-
-    if (publishContainerMetricsToTimelineService) {
-      LOG.info("NodeManager has been configured to publish container " +
-          "metrics to Timeline Service V2.");
-      threadPool =
-          Executors.newCachedThreadPool(
-              new ThreadFactoryBuilder().setNameFormat("TimelineService #%d")
-              .build());
-    } else {
-      LOG.warn("NodeManager has not been configured to publish container " +
-          "metrics to Timeline Service V2.");
-    }
-    
     super.serviceInit(conf);
   }
 
@@ -269,29 +239,8 @@ public class ContainersMonitorImpl extends AbstractService implements
       }
     }
     
-    shutdownAndAwaitTermination();
-    
     super.serviceStop();
   }
-  
-  // TODO remove threadPool after adding non-blocking call in TimelineClient
-  private static void shutdownAndAwaitTermination() {
-    if (threadPool == null) {
-      return;
-    }
-    threadPool.shutdown();
-    try {
-      if (!threadPool.awaitTermination(60, TimeUnit.SECONDS)) {
-        threadPool.shutdownNow(); 
-        if (!threadPool.awaitTermination(60, TimeUnit.SECONDS))
-            LOG.error("ThreadPool did not terminate");
-      }
-    } catch (InterruptedException ie) {
-      threadPool.shutdownNow();
-      // Preserve interrupt status
-      Thread.currentThread().interrupt();
-    }
-  }
 
   @VisibleForTesting
   static class ProcessTreeInfo {
@@ -470,9 +419,6 @@ public class ContainersMonitorImpl extends AbstractService implements
           ContainerId containerId = entry.getKey();
           ProcessTreeInfo ptInfo = entry.getValue();
           
-          ContainerEntity entity = new ContainerEntity();
-          entity.setId(containerId.toString());
-          
           try {
             String pId = ptInfo.getPID();
 
@@ -556,26 +502,6 @@ public class ContainersMonitorImpl extends AbstractService implements
                   containerMetricsUnregisterDelayMs).recordCpuUsage
                   ((int)cpuUsagePercentPerCore, milliVcoresUsed);
             }
-
-            if (publishContainerMetricsToTimelineService) {
-              // if currentPmemUsage data is available
-              if (currentPmemUsage != 
-                  ResourceCalculatorProcessTree.UNAVAILABLE) {
-                TimelineMetric memoryMetric = new TimelineMetric();
-                memoryMetric.setId(ContainerMetric.MEMORY.toString() + pId);
-                memoryMetric.addValue(currentTime, currentPmemUsage);
-                entity.addMetric(memoryMetric);
-              }
-              // if cpuUsageTotalCoresPercentage data is available
-              if (cpuUsageTotalCoresPercentage != 
-                ResourceCalculatorProcessTree.UNAVAILABLE) {
-                TimelineMetric cpuMetric = new TimelineMetric();
-                cpuMetric.setId(ContainerMetric.CPU.toString() + pId);
-                cpuMetric.addValue(currentTime,
-                    cpuUsageTotalCoresPercentage);
-                entity.addMetric(cpuMetric);
-              }
-            }
             
             boolean isMemoryOverLimit = false;
             String msg = "";
@@ -632,23 +558,16 @@ public class ContainersMonitorImpl extends AbstractService implements
               LOG.info("Removed ProcessTree with root " + pId);
             }
 
+            ContainerImpl container =
+                (ContainerImpl) context.getContainers().get(containerId);
+            container.getNMTimelinePublisher().reportContainerResourceUsage(
+                container, currentTime, pId, currentPmemUsage,
+                cpuUsageTotalCoresPercentage);
           } catch (Exception e) {
             // Log the exception and proceed to the next container.
             LOG.warn("Uncaught exception in ContainersMonitorImpl "
                 + "while monitoring resource of " + containerId, e);
           }
-          
-          if (publishContainerMetricsToTimelineService) {
-            try {
-              TimelineClient timelineClient = context.getApplications().get(
-                  containerId.getApplicationAttemptId().getApplicationId()).
-                      getTimelineClient();
-              putEntityWithoutBlocking(timelineClient, entity);
-            } catch (Exception e) {
-              LOG.error("Exception in ContainersMonitorImpl in putting " +
-                  "resource usage metrics to timeline service.", e);
-            }
-          }
         }
         if (LOG.isDebugEnabled()) {
           LOG.debug("Total Resource Usage stats in NM by all containers : "
@@ -671,20 +590,6 @@ public class ContainersMonitorImpl extends AbstractService implements
         }
       }
     }
-    
-    private void putEntityWithoutBlocking(final TimelineClient timelineClient, 
-        final TimelineEntity entity) {
-      Runnable publishWrapper = new Runnable() {
-        public void run() {
-          try {
-            timelineClient.putEntities(entity);
-          } catch (IOException|YarnException e) {
-            LOG.error("putEntityNonBlocking get failed: " + e);
-          }
-        }
-      };
-      threadPool.execute(publishWrapper);
-    }
 
     private String formatErrorMessage(String memTypeExceeded,
         long currentVmemUsage, long vmemLimit,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelineEvent.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/NMTimelineEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelineEvent.java
new file mode 100644
index 0000000..af8d94c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelineEvent.java
@@ -0,0 +1,31 @@
+/**
+ * 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.nodemanager.timelineservice;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+
+public class NMTimelineEvent extends AbstractEvent<NMTimelineEventType> {
+  public NMTimelineEvent(NMTimelineEventType type) {
+    super(type);
+  }
+
+  public NMTimelineEvent(NMTimelineEventType type, long timestamp) {
+    super(type, timestamp);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelineEventType.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/NMTimelineEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelineEventType.java
new file mode 100644
index 0000000..c1129af
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/timelineservice/NMTimelineEventType.java
@@ -0,0 +1,24 @@
+/**
+ * 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.nodemanager.timelineservice;
+
+public enum NMTimelineEventType {
+  // Publish the NM Timeline entity
+  TIMELINE_ENTITY_PUBLISH,
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/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
new file mode 100644
index 0000000..2c5c300
--- /dev/null
+++ 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
@@ -0,0 +1,376 @@
+/**
+ * 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.nodemanager.timelineservice;
+
+import java.util.HashMap;
+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.service.CompositeService;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.timelineservice.ContainerEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.Identifier;
+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.client.api.TimelineClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationContainerFinishedEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl.ContainerMetric;
+import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
+import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
+
+public class NMTimelinePublisher extends CompositeService {
+
+  private static final Log LOG = LogFactory.getLog(NMTimelinePublisher.class);
+
+  private Dispatcher dispatcher;
+  private boolean publishSystemMetrics;
+
+  private Context context;
+
+  private NodeId nodeId;
+
+  private String httpAddress;
+
+  public NMTimelinePublisher(Context context) {
+    super(NMTimelinePublisher.class.getName());
+    this.context = context;
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    publishSystemMetrics =
+        YarnConfiguration.systemMetricsPublisherEnabled(conf);
+
+    if (publishSystemMetrics) {
+      dispatcher = new AsyncDispatcher();
+      dispatcher.register(NMTimelineEventType.class,
+          new ForwardingEventHandler());
+      dispatcher
+          .register(ContainerEventType.class, new ContainerEventHandler());
+      dispatcher.register(ApplicationEventType.class,
+          new ApplicationEventHandler());
+      dispatcher.register(LocalizationEventType.class,
+          new LocalizationEventDispatcher());
+      addIfService(dispatcher);
+      LOG.info("YARN system metrics publishing service is enabled");
+    } else {
+      LOG.info("YARN system metrics publishing service is not enabled");
+    }
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    super.serviceStart();
+    // context will be updated after containerManagerImpl is started
+    // hence NMMetricsPublisher is added subservice of containerManagerImpl
+    this.nodeId = context.getNodeId();
+    this.httpAddress = nodeId.getHost() + ":" + context.getHttpPort();
+  }
+
+  protected void handleNMTimelineEvent(NMTimelineEvent event) {
+    switch (event.getType()) {
+    case TIMELINE_ENTITY_PUBLISH:
+      putEntity(((TimelinePublishEvent) event).getTimelineEntityToPublish(),
+          ((TimelinePublishEvent) event).getApplicationId());
+      break;
+    default:
+      LOG.error("Unknown NMTimelineEvent type: " + event.getType());
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public void reportContainerResourceUsage(Container container,
+      long createdTime, String pId, Long pmemUsage,
+      Float cpuUsageTotalCoresPercentage) {
+    if (publishSystemMetrics
+        && (pmemUsage != ResourceCalculatorProcessTree.UNAVAILABLE || cpuUsageTotalCoresPercentage != ResourceCalculatorProcessTree.UNAVAILABLE)) {
+      ContainerEntity entity =
+          createContainerEntity(container.getContainerId());
+      long currentTimeMillis = System.currentTimeMillis();
+      if (pmemUsage != ResourceCalculatorProcessTree.UNAVAILABLE) {
+        TimelineMetric memoryMetric = new TimelineMetric();
+        memoryMetric.setId(ContainerMetric.MEMORY.toString() + pId);
+        memoryMetric.addValue(currentTimeMillis, pmemUsage);
+        entity.addMetric(memoryMetric);
+      }
+      if (cpuUsageTotalCoresPercentage != ResourceCalculatorProcessTree.UNAVAILABLE) {
+        TimelineMetric cpuMetric = new TimelineMetric();
+        cpuMetric.setId(ContainerMetric.CPU.toString() + pId);
+        cpuMetric.addValue(currentTimeMillis, cpuUsageTotalCoresPercentage);
+        entity.addMetric(cpuMetric);
+      }
+      dispatcher.getEventHandler().handle(
+          new TimelinePublishEvent(entity, container.getContainerId()
+              .getApplicationAttemptId().getApplicationId()));
+    }
+  }
+
+  private void publishContainerCreatedEvent(ContainerEntity entity,
+      ContainerId containerId, Resource resource, Priority priority,
+      long timestamp) {
+    Map<String, Object> entityInfo = new HashMap<String, Object>();
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO,
+        resource.getMemory());
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_VCORE_ENTITY_INFO,
+        resource.getVirtualCores());
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
+        nodeId.getHost());
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
+        nodeId.getPort());
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO,
+        priority.toString());
+    entityInfo.put(
+        ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO,
+        httpAddress);
+    entity.setInfo(entityInfo);
+
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(ContainerMetricsConstants.CREATED_EVENT_TYPE);
+    tEvent.setTimestamp(timestamp);
+
+    entity.addEvent(tEvent);
+    putEntity(entity, containerId.getApplicationAttemptId().getApplicationId());
+  }
+
+  private void publishContainerFinishedEvent(ContainerStatus containerStatus,
+      long timeStamp) {
+    ContainerId containerId = containerStatus.getContainerId();
+    TimelineEntity entity = createContainerEntity(containerId);
+
+    Map<String, Object> eventInfo = new HashMap<String, Object>();
+    eventInfo.put(ContainerMetricsConstants.DIAGNOSTICS_INFO_EVENT_INFO,
+        containerStatus.getDiagnostics());
+    eventInfo.put(ContainerMetricsConstants.EXIT_STATUS_EVENT_INFO,
+        containerStatus.getExitStatus());
+    eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO, containerStatus
+        .getState().toString());
+
+    TimelineEvent tEvent = new TimelineEvent();
+    tEvent.setId(ContainerMetricsConstants.FINISHED_EVENT_TYPE);
+    tEvent.setTimestamp(timeStamp);
+    tEvent.setInfo(eventInfo);
+
+    entity.addEvent(tEvent);
+    putEntity(entity, containerId.getApplicationAttemptId().getApplicationId());
+  }
+
+  private static ContainerEntity createContainerEntity(ContainerId containerId) {
+    ContainerEntity entity = new ContainerEntity();
+    entity.setId(containerId.toString());
+    Identifier parentIdentifier = new Identifier();
+    parentIdentifier
+        .setType(TimelineEntityType.YARN_APPLICATION_ATTEMPT.name());
+    parentIdentifier.setId(containerId.getApplicationAttemptId().toString());
+    entity.setParent(parentIdentifier);
+    return entity;
+  }
+
+  private void putEntity(TimelineEntity entity, ApplicationId appId) {
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Publishing the entity " + entity + ", JSON-style content: "
+            + TimelineUtils.dumpTimelineRecordtoJSON(entity));
+      }
+      TimelineClient timelineClient =
+          context.getApplications().get(appId).getTimelineClient();
+      timelineClient.putEntities(entity);
+    } catch (Exception e) {
+      LOG.error("Error when publishing entity " + entity, e);
+    }
+  }
+
+  public void publishApplicationEvent(ApplicationEvent event) {
+    if (!publishSystemMetrics) {
+      return;
+    }
+    // publish only when the desired event is received
+    switch (event.getType()) {
+    case INIT_APPLICATION:
+    case FINISH_APPLICATION:
+    case APPLICATION_CONTAINER_FINISHED:
+    case APPLICATION_LOG_HANDLING_FAILED:
+      dispatcher.getEventHandler().handle(event);
+      break;
+
+    default:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(event.getType()
+            + " is not a desired ApplicationEvent which needs to be published by"
+            + " NMTimelinePublisher");
+      }
+      break;
+    }
+  }
+
+  public void publishContainerEvent(ContainerEvent event) {
+    if (!publishSystemMetrics) {
+      return;
+    }
+    // publish only when the desired event is received
+    switch (event.getType()) {
+    case INIT_CONTAINER:
+      dispatcher.getEventHandler().handle(event);
+      break;
+
+    default:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(event.getType()
+            + " is not a desired ContainerEvent which needs to be published by"
+            + " NMTimelinePublisher");
+      }
+      break;
+    }
+  }
+
+  public void publishLocalizationEvent(LocalizationEvent event) {
+    if (!publishSystemMetrics) {
+      return;
+    }
+    // publish only when the desired event is received
+    switch (event.getType()) {
+    case CONTAINER_RESOURCES_LOCALIZED:
+    case INIT_CONTAINER_RESOURCES:
+      dispatcher.getEventHandler().handle(event);
+      break;
+
+    default:
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(event.getType()
+            + " is not a desired LocalizationEvent which needs to be published"
+            + " by NMTimelinePublisher");
+      }
+      break;
+    }
+  }
+
+  private class ApplicationEventHandler implements
+      EventHandler<ApplicationEvent> {
+    @Override
+    public void handle(ApplicationEvent event) {
+      switch (event.getType()) {
+      case APPLICATION_CONTAINER_FINISHED:
+        // this is actually used to publish the container Event
+        ApplicationContainerFinishedEvent evnt =
+            (ApplicationContainerFinishedEvent) event;
+        publishContainerFinishedEvent(evnt.getContainerStatus(),
+            event.getTimestamp());
+        break;
+      default:
+        LOG.error("Seems like event type is captured only in "
+            + "publishApplicationEvent method and not handled here");
+        break;
+      }
+    }
+  }
+
+  private class ContainerEventHandler implements EventHandler<ContainerEvent> {
+    @Override
+    public void handle(ContainerEvent event) {
+      ContainerId containerId = event.getContainerID();
+      Container container = context.getContainers().get(containerId);
+      long timestamp = event.getTimestamp();
+      ContainerEntity entity = createContainerEntity(containerId);
+
+      switch (event.getType()) {
+      case INIT_CONTAINER:
+        publishContainerCreatedEvent(entity, containerId,
+            container.getResource(), container.getPriority(), timestamp);
+        break;
+      default:
+        LOG.error("Seems like event type is captured only in "
+            + "publishContainerEvent method and not handled here");
+        break;
+      }
+    }
+  }
+
+  private static final class LocalizationEventDispatcher implements
+      EventHandler<LocalizationEvent> {
+    @Override
+    public void handle(LocalizationEvent event) {
+      switch (event.getType()) {
+      case INIT_CONTAINER_RESOURCES:
+      case CONTAINER_RESOURCES_LOCALIZED:
+        // TODO after priority based flush jira is finished
+        break;
+      default:
+        LOG.error("Seems like event type is captured only in "
+            + "publishLocalizationEvent method and not handled here");
+        break;
+      }
+    }
+  }
+
+  /**
+   * EventHandler implementation which forward events to NMMetricsPublisher.
+   * Making use of it, NMMetricsPublisher can avoid to have a public handle
+   * method.
+   */
+  private final class ForwardingEventHandler implements
+      EventHandler<NMTimelineEvent> {
+
+    @Override
+    public void handle(NMTimelineEvent event) {
+      handleNMTimelineEvent(event);
+    }
+  }
+
+  private static class TimelinePublishEvent extends NMTimelineEvent {
+    private ApplicationId appId;
+    private TimelineEntity entityToPublish;
+
+    public TimelinePublishEvent(TimelineEntity entity, ApplicationId appId) {
+      super(NMTimelineEventType.TIMELINE_ENTITY_PUBLISH, System
+          .currentTimeMillis());
+      this.appId = appId;
+      this.entityToPublish = entity;
+    }
+
+    public ApplicationId getApplicationId() {
+      return appId;
+    }
+
+    public TimelineEntity getTimelineEntityToPublish() {
+      return entityToPublish;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.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/TestNodeStatusUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
index dd76f60..009fd9a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
@@ -83,6 +83,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.proto.YarnServerCommonServiceProtos.NodeHeartbeatResponseProto;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
+import org.apache.hadoop.yarn.server.api.ContainerContext;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
@@ -252,8 +253,10 @@ public class TestNodeStatusUpdater {
                 firstContainerID, InetAddress.getByName("localhost")
                     .getCanonicalHostName(), 1234, user, resource,
                 currentTime + 10000, 123, "password".getBytes(), currentTime));
-        Container container = new ContainerImpl(conf, mockDispatcher,
-            stateStore, launchContext, null, mockMetrics, containerToken);
+        Container container =
+            new ContainerImpl(conf, mockDispatcher, stateStore, launchContext,
+                null, mockMetrics, containerToken,
+                mock(Context.class));
         this.context.getContainers().put(firstContainerID, container);
       } else if (heartBeatID == 2) {
         // Checks on the RM end
@@ -291,8 +294,10 @@ public class TestNodeStatusUpdater {
                 secondContainerID, InetAddress.getByName("localhost")
                     .getCanonicalHostName(), 1234, user, resource,
                 currentTime + 10000, 123, "password".getBytes(), currentTime));
-        Container container = new ContainerImpl(conf, mockDispatcher,
-            stateStore, launchContext, null, mockMetrics, containerToken);
+        Container container =
+            new ContainerImpl(conf, mockDispatcher, stateStore, launchContext,
+                null, mockMetrics, containerToken,
+                mock(Context.class));
         this.context.getContainers().put(secondContainerID, container);
       } else if (heartBeatID == 3) {
         // Checks on the RM end
@@ -1008,7 +1013,8 @@ public class TestNodeStatusUpdater {
             "password".getBytes(), 0);
     Container anyCompletedContainer = new ContainerImpl(conf, null,
         null, null, null, null,
-        BuilderUtils.newContainerTokenIdentifier(containerToken)) {
+        BuilderUtils.newContainerTokenIdentifier(containerToken),
+        mock(Context.class)) {
 
       @Override
       public ContainerState getCurrentState() {
@@ -1029,7 +1035,7 @@ public class TestNodeStatusUpdater {
           "password".getBytes(), 0);
     Container runningContainer =
         new ContainerImpl(conf, null, null, null, null, null,
-          BuilderUtils.newContainerTokenIdentifier(runningContainerToken)) {
+          BuilderUtils.newContainerTokenIdentifier(runningContainerToken), mock(Context.class)) {
           @Override
           public ContainerState getCurrentState() {
             return ContainerState.RUNNING;
@@ -1085,9 +1091,10 @@ public class TestNodeStatusUpdater {
         BuilderUtils.newContainerToken(containerId, "host", 1234, "user",
             BuilderUtils.newResource(1024, 1), 0, 123,
             "password".getBytes(), 0);
+
     Container completedContainer = new ContainerImpl(conf, null,
         null, null, null, null,
-        BuilderUtils.newContainerTokenIdentifier(containerToken)) {
+        BuilderUtils.newContainerTokenIdentifier(containerToken),mock(Context.class)) {
       @Override
       public ContainerState getCurrentState() {
         return ContainerState.COMPLETE;
@@ -1124,7 +1131,8 @@ public class TestNodeStatusUpdater {
             "password".getBytes(), 0);
     Container anyCompletedContainer = new ContainerImpl(conf, null,
         null, null, null, null,
-        BuilderUtils.newContainerTokenIdentifier(containerToken)) {
+        BuilderUtils.newContainerTokenIdentifier(containerToken),
+        mock(Context.class)) {
 
       @Override
       public ContainerState getCurrentState() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.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/amrmproxy/BaseAMRMProxyTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
index bf4fd52..35f2d39 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/amrmproxy/BaseAMRMProxyTest.java
@@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 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.Records;
 import org.junit.After;
@@ -688,5 +689,13 @@ public abstract class BaseAMRMProxyTest {
       return null;
     }
 
+    @Override
+    public void setNMTimelinePublisher(NMTimelinePublisher nmMetricsPublisher) {
+    }
+
+    @Override
+    public NMTimelinePublisher getNMTimelinePublisher() {
+      return null;
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.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/TestAuxServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java
index 1380752..8c4bd25 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
 
 import java.io.File;
 import java.io.IOException;
@@ -53,6 +54,7 @@ import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
 import org.apache.hadoop.yarn.server.api.AuxiliaryService;
 import org.apache.hadoop.yarn.server.api.ContainerInitializationContext;
 import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
+import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.junit.Assert;
@@ -193,7 +195,7 @@ public class TestAuxServices {
         ContainerId.newContainerId(attemptId, 1), "", "",
         Resource.newInstance(1, 1), 0,0,0, Priority.newInstance(0), 0);
     Container container = new ContainerImpl(null, null, null, null, null,
-        null, cti);
+        null, cti, mock(Context.class));
     ContainerId containerId = container.getContainerId();
     Resource resource = container.getResource();
     event = new AuxServicesEvent(AuxServicesEventType.CONTAINER_INIT,container);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/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 31e2105..f29b791 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
@@ -101,10 +101,12 @@ import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreServic
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 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.ConverterUtils;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.Mockito;
 
 public class TestContainerManagerRecovery extends BaseContainerManagerTest {
 
@@ -634,6 +636,12 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
               boolean blockNewContainerRequests) {
             // do nothing
           }
+
+          @Override
+          public NMTimelinePublisher createNMTimelinePublisher(Context context) {
+            NMTimelinePublisher timelinePublisher = mock(NMTimelinePublisher.class);
+            return timelinePublisher;
+          }
     };
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.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/application/TestApplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java
index 002d4cf..38b3172f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java
@@ -38,7 +38,10 @@ 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.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
@@ -579,7 +582,7 @@ public class TestApplication {
 
     public void containerFinished(int containerNum) {
       app.handle(new ApplicationContainerFinishedEvent(containers.get(
-          containerNum).getContainerId()));
+          containerNum).cloneAndGetContainerStatus()));
       drainDispatcherEvents();
     }
 
@@ -616,6 +619,8 @@ public class TestApplication {
     when(c.getLaunchContext()).thenReturn(launchContext);
     when(launchContext.getApplicationACLs()).thenReturn(
         new HashMap<ApplicationAccessType, String>());
+    when(c.cloneAndGetContainerStatus()).thenReturn(BuilderUtils.newContainerStatus(cId,
+        ContainerState.NEW, "", 0, Resource.newInstance(1024, 1)));
     return c;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.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/container/TestContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
index 2ab9842..8a21ed2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
@@ -865,7 +865,7 @@ public class TestContainer {
       when(ctxt.getServiceData()).thenReturn(serviceData);
 
       c = new ContainerImpl(conf, dispatcher, new NMNullStateStoreService(),
-          ctxt, null, metrics, identifier);
+          ctxt, null, metrics, identifier, mock(Context.class));
       dispatcher.register(ContainerEventType.class,
           new EventHandler<ContainerEvent>() {
             @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.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/webapp/MockContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
index 394a92c..37c726f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -144,4 +145,9 @@ public class MockContainer implements Container {
   public NMContainerStatus getNMContainerStatus() {
     return null;
   }
+
+  @Override
+  public Priority getPriority() {
+    return Priority.UNDEFINED;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.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/webapp/TestNMWebServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
index ed94fb6..e839f47 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
@@ -220,7 +220,8 @@ public class TestNMWebServer {
       Container container =
           new ContainerImpl(conf, dispatcher, stateStore, launchContext,
             null, metrics,
-            BuilderUtils.newContainerTokenIdentifier(containerToken)) {
+            BuilderUtils.newContainerTokenIdentifier(containerToken),
+            mock(Context.class)) {
 
             @Override
             public ContainerState getContainerState() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/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 befaa83..4147d42 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
@@ -137,9 +137,19 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
     // intercept the event of the AM container being stopped and remove the app
     // level collector service
     if (context.getContainerType() == ContainerType.APPLICATION_MASTER) {
-      ApplicationId appId = context.getContainerId().
-          getApplicationAttemptId().getApplicationId();
-      removeApplication(appId);
+      final ApplicationId appId =
+          context.getContainerId().getApplicationAttemptId().getApplicationId();
+      new Thread(new Runnable() {
+        public void run() {
+          try {
+            // TODO Temporary Fix until solution for YARN-3995 is finalized.
+            Thread.sleep(1000l);
+          } catch (InterruptedException e) {
+            e.printStackTrace();
+          }
+          removeApplication(appId);
+        }
+      }).start();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/acbf140a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.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/TestPerNodeTimelineCollectorsAuxService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
index 7cc612d..dafc76e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
@@ -98,6 +98,15 @@ public class TestPerNodeTimelineCollectorsAuxService {
     when(context.getContainerType()).thenReturn(
         ContainerType.APPLICATION_MASTER);
     auxService.stopContainer(context);
+
+    // TODO Temporary Fix until solution for YARN-3995 is finalized
+    for (int i = 0; i < 4; i++) {
+      Thread.sleep(500l);
+      if (!auxService.hasApplication(appAttemptId.getApplicationId())) {
+        break;
+      }
+    }
+
     // auxService should not have that app
     assertFalse(auxService.hasApplication(appAttemptId.getApplicationId()));
     auxService.close();


[46/50] [abbrv] hadoop git commit: YARN-4350. TestDistributedShell fails for V2 scenarios. (Naganarasimha G R via Varun Saxena)

Posted by gt...@apache.org.
YARN-4350. TestDistributedShell fails for V2 scenarios. (Naganarasimha G R via Varun Saxena)


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

Branch: refs/heads/feature-YARN-2928
Commit: 41b9f270ac6510fc5b692f3e37946fb95209a93e
Parents: e9b93fc
Author: Varun Saxena <va...@apache.org>
Authored: Sun Dec 20 02:14:54 2015 +0530
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 18:03:31 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                | 3 +++
 .../applications/distributedshell/TestDistributedShell.java    | 1 +
 .../java/org/apache/hadoop/yarn/server/MiniYARNCluster.java    | 6 +++---
 .../org/apache/hadoop/yarn/server/TestMiniYarnCluster.java     | 2 --
 4 files changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/41b9f270/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 611e521..fbd40ba 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -202,6 +202,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
 
     YARN-4460. [Bug fix] RM fails to start when SMP is enabled. (Li Lu via varunsaxena)
 
+    YARN-4350. TestDistributedShell fails for V2 scenarios. (Naganarasimha G R
+    via varunsaxena)
+
 Trunk - Unreleased
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41b9f270/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index c90f97b..f141ca2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -104,6 +104,7 @@ public class TestDistributedShell {
     if (!testName.getMethodName().toLowerCase().contains("v2")) {
       // disable aux-service based timeline collectors
       conf.set(YarnConfiguration.NM_AUX_SERVICES, "");
+      conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 1.0f);
     } else {
       // set version to 2
       conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41b9f270/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
index fab85d7..b80d2dd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
@@ -35,9 +35,9 @@ import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.CompositeService;
-import org.apache.hadoop.service.ServiceStateException;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
@@ -757,8 +757,8 @@ public class MiniYARNCluster extends CompositeService {
       if (!useFixedPorts) {
         String hostname = MiniYARNCluster.getHostname();
         conf.set(YarnConfiguration.TIMELINE_SERVICE_ADDRESS, hostname + ":0");
-        conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, hostname
-            + ":0");
+        conf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+            hostname + ":" + ServerSocketUtil.getPort(9188, 10));
       }
       appHistoryServer.init(conf);
       super.serviceInit(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41b9f270/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYarnCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYarnCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYarnCluster.java
index e67a236..9226ead 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYarnCluster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYarnCluster.java
@@ -74,8 +74,6 @@ public class TestMiniYarnCluster {
       String hostname = MiniYARNCluster.getHostname();
       Assert.assertEquals(hostname + ":0",
         conf.get(YarnConfiguration.TIMELINE_SERVICE_ADDRESS));
-      Assert.assertEquals(hostname + ":0",
-        conf.get(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS));
 
       cluster.start();
 


[18/50] [abbrv] hadoop git commit: YARN-4074. [timeline reader] implement support for querying for flows and flow runs (sjlee via vrushali)

Posted by gt...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.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/HBaseTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
index c514c20..889ae19 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
@@ -20,13 +20,8 @@ package org.apache.hadoop.yarn.server.timelineservice.storage;
 
 import java.io.IOException;
 import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
 import java.util.Set;
-import java.util.TreeSet;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -34,47 +29,17 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Get;
-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.service.AbstractService;
 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.TimelineEvent;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
-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;
-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.AppToFlowRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineReaderUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
-
-import com.google.common.base.Preconditions;
 
 public class HBaseTimelineReaderImpl
     extends AbstractService implements TimelineReader {
 
   private static final Log LOG = LogFactory
       .getLog(HBaseTimelineReaderImpl.class);
-  private static final long DEFAULT_BEGIN_TIME = 0L;
-  private static final long DEFAULT_END_TIME = Long.MAX_VALUE;
 
   private Configuration hbaseConf = null;
   private Connection conn;
-  private EntityTable entityTable;
-  private AppToFlowTable appToFlowTable;
-  private ApplicationTable applicationTable;
 
   public HBaseTimelineReaderImpl() {
     super(HBaseTimelineReaderImpl.class.getName());
@@ -85,9 +50,6 @@ public class HBaseTimelineReaderImpl
     super.serviceInit(conf);
     hbaseConf = HBaseConfiguration.create(conf);
     conn = ConnectionFactory.createConnection(hbaseConf);
-    entityTable = new EntityTable();
-    appToFlowTable = new AppToFlowTable();
-    applicationTable = new ApplicationTable();
   }
 
   @Override
@@ -104,35 +66,10 @@ public class HBaseTimelineReaderImpl
       String flowId, Long flowRunId, String appId, String entityType,
       String entityId, EnumSet<Field> fieldsToRetrieve)
       throws IOException {
-    validateParams(userId, clusterId, appId, entityType, entityId, true);
-    // In reality both should be null or neither should be null
-    if (flowId == null || flowRunId == null) {
-      FlowContext context = lookupFlowContext(clusterId, appId);
-      flowId = context.flowId;
-      flowRunId = context.flowRunId;
-    }
-    if (fieldsToRetrieve == null) {
-      fieldsToRetrieve = EnumSet.noneOf(Field.class);
-    }
-
-    boolean isApplication = isApplicationEntity(entityType);
-    byte[] rowKey = isApplication ?
-        ApplicationRowKey.getRowKey(clusterId, userId, flowId, flowRunId,
-            appId) :
-        EntityRowKey.getRowKey(clusterId, userId, flowId, flowRunId, appId,
-            entityType, entityId);
-    Get get = new Get(rowKey);
-    get.setMaxVersions(Integer.MAX_VALUE);
-    Result result = isApplication ?
-        applicationTable.getResult(hbaseConf, conn, get) :
-        entityTable.getResult(hbaseConf, conn, get);
-    return parseEntity(result, fieldsToRetrieve,
-        false, DEFAULT_BEGIN_TIME, DEFAULT_END_TIME, false, DEFAULT_BEGIN_TIME,
-        DEFAULT_END_TIME, null, null, null, null, null, null, isApplication);
-  }
-
-  private static boolean isApplicationEntity(String entityType) {
-    return TimelineEntityType.YARN_APPLICATION.toString().equals(entityType);
+    TimelineEntityReader reader =
+        TimelineEntityReaderFactory.createSingleEntityReader(userId, clusterId,
+            flowId, flowRunId, appId, entityType, entityId, fieldsToRetrieve);
+    return reader.readEntity(hbaseConf, conn);
   }
 
   @Override
@@ -144,361 +81,12 @@ public class HBaseTimelineReaderImpl
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String> metricFilters, Set<String> eventFilters,
       EnumSet<Field> fieldsToRetrieve) throws IOException {
-    validateParams(userId, clusterId, appId, entityType, null, false);
-    // In reality both should be null or neither should be null
-    if (flowId == null || flowRunId == null) {
-      FlowContext context = lookupFlowContext(clusterId, appId);
-      flowId = context.flowId;
-      flowRunId = context.flowRunId;
-    }
-    if (limit == null) {
-      limit = TimelineReader.DEFAULT_LIMIT;
-    }
-    if (createdTimeBegin == null) {
-      createdTimeBegin = DEFAULT_BEGIN_TIME;
-    }
-    if (createdTimeEnd == null) {
-      createdTimeEnd = DEFAULT_END_TIME;
-    }
-    if (modifiedTimeBegin == null) {
-      modifiedTimeBegin = DEFAULT_BEGIN_TIME;
-    }
-    if (modifiedTimeEnd == null) {
-      modifiedTimeEnd = DEFAULT_END_TIME;
-    }
-    if (fieldsToRetrieve == null) {
-      fieldsToRetrieve = EnumSet.noneOf(Field.class);
-    }
-
-    NavigableSet<TimelineEntity> entities = new TreeSet<>();
-    boolean isApplication = isApplicationEntity(entityType);
-    if (isApplication) {
-      // If getEntities() is called for an application, there can be at most
-      // one entity. If the entity passes the filter, it is returned. Otherwise,
-      // an empty set is returned.
-      byte[] rowKey = ApplicationRowKey.getRowKey(clusterId, userId, flowId,
-          flowRunId, appId);
-      Get get = new Get(rowKey);
-      get.setMaxVersions(Integer.MAX_VALUE);
-      Result result = applicationTable.getResult(hbaseConf, conn, get);
-      TimelineEntity entity = parseEntity(result, fieldsToRetrieve,
-          true, createdTimeBegin, createdTimeEnd, true, modifiedTimeBegin,
-          modifiedTimeEnd, isRelatedTo, relatesTo, infoFilters, configFilters,
-          eventFilters, metricFilters, isApplication);
-      if (entity != null) {
-        entities.add(entity);
-      }
-    } else {
-      // Scan through part of the table to find the entities belong to one app
-      // and one type
-      Scan scan = new Scan();
-      scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
-          clusterId, userId, flowId, flowRunId, appId, entityType));
-      scan.setMaxVersions(Integer.MAX_VALUE);
-      ResultScanner scanner =
-          entityTable.getResultScanner(hbaseConf, conn, scan);
-      for (Result result : scanner) {
-        TimelineEntity entity = parseEntity(result, fieldsToRetrieve,
-            true, createdTimeBegin, createdTimeEnd,
-            true, modifiedTimeBegin, modifiedTimeEnd,
-            isRelatedTo, relatesTo, infoFilters, configFilters, eventFilters,
-            metricFilters, isApplication);
-        if (entity == null) {
-          continue;
-        }
-        if (entities.size() > limit) {
-          entities.pollLast();
-        }
-        entities.add(entity);
-      }
-    }
-    return entities;
-  }
-
-  private FlowContext lookupFlowContext(String clusterId, String appId)
-      throws IOException {
-    byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
-    Get get = new Get(rowKey);
-    Result result = appToFlowTable.getResult(hbaseConf, conn, get);
-    if (result != null && !result.isEmpty()) {
-      return new FlowContext(
-          AppToFlowColumn.FLOW_ID.readResult(result).toString(),
-          ((Number) AppToFlowColumn.FLOW_RUN_ID.readResult(result)).longValue());
-    } else {
-       throw new IOException(
-           "Unable to find the context flow ID and flow run ID for clusterId=" +
-           clusterId + ", appId=" + appId);
-    }
-  }
-
-  private static class FlowContext {
-    private String flowId;
-    private Long flowRunId;
-    public FlowContext(String flowId, Long flowRunId) {
-      this.flowId = flowId;
-      this.flowRunId = flowRunId;
-    }
-  }
-
-  private static void validateParams(String userId, String clusterId,
-      String appId, String entityType, String entityId, boolean checkEntityId) {
-    Preconditions.checkNotNull(userId, "userId shouldn't be null");
-    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
-    Preconditions.checkNotNull(appId, "appId shouldn't be null");
-    Preconditions.checkNotNull(entityType, "entityType shouldn't be null");
-    if (checkEntityId) {
-      Preconditions.checkNotNull(entityId, "entityId shouldn't be null");
-    }
-  }
-
-  private static TimelineEntity parseEntity(
-      Result result, EnumSet<Field> fieldsToRetrieve,
-      boolean checkCreatedTime, long createdTimeBegin, long createdTimeEnd,
-      boolean checkModifiedTime, long modifiedTimeBegin, long modifiedTimeEnd,
-      Map<String, Set<String>> isRelatedTo, Map<String, Set<String>> relatesTo,
-      Map<String, Object> infoFilters, Map<String, String> configFilters,
-      Set<String> eventFilters, Set<String> metricFilters,
-      boolean isApplication)
-          throws IOException {
-    if (result == null || result.isEmpty()) {
-      return null;
-    }
-    TimelineEntity entity = new TimelineEntity();
-    String entityType = isApplication ?
-        TimelineEntityType.YARN_APPLICATION.toString() :
-        EntityColumn.TYPE.readResult(result).toString();
-    entity.setType(entityType);
-    String entityId = isApplication ?
-        ApplicationColumn.ID.readResult(result).toString() :
-        EntityColumn.ID.readResult(result).toString();
-    entity.setId(entityId);
-
-    // fetch created time
-    Number createdTime = isApplication ?
-        (Number)ApplicationColumn.CREATED_TIME.readResult(result) :
-        (Number)EntityColumn.CREATED_TIME.readResult(result);
-    entity.setCreatedTime(createdTime.longValue());
-    if (checkCreatedTime && (entity.getCreatedTime() < createdTimeBegin ||
-        entity.getCreatedTime() > createdTimeEnd)) {
-      return null;
-    }
-
-    // fetch modified time
-    Number modifiedTime = isApplication ?
-        (Number)ApplicationColumn.MODIFIED_TIME.readResult(result) :
-        (Number)EntityColumn.MODIFIED_TIME.readResult(result);
-    entity.setModifiedTime(modifiedTime.longValue());
-    if (checkModifiedTime && (entity.getModifiedTime() < modifiedTimeBegin ||
-        entity.getModifiedTime() > modifiedTimeEnd)) {
-      return null;
-    }
-
-    // fetch is related to entities
-    boolean checkIsRelatedTo = isRelatedTo != null && isRelatedTo.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
-      if (isApplication) {
-        readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO,
-            true);
-      } else {
-        readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO,
-            true);
-      }
-      if (checkIsRelatedTo && !TimelineReaderUtils.matchRelations(
-          entity.getIsRelatedToEntities(), isRelatedTo)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.IS_RELATED_TO)) {
-        entity.getIsRelatedToEntities().clear();
-      }
-    }
-
-    // fetch relates to entities
-    boolean checkRelatesTo = relatesTo != null && relatesTo.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
-      if (isApplication) {
-        readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO,
-            false);
-      } else {
-        readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false);
-      }
-      if (checkRelatesTo && !TimelineReaderUtils.matchRelations(
-          entity.getRelatesToEntities(), relatesTo)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.RELATES_TO)) {
-        entity.getRelatesToEntities().clear();
-      }
-    }
-
-    // fetch info
-    boolean checkInfo = infoFilters != null && infoFilters.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
-      if (isApplication) {
-        readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false);
-      } else {
-        readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false);
-      }
-      if (checkInfo &&
-          !TimelineReaderUtils.matchFilters(entity.getInfo(), infoFilters)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.INFO)) {
-        entity.getInfo().clear();
-      }
-    }
-
-    // fetch configs
-    boolean checkConfigs = configFilters != null && configFilters.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
-      if (isApplication) {
-        readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true);
-      } else {
-        readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true);
-      }
-      if (checkConfigs && !TimelineReaderUtils.matchFilters(
-          entity.getConfigs(), configFilters)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.CONFIGS)) {
-        entity.getConfigs().clear();
-      }
-    }
-
-    // fetch events
-    boolean checkEvents = eventFilters != null && eventFilters.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
-      readEvents(entity, result, isApplication);
-      if (checkEvents && !TimelineReaderUtils.matchEventFilters(
-          entity.getEvents(), eventFilters)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.EVENTS)) {
-        entity.getEvents().clear();
-      }
-    }
-
-    // fetch metrics
-    boolean checkMetrics = metricFilters != null && metricFilters.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
-      readMetrics(entity, result, isApplication);
-      if (checkMetrics && !TimelineReaderUtils.matchMetricFilters(
-          entity.getMetrics(), metricFilters)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.METRICS)) {
-        entity.getMetrics().clear();
-      }
-    }
-    return entity;
-  }
-
-  private static <T> void readRelationship(
-      TimelineEntity entity, Result result, ColumnPrefix<T> prefix,
-      boolean isRelatedTo) throws IOException {
-    // isRelatedTo and relatesTo are of type Map<String, Set<String>>
-    Map<String, Object> columns = prefix.readResults(result);
-    for (Map.Entry<String, Object> column : columns.entrySet()) {
-      for (String id : Separator.VALUES.splitEncoded(
-          column.getValue().toString())) {
-        if (isRelatedTo) {
-          entity.addIsRelatedToEntity(column.getKey(), id);
-        } else {
-          entity.addRelatesToEntity(column.getKey(), id);
-        }
-      }
-    }
-  }
-
-  private static <T> void readKeyValuePairs(
-      TimelineEntity entity, Result result, ColumnPrefix<T> prefix,
-      boolean isConfig) throws IOException {
-    // info and configuration are of type Map<String, Object or String>
-    Map<String, Object> columns = prefix.readResults(result);
-    if (isConfig) {
-      for (Map.Entry<String, Object> column : columns.entrySet()) {
-        entity.addConfig(column.getKey(), column.getValue().toString());
-      }
-    } else {
-      entity.addInfo(columns);
-    }
-  }
-
-  /**
-   * Read events from the entity table or the application table. The column name
-   * is of the form "eventId=timestamp=infoKey" where "infoKey" may be omitted
-   * if there is no info associated with the event.
-   *
-   * See {@link EntityTable} and {@link ApplicationTable} for a more detailed
-   * schema description.
-   */
-  private static void readEvents(TimelineEntity entity, Result result,
-      boolean isApplication) throws IOException {
-    Map<String, TimelineEvent> eventsMap = new HashMap<>();
-    Map<?, Object> eventsResult = isApplication ?
-        ApplicationColumnPrefix.EVENT.
-            readResultsHavingCompoundColumnQualifiers(result) :
-        EntityColumnPrefix.EVENT.
-            readResultsHavingCompoundColumnQualifiers(result);
-    for (Map.Entry<?, Object> eventResult : eventsResult.entrySet()) {
-      byte[][] karr = (byte[][])eventResult.getKey();
-      // the column name is of the form "eventId=timestamp=infoKey"
-      if (karr.length == 3) {
-        String id = Bytes.toString(karr[0]);
-        long ts = TimelineWriterUtils.invert(Bytes.toLong(karr[1]));
-        String key = Separator.VALUES.joinEncoded(id, Long.toString(ts));
-        TimelineEvent event = eventsMap.get(key);
-        if (event == null) {
-          event = new TimelineEvent();
-          event.setId(id);
-          event.setTimestamp(ts);
-          eventsMap.put(key, event);
-        }
-        // handle empty info
-        String infoKey = karr[2].length == 0 ? null : Bytes.toString(karr[2]);
-        if (infoKey != null) {
-          event.addInfo(infoKey, eventResult.getValue());
-        }
-      } else {
-        LOG.warn("incorrectly formatted column name: it will be discarded");
-        continue;
-      }
-    }
-    Set<TimelineEvent> eventsSet = new HashSet<>(eventsMap.values());
-    entity.addEvents(eventsSet);
-  }
-
-  private static void readMetrics(TimelineEntity entity, Result result,
-      boolean isApplication) throws IOException {
-    NavigableMap<String, NavigableMap<Long, Number>> metricsResult;
-    if (isApplication) {
-      metricsResult =
-          ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result);
-    } else {
-      metricsResult =
-          EntityColumnPrefix.METRIC.readResultsWithTimestamps(result);
-    }
-    for (Map.Entry<String, NavigableMap<Long, Number>> metricResult:
-        metricsResult.entrySet()) {
-      TimelineMetric metric = new TimelineMetric();
-      metric.setId(metricResult.getKey());
-      // Simply assume that if the value set contains more than 1 elements, the
-      // metric is a TIME_SERIES metric, otherwise, it's a SINGLE_VALUE metric
-      metric.setType(metricResult.getValue().size() > 1 ?
-          TimelineMetric.Type.TIME_SERIES : TimelineMetric.Type.SINGLE_VALUE);
-      metric.addValues(metricResult.getValue());
-      entity.addMetric(metric);
-    }
+    TimelineEntityReader reader =
+        TimelineEntityReaderFactory.createMultipleEntitiesReader(userId,
+            clusterId, flowId, flowRunId, appId, entityType, limit,
+            createdTimeBegin, createdTimeEnd, modifiedTimeBegin,
+            modifiedTimeEnd, relatesTo, isRelatedTo, infoFilters, configFilters,
+            metricFilters, eventFilters, fieldsToRetrieve);
+    return reader.readEntities(hbaseConf, conn);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.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/TimelineEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
new file mode 100644
index 0000000..0d1134c
--- /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/TimelineEntityReader.java
@@ -0,0 +1,223 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Result;
+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.TimelineReader.Field;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+
+/**
+ * The base class for reading and deserializing timeline entities from the
+ * HBase storage. Different types can be defined for different types of the
+ * entities that are being requested.
+ */
+abstract class TimelineEntityReader {
+  protected final boolean singleEntityRead;
+
+  protected String userId;
+  protected String clusterId;
+  protected String flowId;
+  protected Long flowRunId;
+  protected String appId;
+  protected String entityType;
+  protected EnumSet<Field> fieldsToRetrieve;
+  // used only for a single entity read mode
+  protected String entityId;
+  // used only for multiple entity read mode
+  protected Long limit;
+  protected Long createdTimeBegin;
+  protected Long createdTimeEnd;
+  protected Long modifiedTimeBegin;
+  protected Long modifiedTimeEnd;
+  protected Map<String, Set<String>> relatesTo;
+  protected Map<String, Set<String>> isRelatedTo;
+  protected Map<String, Object> infoFilters;
+  protected Map<String, String> configFilters;
+  protected Set<String> metricFilters;
+  protected Set<String> eventFilters;
+
+  /**
+   * Main table the entity reader uses.
+   */
+  protected BaseTable<?> table;
+
+  /**
+   * Instantiates a reader for multiple-entity reads.
+   */
+  protected TimelineEntityReader(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      EnumSet<Field> fieldsToRetrieve) {
+    this.singleEntityRead = false;
+    this.userId = userId;
+    this.clusterId = clusterId;
+    this.flowId = flowId;
+    this.flowRunId = flowRunId;
+    this.appId = appId;
+    this.entityType = entityType;
+    this.fieldsToRetrieve = fieldsToRetrieve;
+    this.limit = limit;
+    this.createdTimeBegin = createdTimeBegin;
+    this.createdTimeEnd = createdTimeEnd;
+    this.modifiedTimeBegin = modifiedTimeBegin;
+    this.modifiedTimeEnd = modifiedTimeEnd;
+    this.relatesTo = relatesTo;
+    this.isRelatedTo = isRelatedTo;
+    this.infoFilters = infoFilters;
+    this.configFilters = configFilters;
+    this.metricFilters = metricFilters;
+    this.eventFilters = eventFilters;
+
+    this.table = getTable();
+  }
+
+  /**
+   * Instantiates a reader for single-entity reads.
+   */
+  protected TimelineEntityReader(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      String entityId, EnumSet<Field> fieldsToRetrieve) {
+    this.singleEntityRead = true;
+    this.userId = userId;
+    this.clusterId = clusterId;
+    this.flowId = flowId;
+    this.flowRunId = flowRunId;
+    this.appId = appId;
+    this.entityType = entityType;
+    this.fieldsToRetrieve = fieldsToRetrieve;
+    this.entityId = entityId;
+
+    this.table = getTable();
+  }
+
+  /**
+   * Reads and deserializes a single timeline entity from the HBase storage.
+   */
+  public TimelineEntity readEntity(Configuration hbaseConf, Connection conn)
+      throws IOException {
+    validateParams();
+    augmentParams(hbaseConf, conn);
+
+    Result result = getResult(hbaseConf, conn);
+    return parseEntity(result);
+  }
+
+  /**
+   * Reads and deserializes a set of timeline entities from the HBase storage.
+   * It goes through all the results available, and returns the number of
+   * entries as specified in the limit in the entity's natural sort order.
+   */
+  public Set<TimelineEntity> readEntities(Configuration hbaseConf,
+      Connection conn) throws IOException {
+    validateParams();
+    augmentParams(hbaseConf, conn);
+
+    NavigableSet<TimelineEntity> entities = new TreeSet<>();
+    Iterable<Result> results = getResults(hbaseConf, conn);
+    for (Result result : results) {
+      TimelineEntity entity = parseEntity(result);
+      if (entity == null) {
+        continue;
+      }
+      entities.add(entity);
+      if (entities.size() > limit) {
+        entities.pollLast();
+      }
+    }
+    return entities;
+  }
+
+  /**
+   * Returns the main table to be used by the entity reader.
+   */
+  protected abstract BaseTable<?> getTable();
+
+  /**
+   * Validates the required parameters to read the entities.
+   */
+  protected abstract void validateParams();
+
+  /**
+   * Sets certain parameters to defaults if the values are not provided.
+   */
+  protected abstract void augmentParams(Configuration hbaseConf,
+      Connection conn) throws IOException;
+
+  /**
+   * Fetches a {@link Result} instance for a single-entity read.
+   *
+   * @return the {@link Result} instance or null if no such record is found.
+   */
+  protected abstract Result getResult(Configuration hbaseConf, Connection conn)
+      throws IOException;
+
+  /**
+   * Fetches an iterator for {@link Result} instances for a multi-entity read.
+   */
+  protected abstract Iterable<Result> getResults(Configuration hbaseConf,
+      Connection conn) throws IOException;
+
+  /**
+   * Given a {@link Result} instance, deserializes and creates a
+   * {@link TimelineEntity}.
+   *
+   * @return the {@link TimelineEntity} instance, or null if the {@link Result}
+   * is null or empty.
+   */
+  protected abstract TimelineEntity parseEntity(Result result)
+      throws IOException;
+
+  /**
+   * Helper method for reading and deserializing {@link TimelineMetric} objects
+   * using the specified column prefix. The timeline metrics then are added to
+   * the given timeline entity.
+   */
+  protected void readMetrics(TimelineEntity entity, Result result,
+      ColumnPrefix<?> columnPrefix) throws IOException {
+    NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
+        columnPrefix.readResultsWithTimestamps(result);
+    for (Map.Entry<String, NavigableMap<Long, Number>> metricResult:
+        metricsResult.entrySet()) {
+      TimelineMetric metric = new TimelineMetric();
+      metric.setId(metricResult.getKey());
+      // Simply assume that if the value set contains more than 1 elements, the
+      // metric is a TIME_SERIES metric, otherwise, it's a SINGLE_VALUE metric
+      metric.setType(metricResult.getValue().size() > 1 ?
+          TimelineMetric.Type.TIME_SERIES : TimelineMetric.Type.SINGLE_VALUE);
+      metric.addValues(metricResult.getValue());
+      entity.addMetric(metric);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.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/TimelineEntityReaderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.java
new file mode 100644
index 0000000..4fdef40
--- /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/TimelineEntityReaderFactory.java
@@ -0,0 +1,97 @@
+/**
+ * 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;
+
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
+
+/**
+ * Factory methods for instantiating a timeline entity reader.
+ */
+class TimelineEntityReaderFactory {
+  /**
+   * Creates a timeline entity reader instance for reading a single entity with
+   * the specified input.
+   */
+  public static TimelineEntityReader createSingleEntityReader(String userId,
+      String clusterId, String flowId, Long flowRunId, String appId,
+      String entityType, String entityId, EnumSet<Field> fieldsToRetrieve) {
+    // currently the types that are handled separate from the generic entity
+    // table are application, flow run, and flow activity entities
+    if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) {
+      return new ApplicationEntityReader(userId, clusterId, flowId, flowRunId,
+          appId, entityType, entityId, fieldsToRetrieve);
+    } else if (TimelineEntityType.YARN_FLOW_RUN.matches(entityType)) {
+      return new FlowRunEntityReader(userId, clusterId, flowId, flowRunId,
+          appId, entityType, entityId, fieldsToRetrieve);
+    } else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) {
+      return new FlowActivityEntityReader(userId, clusterId, flowId, flowRunId,
+          appId, entityType, entityId, fieldsToRetrieve);
+    } else {
+      // assume we're dealing with a generic entity read
+      return new GenericEntityReader(userId, clusterId, flowId, flowRunId,
+        appId, entityType, entityId, fieldsToRetrieve);
+    }
+  }
+
+  /**
+   * Creates a timeline entity reader instance for reading set of entities with
+   * the specified input and predicates.
+   */
+  public static TimelineEntityReader createMultipleEntitiesReader(String userId,
+      String clusterId, String flowId, Long flowRunId, String appId,
+      String entityType, Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      EnumSet<Field> fieldsToRetrieve) {
+    // currently the types that are handled separate from the generic entity
+    // table are application, flow run, and flow activity entities
+    if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) {
+      return new ApplicationEntityReader(userId, clusterId, flowId, flowRunId,
+          appId, entityType, limit, createdTimeBegin, createdTimeEnd,
+          modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
+          infoFilters, configFilters, metricFilters, eventFilters,
+          fieldsToRetrieve);
+    } else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) {
+      return new FlowActivityEntityReader(userId, clusterId, flowId, flowRunId,
+          appId, entityType, limit, createdTimeBegin, createdTimeEnd,
+          modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
+          infoFilters, configFilters, metricFilters, eventFilters,
+          fieldsToRetrieve);
+    } else if (TimelineEntityType.YARN_FLOW_RUN.matches(entityType)) {
+      return new FlowRunEntityReader(userId, clusterId, flowId, flowRunId,
+          appId, entityType, limit, createdTimeBegin, createdTimeEnd,
+          modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
+          infoFilters, configFilters, metricFilters, eventFilters,
+          fieldsToRetrieve);
+    } else {
+      // assume we're dealing with a generic entity read
+      return new GenericEntityReader(userId, clusterId, flowId, flowRunId,
+          appId, entityType, limit, createdTimeBegin, createdTimeEnd,
+          modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
+          infoFilters, configFilters, metricFilters, eventFilters,
+          fieldsToRetrieve);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java
index 5f3868b..e3b5a87 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java
@@ -19,14 +19,46 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.application;
 
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
 
 /**
  * Represents a rowkey for the application table.
  */
 public class ApplicationRowKey {
-  // TODO: more methods are needed for this class.
+  private final String clusterId;
+  private final String userId;
+  private final String flowId;
+  private final long flowRunId;
+  private final String appId;
 
-  // TODO: API needs to be cleaned up.
+  public ApplicationRowKey(String clusterId, String userId, String flowId,
+      long flowRunId, String appId) {
+    this.clusterId = clusterId;
+    this.userId = userId;
+    this.flowId = flowId;
+    this.flowRunId = flowRunId;
+    this.appId = appId;
+  }
+
+  public String getClusterId() {
+    return clusterId;
+  }
+
+  public String getUserId() {
+    return userId;
+  }
+
+  public String getFlowId() {
+    return flowId;
+  }
+
+  public long getFlowRunId() {
+    return flowRunId;
+  }
+
+  public String getAppId() {
+    return appId;
+  }
 
   /**
    * Constructs a row key for the application table as follows:
@@ -46,22 +78,32 @@ public class ApplicationRowKey {
             flowId));
     // Note that flowRunId is a long, so we can't encode them all at the same
     // time.
-    byte[] second = Bytes.toBytes(ApplicationRowKey.invert(flowRunId));
+    byte[] second = Bytes.toBytes(TimelineWriterUtils.invert(flowRunId));
     byte[] third = Bytes.toBytes(appId);
     return Separator.QUALIFIERS.join(first, second, third);
   }
 
   /**
-   * Converts a timestamp into its inverse timestamp to be used in (row) keys
-   * where we want to have the most recent timestamp in the top of the table
-   * (scans start at the most recent timestamp first).
-   *
-   * @param key value to be inverted so that the latest version will be first in
-   *          a scan.
-   * @return inverted long
+   * Given the raw row key as bytes, returns the row key as an object.
    */
-  public static long invert(Long key) {
-    return Long.MAX_VALUE - key;
-  }
+  public static ApplicationRowKey parseRowKey(byte[] rowKey) {
+    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
+
+    if (rowKeyComponents.length < 5) {
+      throw new IllegalArgumentException("the row key is not valid for " +
+          "an application");
+    }
 
+    String clusterId =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[0]));
+    String userId =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[1]));
+    String flowId =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[2]));
+    long flowRunId =
+        TimelineWriterUtils.invert(Bytes.toLong(rowKeyComponents[3]));
+    String appId =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[4]));
+    return new ApplicationRowKey(clusterId, userId, flowId, flowRunId, appId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/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 ad4fec6..ca88056 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
@@ -24,6 +24,22 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
  * Represents a rowkey for the app_flow table.
  */
 public class AppToFlowRowKey {
+  private final String clusterId;
+  private final String appId;
+
+  public AppToFlowRowKey(String clusterId, String appId) {
+    this.clusterId = clusterId;
+    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}
@@ -36,4 +52,19 @@ public class AppToFlowRowKey {
     return Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId, appId));
   }
 
+  /**
+   * Given the raw row key as bytes, returns the row key as an object.
+   */
+  public static AppToFlowRowKey parseRowKey(byte[] rowKey) {
+    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
+
+    if (rowKeyComponents.length < 2) {
+      throw new IllegalArgumentException("the row key is not valid for " +
+          "the app-to-flow table");
+    }
+
+    String clusterId = Bytes.toString(rowKeyComponents[0]);
+    String appId = Bytes.toString(rowKeyComponents[1]);
+    return new AppToFlowRowKey(clusterId, appId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.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/BaseTable.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/BaseTable.java
index abba79a..9545438 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/BaseTable.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/BaseTable.java
@@ -31,7 +31,8 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 
 /**
- * Implements behavior common to tables used in the timeline service storage.
+ * Implements behavior common to tables used in the timeline service storage. It
+ * is thread-safe, and can be used by multiple threads concurrently.
  *
  * @param <T> reference to the table instance class itself for type safety.
  */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java
index 9a72be0..6a534ed73 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage.entity;
 
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
 
@@ -26,9 +25,52 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWrit
  * Represents a rowkey for the entity table.
  */
 public class EntityRowKey {
-  // TODO: more methods are needed for this class.
+  private final String clusterId;
+  private final String userId;
+  private final String flowId;
+  private final long flowRunId;
+  private final String appId;
+  private final String entityType;
+  private final String entityId;
 
-  // TODO: API needs to be cleaned up.
+  public EntityRowKey(String clusterId, String userId, String flowId,
+      long flowRunId, String appId, String entityType, String entityId) {
+    this.clusterId = clusterId;
+    this.userId = userId;
+    this.flowId = flowId;
+    this.flowRunId = flowRunId;
+    this.appId = appId;
+    this.entityType = entityType;
+    this.entityId = entityId;
+  }
+
+  public String getClusterId() {
+    return clusterId;
+  }
+
+  public String getUserId() {
+    return userId;
+  }
+
+  public String getFlowId() {
+    return flowId;
+  }
+
+  public long getFlowRunId() {
+    return flowRunId;
+  }
+
+  public String getAppId() {
+    return appId;
+  }
+
+  public String getEntityType() {
+    return entityType;
+  }
+
+  public String getEntityId() {
+    return entityId;
+  }
 
   /**
    * Constructs a row key prefix for the entity table as follows:
@@ -106,4 +148,32 @@ public class EntityRowKey {
     return Separator.QUALIFIERS.join(first, second, third);
   }
 
+  /**
+   * Given the raw row key as bytes, returns the row key as an object.
+   */
+  public static EntityRowKey parseRowKey(byte[] rowKey) {
+    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
+
+    if (rowKeyComponents.length < 7) {
+      throw new IllegalArgumentException("the row key is not valid for " +
+          "an entity");
+    }
+
+    String userId =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[0]));
+    String clusterId =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[1]));
+    String flowId =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[2]));
+    long flowRunId =
+        TimelineWriterUtils.invert(Bytes.toLong(rowKeyComponents[3]));
+    String appId =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[4]));
+    String entityType =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[5]));
+    String entityId =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[6]));
+    return new EntityRowKey(clusterId, userId, flowId, flowRunId, appId,
+        entityType, entityId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
index 19e4e83..18ca599 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
@@ -55,6 +55,10 @@ public class FlowActivityRowKey {
     return flowId;
   }
 
+  public static byte[] getRowKeyPrefix(String clusterId) {
+    return Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId, ""));
+  }
+
   /**
    * Constructs a row key for the flow activity table as follows:
    * {@code clusterId!dayTimestamp!user!flowId}
@@ -65,7 +69,8 @@ public class FlowActivityRowKey {
    * @param flowId
    * @return byte array with the row key prefix
    */
-  public static byte[] getRowKey(String clusterId, String userId, String flowId) {
+  public static byte[] getRowKey(String clusterId, String userId,
+      String flowId) {
     long dayTs = TimelineWriterUtils.getTopOfTheDayTimestamp(System
         .currentTimeMillis());
     return getRowKey(clusterId, dayTs, userId, flowId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java
index e133241..880d481 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java
@@ -25,7 +25,34 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWrit
  * Represents a rowkey for the flow run table.
  */
 public class FlowRunRowKey {
-  // TODO: more methods are needed for this class like parse row key
+  private final String clusterId;
+  private final String userId;
+  private final String flowId;
+  private final long flowRunId;
+
+  public FlowRunRowKey(String clusterId, String userId, String flowId,
+      long flowRunId) {
+    this.clusterId = clusterId;
+    this.userId = userId;
+    this.flowId = flowId;
+    this.flowRunId = flowRunId;
+  }
+
+  public String getClusterId() {
+    return clusterId;
+  }
+
+  public String getUserId() {
+    return userId;
+  }
+
+  public String getFlowId() {
+    return flowId;
+  }
+
+  public long getFlowRunId() {
+    return flowRunId;
+  }
 
   /**
    * Constructs a row key for the entity table as follows: {
@@ -47,4 +74,25 @@ public class FlowRunRowKey {
     return Separator.QUALIFIERS.join(first, second);
   }
 
+  /**
+   * Given the raw row key as bytes, returns the row key as an object.
+   */
+  public static FlowRunRowKey parseRowKey(byte[] rowKey) {
+    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
+
+    if (rowKeyComponents.length < 4) {
+      throw new IllegalArgumentException("the row key is not valid for " +
+          "a flow run");
+    }
+
+    String clusterId =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[0]));
+    String userId =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[1]));
+    String flowId =
+        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[2]));
+    long flowRunId =
+        TimelineWriterUtils.invert(Bytes.toLong(rowKeyComponents[3]));
+    return new FlowRunRowKey(clusterId, userId, flowId, flowRunId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.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/flow/FlowScanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
index a1948aa..651bb3a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
@@ -18,6 +18,15 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
@@ -33,15 +42,6 @@ import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
 import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
 
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
 /**
  * Invoked via the coprocessor when a Get or a Scan is issued for flow run
  * table. Looks through the list of cells per row, checks their tags and does

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.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/TestHBaseTimelineStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index 3962341..01920b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -508,32 +508,28 @@ public class TestHBaseTimelineStorage {
   private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user,
       String flow, long runid, String appName, TimelineEntity te) {
 
-    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
-
-    assertTrue(rowKeyComponents.length == 7);
-    assertEquals(user, Bytes.toString(rowKeyComponents[0]));
-    assertEquals(cluster, Bytes.toString(rowKeyComponents[1]));
-    assertEquals(flow, Bytes.toString(rowKeyComponents[2]));
-    assertEquals(TimelineWriterUtils.invert(runid),
-        Bytes.toLong(rowKeyComponents[3]));
-    assertEquals(appName, Bytes.toString(rowKeyComponents[4]));
-    assertEquals(te.getType(), Bytes.toString(rowKeyComponents[5]));
-    assertEquals(te.getId(), Bytes.toString(rowKeyComponents[6]));
+    EntityRowKey key = EntityRowKey.parseRowKey(rowKey);
+
+    assertEquals(user, key.getUserId());
+    assertEquals(cluster, key.getClusterId());
+    assertEquals(flow, key.getFlowId());
+    assertEquals(runid, key.getFlowRunId());
+    assertEquals(appName, key.getAppId());
+    assertEquals(te.getType(), key.getEntityType());
+    assertEquals(te.getId(), key.getEntityId());
     return true;
   }
 
   private boolean isApplicationRowKeyCorrect(byte[] rowKey, String cluster,
       String user, String flow, long runid, String appName) {
 
-    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
+    ApplicationRowKey key = ApplicationRowKey.parseRowKey(rowKey);
 
-    assertTrue(rowKeyComponents.length == 5);
-    assertEquals(cluster, Bytes.toString(rowKeyComponents[0]));
-    assertEquals(user, Bytes.toString(rowKeyComponents[1]));
-    assertEquals(flow, Bytes.toString(rowKeyComponents[2]));
-    assertEquals(TimelineWriterUtils.invert(runid),
-        Bytes.toLong(rowKeyComponents[3]));
-    assertEquals(appName, Bytes.toString(rowKeyComponents[4]));
+    assertEquals(cluster, key.getClusterId());
+    assertEquals(user, key.getUserId());
+    assertEquals(flow, key.getFlowId());
+    assertEquals(runid, key.getFlowRunId());
+    assertEquals(appName, key.getAppId());
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.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/flow/TestFlowDataGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java
index f8331fa..d18613a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java
@@ -45,7 +45,7 @@ class TestFlowDataGenerator {
     String type = TimelineEntityType.YARN_APPLICATION.toString();
     entity.setId(id);
     entity.setType(type);
-    Long cTime = 1425016501000L;
+    long cTime = 1425016501000L;
     entity.setCreatedTime(cTime);
 
     // add metrics
@@ -54,8 +54,8 @@ class TestFlowDataGenerator {
     m1.setId(metric1);
     Map<Long, Number> metricValues = new HashMap<Long, Number>();
     long ts = System.currentTimeMillis();
-    metricValues.put(ts - 100000, 2);
-    metricValues.put(ts - 80000, 40);
+    metricValues.put(ts - 100000, 2L);
+    metricValues.put(ts - 80000, 40L);
     m1.setType(Type.TIME_SERIES);
     m1.setValues(metricValues);
     metrics.add(m1);
@@ -64,8 +64,8 @@ class TestFlowDataGenerator {
     m2.setId(metric2);
     metricValues = new HashMap<Long, Number>();
     ts = System.currentTimeMillis();
-    metricValues.put(ts - 100000, 31);
-    metricValues.put(ts - 80000, 57);
+    metricValues.put(ts - 100000, 31L);
+    metricValues.put(ts - 80000, 57L);
     m2.setType(Type.TIME_SERIES);
     m2.setValues(metricValues);
     metrics.add(m2);
@@ -80,7 +80,7 @@ class TestFlowDataGenerator {
     String type = TimelineEntityType.YARN_APPLICATION.toString();
     entity.setId(id);
     entity.setType(type);
-    Long cTime = 1425016501000L;
+    long cTime = 1425016501000L;
     entity.setCreatedTime(cTime);
     // add metrics
     Set<TimelineMetric> metrics = new HashSet<>();
@@ -103,8 +103,8 @@ class TestFlowDataGenerator {
     String type = TimelineEntityType.YARN_APPLICATION.toString();
     entity.setId(id);
     entity.setType(type);
-    Long cTime = 20000000000000L;
-    Long mTime = 1425026901000L;
+    long cTime = 20000000000000L;
+    long mTime = 1425026901000L;
     entity.setCreatedTime(cTime);
     entity.setModifiedTime(mTime);
     // add metrics
@@ -113,10 +113,10 @@ class TestFlowDataGenerator {
     m1.setId(metric1);
     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 - 120000, 100000000L);
+    metricValues.put(ts - 100000, 200000000L);
+    metricValues.put(ts - 80000, 300000000L);
+    metricValues.put(ts - 60000, 400000000L);
     metricValues.put(ts - 40000, 50000000000L);
     metricValues.put(ts - 20000, 60000000000L);
     m1.setType(Type.TIME_SERIES);
@@ -126,7 +126,7 @@ class TestFlowDataGenerator {
 
     TimelineEvent event = new TimelineEvent();
     event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    Long expTs = 1436512802000L;
+    long expTs = 1436512802000L;
     event.setTimestamp(expTs);
     String expKey = "foo_event";
     Object expVal = "test";
@@ -142,9 +142,9 @@ class TestFlowDataGenerator {
     return entity;
   }
 
-  static TimelineEntity getEntityGreaterStartTime() {
+  static TimelineEntity getEntityGreaterStartTime(long startTs) {
     TimelineEntity entity = new TimelineEntity();
-    entity.setCreatedTime(30000000000000L);
+    entity.setCreatedTime(startTs);
     entity.setId("flowRunHello with greater start time");
     String type = TimelineEntityType.YARN_APPLICATION.toString();
     entity.setType(type);
@@ -173,14 +173,13 @@ class TestFlowDataGenerator {
     return entity;
   }
 
-  static TimelineEntity getEntityMinStartTime() {
+  static TimelineEntity getEntityMinStartTime(long startTs) {
     TimelineEntity entity = new TimelineEntity();
     String id = "flowRunHelloMInStartTime";
     String type = TimelineEntityType.YARN_APPLICATION.toString();
     entity.setId(id);
     entity.setType(type);
-    Long cTime = 10000000000000L;
-    entity.setCreatedTime(cTime);
+    entity.setCreatedTime(startTs);
     TimelineEvent event = new TimelineEvent();
     event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
     event.setTimestamp(System.currentTimeMillis());
@@ -195,12 +194,12 @@ class TestFlowDataGenerator {
     String type = TimelineEntityType.YARN_APPLICATION.toString();
     entity.setId(id);
     entity.setType(type);
-    Long cTime = 1425016501000L;
+    long cTime = 1425016501000L;
     entity.setCreatedTime(cTime);
 
     TimelineEvent event = new TimelineEvent();
     event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    Long expTs = 1436512802000L;
+    long expTs = 1436512802000L;
     event.setTimestamp(expTs);
     String expKey = "foo_event";
     Object expVal = "test";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
index b4a0c74..6bdec6b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
@@ -21,19 +21,16 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 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 java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
+import java.util.NavigableSet;
 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.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
-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.hbase.client.Get;
@@ -42,26 +39,17 @@ 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.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.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.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl;
 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.application.ApplicationTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnFamily;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -119,11 +107,13 @@ public class TestHBaseStorageFlowActivity {
     String user = "testWriteFlowRunMinMaxToHBase_user1";
     String flow = "testing_flowRun_flow_name";
     String flowVersion = "CF7022C10F1354";
-    Long runid = 1002345678919L;
+    long runid = 1002345678919L;
     String appName = "application_100000000000_1111";
+    long minStartTs = 10000000000000L;
+    long greaterStartTs = 30000000000000L;
     long endTs = 1439750690000L;
     TimelineEntity entityMinStartTime = TestFlowDataGenerator
-        .getEntityMinStartTime();
+        .getEntityMinStartTime(minStartTs);
 
     try {
       hbi = new HBaseTimelineWriterImpl(c1);
@@ -146,7 +136,7 @@ public class TestHBaseStorageFlowActivity {
 
       // writer another entity with greater start time
       TimelineEntity entityGreaterStartTime = TestFlowDataGenerator
-          .getEntityGreaterStartTime();
+          .getEntityGreaterStartTime(greaterStartTs);
       te = new TimelineEntities();
       te.addEntity(entityGreaterStartTime);
       appName = "application_1000000000000000_2222";
@@ -181,6 +171,31 @@ public class TestHBaseStorageFlowActivity {
     assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
     assertEquals(1, values.size());
     checkFlowActivityRunId(runid, flowVersion, values);
+
+    // use the timeline reader to verify data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+      // get the flow activity entity
+      Set<TimelineEntity> entities =
+          hbr.getEntities(null, cluster, null, null, null,
+              TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), 10L, null, null,
+              null, null, null, null, null, null, null, null, null);
+      assertEquals(1, entities.size());
+      for (TimelineEntity e : entities) {
+        FlowActivityEntity flowActivity = (FlowActivityEntity)e;
+        assertEquals(cluster, flowActivity.getCluster());
+        assertEquals(user, flowActivity.getUser());
+        assertEquals(flow, flowActivity.getFlowName());
+        assertEquals(dayTs, flowActivity.getDate().getTime());
+        Set<FlowRunEntity> flowRuns = flowActivity.getFlowRuns();
+        assertEquals(1, flowRuns.size());
+      }
+    } finally {
+      hbr.close();
+    }
   }
 
   /**
@@ -193,7 +208,7 @@ public class TestHBaseStorageFlowActivity {
     String user = "testWriteFlowActivityOneFlow_user1";
     String flow = "flow_activity_test_flow_name";
     String flowVersion = "A122110F135BC4";
-    Long runid = 1001111178919L;
+    long runid = 1001111178919L;
 
     TimelineEntities te = new TimelineEntities();
     TimelineEntity entityApp1 = TestFlowDataGenerator.getFlowApp1();
@@ -212,10 +227,35 @@ public class TestHBaseStorageFlowActivity {
     }
     // check flow activity
     checkFlowActivityTable(cluster, user, flow, flowVersion, runid, c1);
+
+    // use the reader to verify the data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+
+      Set<TimelineEntity> entities =
+          hbr.getEntities(user, cluster, flow, null, null,
+              TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), 10L, null, null,
+              null, null, null, null, null, null, null, null, null);
+      assertEquals(1, entities.size());
+      for (TimelineEntity e : entities) {
+        FlowActivityEntity entity = (FlowActivityEntity)e;
+        NavigableSet<FlowRunEntity> flowRuns = entity.getFlowRuns();
+        assertEquals(1, flowRuns.size());
+        for (FlowRunEntity flowRun : flowRuns) {
+          assertEquals(runid, flowRun.getRunId());
+          assertEquals(flowVersion, flowRun.getVersion());
+        }
+      }
+    } finally {
+      hbr.close();
+    }
   }
 
   private void checkFlowActivityTable(String cluster, String user, String flow,
-      String flowVersion, Long runid, Configuration c1) throws IOException {
+      String flowVersion, long runid, Configuration c1) throws IOException {
     Scan s = new Scan();
     s.addFamily(FlowActivityColumnFamily.INFO.getBytes());
     byte[] startRow = FlowActivityRowKey.getRowKey(cluster, user, flow);
@@ -263,7 +303,7 @@ public class TestHBaseStorageFlowActivity {
     String user = "testManyRunsFlowActivity_c_user1";
     String flow = "flow_activity_test_flow_name";
     String flowVersion1 = "A122110F135BC4";
-    Long runid1 = 11111111111L;
+    long runid1 = 11111111111L;
 
     String flowVersion2 = "A12222222222C4";
     long runid2 = 2222222222222L;
@@ -303,11 +343,50 @@ public class TestHBaseStorageFlowActivity {
     checkFlowActivityTableSeveralRuns(cluster, user, flow, c1, flowVersion1,
         runid1, flowVersion2, runid2, flowVersion3, runid3);
 
+    // use the timeline reader to verify data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+
+      Set<TimelineEntity> entities =
+          hbr.getEntities(null, cluster, null, null, null,
+              TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), 10L, null, null,
+              null, null, null, null, null, null, null, null, null);
+      assertEquals(1, entities.size());
+      for (TimelineEntity e : entities) {
+        FlowActivityEntity flowActivity = (FlowActivityEntity)e;
+        assertEquals(cluster, flowActivity.getCluster());
+        assertEquals(user, flowActivity.getUser());
+        assertEquals(flow, flowActivity.getFlowName());
+        long dayTs = TimelineWriterUtils.getTopOfTheDayTimestamp(System
+            .currentTimeMillis());
+        assertEquals(dayTs, flowActivity.getDate().getTime());
+        Set<FlowRunEntity> flowRuns = flowActivity.getFlowRuns();
+        assertEquals(3, flowRuns.size());
+        for (FlowRunEntity flowRun : flowRuns) {
+          long runId = flowRun.getRunId();
+          String version = flowRun.getVersion();
+          if (runId == runid1) {
+            assertEquals(flowVersion1, version);
+          } else if (runId == runid2) {
+            assertEquals(flowVersion2, version);
+          } else if (runId == runid3) {
+            assertEquals(flowVersion3, version);
+          } else {
+            fail("unknown run id: " + runId);
+          }
+        }
+      }
+    } finally {
+      hbr.close();
+    }
   }
 
   private void checkFlowActivityTableSeveralRuns(String cluster, String user,
-      String flow, Configuration c1, String flowVersion1, Long runid1,
-      String flowVersion2, Long runid2, String flowVersion3, Long runid3)
+      String flow, Configuration c1, String flowVersion1, long runid1,
+      String flowVersion2, long runid2, String flowVersion3, long runid3)
       throws IOException {
     Scan s = new Scan();
     s.addFamily(FlowActivityColumnFamily.INFO.getBytes());
@@ -351,7 +430,7 @@ public class TestHBaseStorageFlowActivity {
     assertEquals(1, rowCount);
   }
 
-  private void checkFlowActivityRunId(Long runid, String flowVersion,
+  private void checkFlowActivityRunId(long runid, String flowVersion,
       Map<byte[], byte[]> values) throws IOException {
     byte[] rq = ColumnHelper.getColumnQualifier(
         FlowActivityColumnPrefix.RUN_ID.getColumnPrefixBytes(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/25f58b40/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
index bf524ea..b0f83b7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
@@ -21,20 +21,15 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 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 java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
 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.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
-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.hbase.client.Get;
@@ -42,32 +37,16 @@ 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.client.Table;
-import org.apache.hadoop.hbase.util.Bytes;
+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.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.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl;
 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.application.ApplicationTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
 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.TimelineWriterUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnFamily;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnFamily;
-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.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -125,11 +104,13 @@ public class TestHBaseStorageFlowRun {
     String user = "testWriteFlowRunMinMaxToHBase_user1";
     String flow = "testing_flowRun_flow_name";
     String flowVersion = "CF7022C10F1354";
-    Long runid = 1002345678919L;
+    long runid = 1002345678919L;
     String appName = "application_100000000000_1111";
+    long minStartTs = 10000000000000L;
+    long greaterStartTs = 30000000000000L;
     long endTs = 1439750690000L;
     TimelineEntity entityMinStartTime = TestFlowDataGenerator
-        .getEntityMinStartTime();
+        .getEntityMinStartTime(minStartTs);
 
     try {
       hbi = new HBaseTimelineWriterImpl(c1);
@@ -152,7 +133,7 @@ public class TestHBaseStorageFlowRun {
 
       // writer another entity with greater start time
       TimelineEntity entityGreaterStartTime = TestFlowDataGenerator
-          .getEntityGreaterStartTime();
+          .getEntityGreaterStartTime(greaterStartTs);
       te = new TimelineEntities();
       te.addEntity(entityGreaterStartTime);
       appName = "application_1000000000000000_2222";
@@ -183,24 +164,29 @@ public class TestHBaseStorageFlowRun {
         .getBytes());
 
     assertEquals(2, r1.size());
-    Long starttime = (Long) GenericObjectMapper.read(values
+    long starttime = (Long) GenericObjectMapper.read(values
         .get(FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes()));
-    Long expmin = entityMinStartTime.getCreatedTime();
-    assertEquals(expmin, starttime);
+    assertEquals(minStartTs, starttime);
     assertEquals(endTs, GenericObjectMapper.read(values
         .get(FlowRunColumn.MAX_END_TIME.getColumnQualifierBytes())));
-  }
 
-  boolean isFlowRunRowKeyCorrect(byte[] rowKey, String cluster, String user,
-      String flow, Long runid) {
-    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey, -1);
-    assertTrue(rowKeyComponents.length == 4);
-    assertEquals(cluster, Bytes.toString(rowKeyComponents[0]));
-    assertEquals(user, Bytes.toString(rowKeyComponents[1]));
-    assertEquals(flow, Bytes.toString(rowKeyComponents[2]));
-    assertEquals(TimelineWriterUtils.invert(runid),
-        Bytes.toLong(rowKeyComponents[3]));
-    return true;
+    // use the timeline reader to verify data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+      // get the flow run entity
+      TimelineEntity entity =
+          hbr.getEntity(user, cluster, flow, runid, null,
+              TimelineEntityType.YARN_FLOW_RUN.toString(), null, null);
+      assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
+      FlowRunEntity flowRun = (FlowRunEntity)entity;
+      assertEquals(minStartTs, flowRun.getStartTime());
+      assertEquals(endTs, flowRun.getMaxEndTime());
+    } finally {
+      hbr.close();
+    }
   }
 
   /**
@@ -218,7 +204,7 @@ public class TestHBaseStorageFlowRun {
     String user = "testWriteFlowRunMetricsOneFlow_user1";
     String flow = "testing_flowRun_metrics_flow_name";
     String flowVersion = "CF7022C10F1354";
-    Long runid = 1002345678919L;
+    long runid = 1002345678919L;
 
     TimelineEntities te = new TimelineEntities();
     TimelineEntity entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1();
@@ -244,6 +230,41 @@ public class TestHBaseStorageFlowRun {
 
     // check flow run
     checkFlowRunTable(cluster, user, flow, runid, c1);
+
+    // use the timeline reader to verify data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+      TimelineEntity entity =
+          hbr.getEntity(user, cluster, flow, runid, null,
+            TimelineEntityType.YARN_FLOW_RUN.toString(), null, null);
+      assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
+      Set<TimelineMetric> metrics = entity.getMetrics();
+      assertEquals(2, metrics.size());
+      for (TimelineMetric metric : metrics) {
+        String id = metric.getId();
+        Map<Long, Number> values = metric.getValues();
+        assertEquals(1, values.size());
+        Number value = null;
+        for (Number n : values.values()) {
+          value = n;
+        }
+        switch (id) {
+        case metric1:
+          assertEquals(141, value);
+          break;
+        case metric2:
+          assertEquals(57, value);
+          break;
+        default:
+          fail("unrecognized metric: " + id);
+        }
+      }
+    } finally {
+      hbr.close();
+    }
   }
 
   private void checkFlowRunTable(String cluster, String user, String flow,


[42/50] [abbrv] hadoop git commit: YARN-4445. Unify the term flowId and flowName in timeline v2 codebase. Contributed by Zhan Zhang.

Posted by gt...@apache.org.
YARN-4445. Unify the term flowId and flowName in timeline v2 codebase.
Contributed by Zhan Zhang.


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

Branch: refs/heads/feature-YARN-2928
Commit: dfaa7c5f965cb2dd1728dbb82fdaeb2c643af14a
Parents: 5de8b14
Author: Li Lu <gt...@apache.org>
Authored: Wed Dec 16 16:42:57 2015 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 18:03:30 2016 -0800

----------------------------------------------------------------------
 .../mapred/TestMRTimelineEventHandling.java     |  2 +-
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../distributedshell/TestDistributedShell.java  |  2 +-
 .../yarn/util/timeline/TimelineUtils.java       |  2 +-
 .../TestSystemMetricsPublisherForV2.java        |  2 +-
 .../collector/AppLevelTimelineCollector.java    |  2 +-
 .../reader/TimelineReaderManager.java           |  8 +--
 .../reader/TimelineReaderWebServices.java       | 74 ++++++++++----------
 .../storage/ApplicationEntityReader.java        | 20 +++---
 .../storage/FileSystemTimelineReaderImpl.java   | 14 ++--
 .../storage/FlowActivityEntityReader.java       | 10 +--
 .../storage/FlowRunEntityReader.java            | 16 ++---
 .../storage/GenericEntityReader.java            | 22 +++---
 .../storage/HBaseTimelineReaderImpl.java        |  8 +--
 .../storage/TimelineEntityReader.java           | 10 +--
 .../storage/TimelineEntityReaderFactory.java    | 20 +++---
 .../timelineservice/storage/TimelineReader.java |  8 +--
 .../storage/application/ApplicationRowKey.java  | 38 +++++-----
 .../storage/application/ApplicationTable.java   |  2 +-
 .../apptoflow/AppToFlowColumnFamily.java        |  2 +-
 .../storage/apptoflow/AppToFlowTable.java       |  4 +-
 .../storage/entity/EntityRowKey.java            | 38 +++++-----
 .../storage/entity/EntityTable.java             |  2 +-
 .../storage/flow/FlowActivityRowKey.java        | 30 ++++----
 .../storage/flow/FlowActivityTable.java         |  2 +-
 .../storage/flow/FlowRunRowKey.java             | 30 ++++----
 .../storage/flow/FlowRunTable.java              |  2 +-
 .../reader/TestTimelineReaderWebServices.java   |  2 +-
 ...stTimelineReaderWebServicesHBaseStorage.java |  4 +-
 .../storage/TestHBaseTimelineStorage.java       |  4 +-
 .../flow/TestHBaseStorageFlowActivity.java      |  6 +-
 31 files changed, 196 insertions(+), 193 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
index cf6d64f..a9bbdf5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
@@ -244,7 +244,7 @@ public class TestMRTimelineEventHandling {
     Assert.assertTrue(tmpRootFolder.isDirectory());
     String basePath = tmpRoot + YarnConfiguration.DEFAULT_RM_CLUSTER_ID + "/" +
         UserGroupInformation.getCurrentUser().getShortUserName() +
-        "/" + TimelineUtils.generateDefaultFlowIdBasedOnAppId(appId) +
+        "/" + TimelineUtils.generateDefaultFlowNameBasedOnAppId(appId) +
         "/1/1/" + appId.toString();
     // for this test, we expect MAPREDUCE_JOB and MAPREDUCE_TASK dirs
     String outputDirJob = basePath + "/MAPREDUCE_JOB/";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1fc0bf6..169a907 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -146,6 +146,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3862. Support for fetching specific configs and metrics based on
     prefixes (Varun Saxena via sjlee)
 
+    YARN-4445. Unify the term flowId and flowName in timeline v2 codebase. 
+    (Zhan Zhang via gtcarrera9). 
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index b3ff9b5..c90f97b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -379,7 +379,7 @@ public class TestDistributedShell {
           YarnConfiguration.DEFAULT_RM_CLUSTER_ID + "/" +
           UserGroupInformation.getCurrentUser().getShortUserName() +
           (defaultFlow ? "/" +
-              TimelineUtils.generateDefaultFlowIdBasedOnAppId(appId) +
+              TimelineUtils.generateDefaultFlowNameBasedOnAppId(appId) +
               "/1/1/" : "/test_flow_name/test_flow_version/12345678/") +
               appId.toString();
       // for this test, we expect DS_APP_ATTEMPT AND DS_CONTAINER dirs

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
index 137b7c5..64aa7d4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/timeline/TimelineUtils.java
@@ -125,7 +125,7 @@ public class TimelineUtils {
     return SecurityUtil.buildTokenService(timelineServiceAddr);
   }
 
-  public static String generateDefaultFlowIdBasedOnAppId(ApplicationId appId) {
+  public static String generateDefaultFlowNameBasedOnAppId(ApplicationId appId) {
     return "flow_" + appId.getClusterTimestamp() + "_" + appId.getId();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/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 57258d5..9100366 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
@@ -307,7 +307,7 @@ public class TestSystemMetricsPublisherForV2 {
             + "/"
             + app.getUser()
             + "/"
-            + TimelineUtils.generateDefaultFlowIdBasedOnAppId(app
+            + TimelineUtils.generateDefaultFlowNameBasedOnAppId(app
                 .getApplicationId()) + "/" + DEFAULT_FLOW_VERSION + "/"
             + DEFAULT_FLOW_RUN + "/" + app.getApplicationId();
     return outputDirApp;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/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 fa32211..b148491 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
@@ -55,7 +55,7 @@ public class AppLevelTimelineCollector extends TimelineCollector {
     // Current user usually is not the app user, but keep this field non-null
     context.setUserId(UserGroupInformation.getCurrentUser().getShortUserName());
     // Use app ID to generate a default flow name for orphan app
-    context.setFlowName(TimelineUtils.generateDefaultFlowIdBasedOnAppId(appId));
+    context.setFlowName(TimelineUtils.generateDefaultFlowNameBasedOnAppId(appId));
     // Set the flow version to string 1 if it's an orphan app
     context.setFlowVersion("1");
     // Set the flow run ID to 1 if it's an orphan app

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/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 294b05b..37f1671 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
@@ -66,7 +66,7 @@ public class TimelineReaderManager extends AbstractService {
    * @see TimelineReader#getEntities
    */
   Set<TimelineEntity> getEntities(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       Long limit, Long createdTimeBegin, Long createdTimeEnd,
       Long modifiedTimeBegin, Long modifiedTimeEnd,
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
@@ -74,7 +74,7 @@ public class TimelineReaderManager extends AbstractService {
       Set<String>  metricFilters, Set<String> eventFilters,
       EnumSet<Field> fieldsToRetrieve) throws IOException {
     String cluster = getClusterID(clusterId, getConfig());
-    return reader.getEntities(userId, cluster, flowId, flowRunId, appId,
+    return reader.getEntities(userId, cluster, flowName, flowRunId, appId,
         entityType, limit, createdTimeBegin, createdTimeEnd, modifiedTimeBegin,
         modifiedTimeEnd, relatesTo, isRelatedTo, infoFilters, configFilters,
         metricFilters, eventFilters, null, null, fieldsToRetrieve);
@@ -87,10 +87,10 @@ public class TimelineReaderManager extends AbstractService {
    * @see TimelineReader#getEntity
    */
   public TimelineEntity getEntity(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       String entityId, EnumSet<Field> fields) throws IOException {
     String cluster = getClusterID(clusterId, getConfig());
-    return reader.getEntity(userId, cluster, flowId, flowRunId, appId,
+    return reader.getEntity(userId, cluster, flowName, flowRunId, appId,
         entityType, entityId, null, null, fields);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/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 d3ff8b7..f0283a5 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
@@ -326,7 +326,7 @@ public class TimelineReaderWebServices {
       @PathParam("appid") String appId,
       @PathParam("entitytype") String entityType,
       @QueryParam("userid") String userId,
-      @QueryParam("flowid") String flowId,
+      @QueryParam("flowname") String flowName,
       @QueryParam("flowrunid") String flowRunId,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
@@ -340,7 +340,7 @@ public class TimelineReaderWebServices {
       @QueryParam("metricfilters") String metricfilters,
       @QueryParam("eventfilters") String eventfilters,
        @QueryParam("fields") String fields) {
-    return getEntities(req, res, null, appId, entityType, userId, flowId,
+    return getEntities(req, res, null, appId, entityType, userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, modifiedTimeStart,
         modifiedTimeEnd, relatesTo, isRelatedTo, infofilters, conffilters,
         metricfilters, eventfilters, fields);
@@ -359,7 +359,7 @@ public class TimelineReaderWebServices {
       @PathParam("appid") String appId,
       @PathParam("entitytype") String entityType,
       @QueryParam("userid") String userId,
-      @QueryParam("flowid") String flowId,
+      @QueryParam("flowname") String flowName,
       @QueryParam("flowrunid") String flowRunId,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
@@ -384,7 +384,7 @@ public class TimelineReaderWebServices {
     Set<TimelineEntity> entities = null;
     try {
       entities = timelineReaderManager.getEntities(
-          parseStr(userId), parseStr(clusterId), parseStr(flowId),
+          parseStr(userId), parseStr(clusterId), parseStr(flowName),
           parseLongStr(flowRunId), parseStr(appId), parseStr(entityType),
           parseLongStr(limit), parseLongStr(createdTimeStart),
           parseLongStr(createdTimeEnd), parseLongStr(modifiedTimeStart),
@@ -423,11 +423,11 @@ public class TimelineReaderWebServices {
       @PathParam("entitytype") String entityType,
       @PathParam("entityid") String entityId,
       @QueryParam("userid") String userId,
-      @QueryParam("flowid") String flowId,
+      @QueryParam("flowname") String flowName,
       @QueryParam("flowrunid") String flowRunId,
       @QueryParam("fields") String fields) {
     return getEntity(req, res, null, appId, entityType, entityId, userId,
-        flowId, flowRunId, fields);
+        flowName, flowRunId, fields);
   }
 
   /**
@@ -444,7 +444,7 @@ public class TimelineReaderWebServices {
       @PathParam("entitytype") String entityType,
       @PathParam("entityid") String entityId,
       @QueryParam("userid") String userId,
-      @QueryParam("flowid") String flowId,
+      @QueryParam("flowname") String flowName,
       @QueryParam("flowrunid") String flowRunId,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
@@ -458,7 +458,7 @@ public class TimelineReaderWebServices {
     TimelineEntity entity = null;
     try {
       entity = timelineReaderManager.getEntity(
-          parseStr(userId), parseStr(clusterId), parseStr(flowId),
+          parseStr(userId), parseStr(clusterId), parseStr(flowName),
           parseLongStr(flowRunId), parseStr(appId), parseStr(entityType),
           parseStr(entityId), parseFieldsStr(fields, COMMA_DELIMITER));
     } catch (Exception e) {
@@ -481,30 +481,30 @@ public class TimelineReaderWebServices {
    * Cluster ID is not provided by client so default cluster ID has to be taken.
    */
   @GET
-  @Path("/flowrun/{userid}/{flowid}/{flowrunid}/")
+  @Path("/flowrun/{userid}/{flowname}/{flowrunid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public TimelineEntity getFlowRun(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("userid") String userId,
-      @PathParam("flowid") String flowId,
+      @PathParam("flowname") String flowName,
       @PathParam("flowrunid") String flowRunId,
       @QueryParam("fields") String fields) {
-    return getFlowRun(req, res, userId, null, flowId, flowRunId, fields);
+    return getFlowRun(req, res, userId, null, flowName, flowRunId, fields);
   }
 
   /**
    * Return a single flow run for the given user, cluster, flow id and run id.
    */
   @GET
-  @Path("/flowrun/{userid}/{clusterid}/{flowid}/{flowrunid}/")
+  @Path("/flowrun/{userid}/{clusterid}/{flowname}/{flowrunid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public TimelineEntity getFlowRun(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("userid") String userId,
       @PathParam("clusterid") String clusterId,
-      @PathParam("flowid") String flowId,
+      @PathParam("flowname") String flowName,
       @PathParam("flowrunid") String flowRunId,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
@@ -518,7 +518,7 @@ public class TimelineReaderWebServices {
     TimelineEntity entity = null;
     try {
       entity = timelineReaderManager.getEntity(parseStr(userId),
-          parseStr(clusterId), parseStr(flowId), parseLongStr(flowRunId), null,
+          parseStr(clusterId), parseStr(flowName), parseLongStr(flowRunId), null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null,
           parseFieldsStr(fields, COMMA_DELIMITER));
     } catch (Exception e) {
@@ -528,7 +528,7 @@ public class TimelineReaderWebServices {
     if (entity == null) {
       LOG.info("Processed URL " + url + " but flowrun not found (Took " +
           (endTime - startTime) + " ms.)");
-      throw new NotFoundException("Flow run {flow id: " + parseStr(flowId) +
+      throw new NotFoundException("Flow run {flow id: " + parseStr(flowName) +
           ", run id: " + parseLongStr(flowRunId) + " } is not found");
     }
     LOG.info("Processed URL " + url +
@@ -541,18 +541,18 @@ public class TimelineReaderWebServices {
    * Cluster ID is not provided by client so default cluster ID has to be taken.
    */
   @GET
-  @Path("/flowruns/{userid}/{flowid}/")
+  @Path("/flowruns/{userid}/{flowname}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getFlowRuns(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("userid") String userId,
-      @PathParam("flowid") String flowId,
+      @PathParam("flowname") String flowName,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
       @QueryParam("fields") String fields) {
-    return getFlowRuns(req, res, userId, null, flowId, limit, createdTimeStart,
+    return getFlowRuns(req, res, userId, null, flowName, limit, createdTimeStart,
         createdTimeEnd, fields);
   }
 
@@ -560,14 +560,14 @@ public class TimelineReaderWebServices {
    * Return a set of flow runs for the given user, cluster and flow id.
    */
   @GET
-  @Path("/flowruns/{userid}/{clusterid}/{flowid}/")
+  @Path("/flowruns/{userid}/{clusterid}/{flowname}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getFlowRuns(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("userid") String userId,
       @PathParam("clusterid") String clusterId,
-      @PathParam("flowid") String flowId,
+      @PathParam("flowname") String flowName,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
@@ -583,7 +583,7 @@ public class TimelineReaderWebServices {
     Set<TimelineEntity> entities = null;
     try {
       entities = timelineReaderManager.getEntities(
-          parseStr(userId), parseStr(clusterId), parseStr(flowId), null, null,
+          parseStr(userId), parseStr(clusterId), parseStr(flowName), null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), parseLongStr(limit),
           parseLongStr(createdTimeStart), parseLongStr(createdTimeEnd), null,
           null, null, null, null, null, null, null,
@@ -692,11 +692,11 @@ public class TimelineReaderWebServices {
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("appid") String appId,
-      @QueryParam("flowid") String flowId,
+      @QueryParam("flowname") String flowName,
       @QueryParam("flowrunid") String flowRunId,
       @QueryParam("userid") String userId,
       @QueryParam("fields") String fields) {
-    return getApp(req, res, null, appId, flowId, flowRunId, userId, fields);
+    return getApp(req, res, null, appId, flowName, flowRunId, userId, fields);
   }
 
   /**
@@ -710,7 +710,7 @@ public class TimelineReaderWebServices {
       @Context HttpServletResponse res,
       @PathParam("clusterid") String clusterId,
       @PathParam("appid") String appId,
-      @QueryParam("flowid") String flowId,
+      @QueryParam("flowname") String flowName,
       @QueryParam("flowrunid") String flowRunId,
       @QueryParam("userid") String userId,
       @QueryParam("fields") String fields) {
@@ -725,7 +725,7 @@ public class TimelineReaderWebServices {
     TimelineEntity entity = null;
     try {
       entity = timelineReaderManager.getEntity(parseStr(userId),
-          parseStr(clusterId), parseStr(flowId), parseLongStr(flowRunId),
+          parseStr(clusterId), parseStr(flowName), parseLongStr(flowRunId),
           parseStr(appId), TimelineEntityType.YARN_APPLICATION.toString(), null,
           parseFieldsStr(fields, COMMA_DELIMITER));
     } catch (Exception e) {
@@ -749,13 +749,13 @@ public class TimelineReaderWebServices {
    * is reached, will be returned.
    */
   @GET
-  @Path("/flowrunapps/{userid}/{flowid}/{flowrunid}/")
+  @Path("/flowrunapps/{userid}/{flowname}/{flowrunid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getFlowRunApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("userid") String userId,
-      @PathParam("flowid") String flowId,
+      @PathParam("flowname") String flowName,
       @PathParam("flowrunid") String flowRunId,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
@@ -770,7 +770,7 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("fields") String fields) {
     return getEntities(req, res, null, null,
-        TimelineEntityType.YARN_APPLICATION.toString(), userId, flowId,
+        TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, modifiedTimeStart,
         modifiedTimeEnd, relatesTo, isRelatedTo, infofilters, conffilters,
         metricfilters, eventfilters, fields);
@@ -782,14 +782,14 @@ public class TimelineReaderWebServices {
    * till the limit is reached, will be returned.
    */
   @GET
-  @Path("/flowrunapps/{userid}/{clusterid}/{flowid}/{flowrunid}/")
+  @Path("/flowrunapps/{userid}/{clusterid}/{flowname}/{flowrunid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getFlowRunApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("userid") String userId,
       @PathParam("clusterid") String clusterId,
-      @PathParam("flowid") String flowId,
+      @PathParam("flowname") String flowName,
       @PathParam("flowrunid") String flowRunId,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
@@ -804,7 +804,7 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("fields") String fields) {
     return getEntities(req, res, clusterId, null,
-        TimelineEntityType.YARN_APPLICATION.toString(), userId, flowId,
+        TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, modifiedTimeStart,
         modifiedTimeEnd, relatesTo, isRelatedTo, infofilters, conffilters,
         metricfilters, eventfilters, fields);
@@ -817,13 +817,13 @@ public class TimelineReaderWebServices {
    * reached, will be returned.
    */
   @GET
-  @Path("/flowapps/{userid}/{flowid}/")
+  @Path("/flowapps/{userid}/{flowname}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getFlowApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("userid") String userId,
-      @PathParam("flowid") String flowId,
+      @PathParam("flowname") String flowName,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
@@ -837,7 +837,7 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("fields") String fields) {
     return getEntities(req, res, null, null,
-        TimelineEntityType.YARN_APPLICATION.toString(), userId, flowId,
+        TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         null, limit, createdTimeStart, createdTimeEnd, modifiedTimeStart,
         modifiedTimeEnd, relatesTo, isRelatedTo, infofilters, conffilters,
         metricfilters, eventfilters, fields);
@@ -849,14 +849,14 @@ public class TimelineReaderWebServices {
    * is reached, will be returned.
    */
   @GET
-  @Path("/flowapps/{userid}/{clusterid}/{flowid}/")
+  @Path("/flowapps/{userid}/{clusterid}/{flowname}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getFlowApps(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("userid") String userId,
       @PathParam("clusterid") String clusterId,
-      @PathParam("flowid") String flowId,
+      @PathParam("flowname") String flowName,
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
@@ -870,7 +870,7 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("fields") String fields) {
     return getEntities(req, res, clusterId, null,
-        TimelineEntityType.YARN_APPLICATION.toString(), userId, flowId,
+        TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         null, limit, createdTimeStart, createdTimeEnd, modifiedTimeStart,
         modifiedTimeEnd, relatesTo, isRelatedTo, infofilters, conffilters,
         metricfilters, eventfilters, fields);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/ApplicationEntityReader.java
index 7082a5e..d812a6c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/ApplicationEntityReader.java
@@ -60,7 +60,7 @@ class ApplicationEntityReader extends GenericEntityReader {
       new ApplicationTable();
 
   public ApplicationEntityReader(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       Long limit, Long createdTimeBegin, Long createdTimeEnd,
       Long modifiedTimeBegin, Long modifiedTimeEnd,
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
@@ -68,7 +68,7 @@ class ApplicationEntityReader extends GenericEntityReader {
       Set<String> metricFilters, Set<String> eventFilters,
       TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
         eventFilters, confsToRetrieve, metricsToRetrieve, fieldsToRetrieve,
@@ -76,10 +76,10 @@ class ApplicationEntityReader extends GenericEntityReader {
   }
 
   public ApplicationEntityReader(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       String entityId, TimelineFilterList confsToRetrieve,
       TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowId, flowRunId, appId, entityType, entityId,
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
         confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
   }
 
@@ -173,7 +173,7 @@ class ApplicationEntityReader extends GenericEntityReader {
   protected Result getResult(Configuration hbaseConf, Connection conn,
       FilterList filterList) throws IOException {
     byte[] rowKey =
-        ApplicationRowKey.getRowKey(clusterId, userId, flowId, flowRunId,
+        ApplicationRowKey.getRowKey(clusterId, userId, flowName, flowRunId,
             appId);
     Get get = new Get(rowKey);
     get.setMaxVersions(Integer.MAX_VALUE);
@@ -191,7 +191,7 @@ class ApplicationEntityReader extends GenericEntityReader {
       Preconditions.checkNotNull(appId, "appId shouldn't be null");
     } else {
       Preconditions.checkNotNull(userId, "userId shouldn't be null");
-      Preconditions.checkNotNull(flowId, "flowId shouldn't be null");
+      Preconditions.checkNotNull(flowName, "flowName shouldn't be null");
     }
   }
 
@@ -199,10 +199,10 @@ class ApplicationEntityReader extends GenericEntityReader {
   protected void augmentParams(Configuration hbaseConf, Connection conn)
       throws IOException {
     if (singleEntityRead) {
-      if (flowId == null || flowRunId == null || userId == null) {
+      if (flowName == null || flowRunId == null || userId == null) {
         FlowContext context =
             lookupFlowContext(clusterId, appId, hbaseConf, conn);
-        flowId = context.flowId;
+        flowName = context.flowName;
         flowRunId = context.flowRunId;
         userId = context.userId;
       }
@@ -244,10 +244,10 @@ class ApplicationEntityReader extends GenericEntityReader {
     Scan scan = new Scan();
     if (flowRunId != null) {
       scan.setRowPrefixFilter(ApplicationRowKey.
-          getRowKeyPrefix(clusterId, userId, flowId, flowRunId));
+          getRowKeyPrefix(clusterId, userId, flowName, flowRunId));
     } else {
       scan.setRowPrefixFilter(ApplicationRowKey.
-          getRowKeyPrefix(clusterId, userId, flowId));
+          getRowKeyPrefix(clusterId, userId, flowName));
     }
     FilterList newList = new FilterList();
     newList.addFilter(new PageFilter(limit));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/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 48bf844..36b3f0d 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
@@ -150,11 +150,11 @@ public class FileSystemTimelineReaderImpl extends AbstractService
     }
   }
 
-  private String getFlowRunPath(String userId, String clusterId, String flowId,
+  private String getFlowRunPath(String userId, String clusterId, String flowName,
       Long flowRunId, String appId)
       throws IOException {
-    if (userId != null && flowId != null && flowRunId != null) {
-      return userId + "/" + flowId + "/" + flowRunId;
+    if (userId != null && flowName != null && flowRunId != null) {
+      return userId + "/" + flowName + "/" + flowRunId;
     }
     if (clusterId == null || appId == null) {
       throw new IOException("Unable to get flow info");
@@ -387,11 +387,11 @@ public class FileSystemTimelineReaderImpl extends AbstractService
 
   @Override
   public TimelineEntity getEntity(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       String entityId, TimelineFilterList confsToRetrieve,
       TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve)
       throws IOException {
-    String flowRunPath = getFlowRunPath(userId, clusterId, flowId,
+    String flowRunPath = getFlowRunPath(userId, clusterId, flowName,
         flowRunId, appId);
     File dir = new File(new File(rootPath, ENTITIES_DIR),
         clusterId + "/" + flowRunPath + "/" + appId + "/" + entityType);
@@ -411,7 +411,7 @@ public class FileSystemTimelineReaderImpl extends AbstractService
 
   @Override
   public Set<TimelineEntity> getEntities(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       Long limit, Long createdTimeBegin, Long createdTimeEnd,
       Long modifiedTimeBegin, Long modifiedTimeEnd,
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
@@ -420,7 +420,7 @@ public class FileSystemTimelineReaderImpl extends AbstractService
       TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve) throws IOException {
     String flowRunPath =
-        getFlowRunPath(userId, clusterId, flowId, flowRunId, appId);
+        getFlowRunPath(userId, clusterId, flowName, flowRunId, appId);
     File dir =
         new File(new File(rootPath, ENTITIES_DIR),
             clusterId + "/" + flowRunPath + "/" + appId + "/" + entityType);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.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/FlowActivityEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
index 71dd0a1..7e8d4ba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
@@ -49,23 +49,23 @@ class FlowActivityEntityReader extends TimelineEntityReader {
       new FlowActivityTable();
 
   public FlowActivityEntityReader(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       Long limit, Long createdTimeBegin, Long createdTimeEnd,
       Long modifiedTimeBegin, Long modifiedTimeEnd,
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String> metricFilters, Set<String> eventFilters,
       EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
         eventFilters, null, null, fieldsToRetrieve, true);
   }
 
   public FlowActivityEntityReader(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       String entityId, EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowId, flowRunId, appId, entityType, entityId,
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
         null, null, fieldsToRetrieve);
   }
 
@@ -135,7 +135,7 @@ class FlowActivityEntityReader extends TimelineEntityReader {
 
     long time = rowKey.getDayTimestamp();
     String user = rowKey.getUserId();
-    String flowName = rowKey.getFlowId();
+    String flowName = rowKey.getFlowName();
 
     FlowActivityEntity flowActivity =
         new FlowActivityEntity(clusterId, time, user, flowName);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/FlowRunEntityReader.java
index 1895fa6..c9076ee 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/FlowRunEntityReader.java
@@ -58,7 +58,7 @@ class FlowRunEntityReader extends TimelineEntityReader {
   private static final FlowRunTable FLOW_RUN_TABLE = new FlowRunTable();
 
   public FlowRunEntityReader(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       Long limit, Long createdTimeBegin, Long createdTimeEnd,
       Long modifiedTimeBegin, Long modifiedTimeEnd,
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
@@ -66,17 +66,17 @@ class FlowRunEntityReader extends TimelineEntityReader {
       Set<String> metricFilters, Set<String> eventFilters,
       TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
         eventFilters, null, metricsToRetrieve, fieldsToRetrieve, true);
   }
 
   public FlowRunEntityReader(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       String entityId, TimelineFilterList confsToRetrieve,
       TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowId, flowRunId, appId, entityType, entityId,
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
         null, metricsToRetrieve, fieldsToRetrieve);
   }
 
@@ -92,7 +92,7 @@ class FlowRunEntityReader extends TimelineEntityReader {
   protected void validateParams() {
     Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
     Preconditions.checkNotNull(userId, "userId shouldn't be null");
-    Preconditions.checkNotNull(flowId, "flowId shouldn't be null");
+    Preconditions.checkNotNull(flowName, "flowName shouldn't be null");
     if (singleEntityRead) {
       Preconditions.checkNotNull(flowRunId, "flowRunId shouldn't be null");
     }
@@ -155,7 +155,7 @@ class FlowRunEntityReader extends TimelineEntityReader {
   protected Result getResult(Configuration hbaseConf, Connection conn,
       FilterList filterList) throws IOException {
     byte[] rowKey =
-        FlowRunRowKey.getRowKey(clusterId, userId, flowId, flowRunId);
+        FlowRunRowKey.getRowKey(clusterId, userId, flowName, flowRunId);
     Get get = new Get(rowKey);
     get.setMaxVersions(Integer.MAX_VALUE);
     if (filterList != null && !filterList.getFilters().isEmpty()) {
@@ -169,7 +169,7 @@ class FlowRunEntityReader extends TimelineEntityReader {
       Connection conn, FilterList filterList) throws IOException {
     Scan scan = new Scan();
     scan.setRowPrefixFilter(
-        FlowRunRowKey.getRowKeyPrefix(clusterId, userId, flowId));
+        FlowRunRowKey.getRowKeyPrefix(clusterId, userId, flowName));
     FilterList newList = new FilterList();
     newList.addFilter(new PageFilter(limit));
     if (filterList != null && !filterList.getFilters().isEmpty()) {
@@ -183,7 +183,7 @@ class FlowRunEntityReader extends TimelineEntityReader {
   protected TimelineEntity parseEntity(Result result) throws IOException {
     FlowRunEntity flowRun = new FlowRunEntity();
     flowRun.setUser(userId);
-    flowRun.setName(flowId);
+    flowRun.setName(flowName);
     if (singleEntityRead) {
       flowRun.setRunId(flowRunId);
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.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/GenericEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
index dcb8b89..784dfd5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
@@ -76,7 +76,7 @@ class GenericEntityReader extends TimelineEntityReader {
   private final AppToFlowTable appToFlowTable = new AppToFlowTable();
 
   public GenericEntityReader(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       Long limit, Long createdTimeBegin, Long createdTimeEnd,
       Long modifiedTimeBegin, Long modifiedTimeEnd,
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
@@ -84,7 +84,7 @@ class GenericEntityReader extends TimelineEntityReader {
       Set<String> metricFilters, Set<String> eventFilters,
       TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve, boolean sortedKeys) {
-    super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
         eventFilters, confsToRetrieve, metricsToRetrieve, fieldsToRetrieve,
@@ -92,10 +92,10 @@ class GenericEntityReader extends TimelineEntityReader {
   }
 
   public GenericEntityReader(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       String entityId, TimelineFilterList confsToRetrieve,
       TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowId, flowRunId, appId, entityType, entityId,
+    super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
         confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
   }
 
@@ -204,11 +204,11 @@ class GenericEntityReader extends TimelineEntityReader {
 
   protected static class FlowContext {
     protected final String userId;
-    protected final String flowId;
+    protected final String flowName;
     protected final Long flowRunId;
-    public FlowContext(String user, String flowId, Long flowRunId) {
+    public FlowContext(String user, String flowName, Long flowRunId) {
       this.userId = user;
-      this.flowId = flowId;
+      this.flowName = flowName;
       this.flowRunId = flowRunId;
     }
   }
@@ -227,10 +227,10 @@ class GenericEntityReader extends TimelineEntityReader {
   protected void augmentParams(Configuration hbaseConf, Connection conn)
       throws IOException {
     // In reality all three should be null or neither should be null
-    if (flowId == null || flowRunId == null || userId == null) {
+    if (flowName == null || flowRunId == null || userId == null) {
       FlowContext context =
           lookupFlowContext(clusterId, appId, hbaseConf, conn);
-      flowId = context.flowId;
+      flowName = context.flowName;
       flowRunId = context.flowRunId;
       userId = context.userId;
     }
@@ -269,7 +269,7 @@ class GenericEntityReader extends TimelineEntityReader {
   protected Result getResult(Configuration hbaseConf, Connection conn,
       FilterList filterList) throws IOException {
     byte[] rowKey =
-        EntityRowKey.getRowKey(clusterId, userId, flowId, flowRunId, appId,
+        EntityRowKey.getRowKey(clusterId, userId, flowName, flowRunId, appId,
             entityType, entityId);
     Get get = new Get(rowKey);
     get.setMaxVersions(Integer.MAX_VALUE);
@@ -286,7 +286,7 @@ class GenericEntityReader extends TimelineEntityReader {
     // and one type
     Scan scan = new Scan();
     scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
-        clusterId, userId, flowId, flowRunId, appId, entityType));
+        clusterId, userId, flowName, flowRunId, appId, entityType));
     scan.setMaxVersions(Integer.MAX_VALUE);
     if (filterList != null && !filterList.getFilters().isEmpty()) {
       scan.setFilter(filterList);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.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/HBaseTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
index 9e4b26a..96c5a19 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
@@ -64,20 +64,20 @@ public class HBaseTimelineReaderImpl
 
   @Override
   public TimelineEntity getEntity(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       String entityId, TimelineFilterList confsToRetrieve,
       TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve)
       throws IOException {
     TimelineEntityReader reader =
         TimelineEntityReaderFactory.createSingleEntityReader(userId, clusterId,
-            flowId, flowRunId, appId, entityType, entityId, confsToRetrieve,
+            flowName, flowRunId, appId, entityType, entityId, confsToRetrieve,
             metricsToRetrieve, fieldsToRetrieve);
     return reader.readEntity(hbaseConf, conn);
   }
 
   @Override
   public Set<TimelineEntity> getEntities(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       Long limit, Long createdTimeBegin, Long createdTimeEnd,
       Long modifiedTimeBegin, Long modifiedTimeEnd,
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
@@ -87,7 +87,7 @@ public class HBaseTimelineReaderImpl
       EnumSet<Field> fieldsToRetrieve) throws IOException {
     TimelineEntityReader reader =
         TimelineEntityReaderFactory.createMultipleEntitiesReader(userId,
-            clusterId, flowId, flowRunId, appId, entityType, limit,
+            clusterId, flowName, flowRunId, appId, entityType, limit,
             createdTimeBegin, createdTimeEnd, modifiedTimeBegin,
             modifiedTimeEnd, relatesTo, isRelatedTo, infoFilters, configFilters,
             metricFilters, eventFilters, confsToRetrieve, metricsToRetrieve,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.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/TimelineEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
index 7178aab..a26c0c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
@@ -53,7 +53,7 @@ abstract class TimelineEntityReader {
 
   protected String userId;
   protected String clusterId;
-  protected String flowId;
+  protected String flowName;
   protected Long flowRunId;
   protected String appId;
   protected String entityType;
@@ -92,7 +92,7 @@ abstract class TimelineEntityReader {
    * Instantiates a reader for multiple-entity reads.
    */
   protected TimelineEntityReader(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       Long limit, Long createdTimeBegin, Long createdTimeEnd,
       Long modifiedTimeBegin, Long modifiedTimeEnd,
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
@@ -104,7 +104,7 @@ abstract class TimelineEntityReader {
     this.sortedKeys = sortedKeys;
     this.userId = userId;
     this.clusterId = clusterId;
-    this.flowId = flowId;
+    this.flowName = flowName;
     this.flowRunId = flowRunId;
     this.appId = appId;
     this.entityType = entityType;
@@ -130,13 +130,13 @@ abstract class TimelineEntityReader {
    * Instantiates a reader for single-entity reads.
    */
   protected TimelineEntityReader(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       String entityId, TimelineFilterList confsToRetrieve,
       TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
     this.singleEntityRead = true;
     this.userId = userId;
     this.clusterId = clusterId;
-    this.flowId = flowId;
+    this.flowName = flowName;
     this.flowRunId = flowRunId;
     this.appId = appId;
     this.entityType = entityType;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.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/TimelineEntityReaderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.java
index 16204c5..36ed4ca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.java
@@ -34,23 +34,23 @@ class TimelineEntityReaderFactory {
    * the specified input.
    */
   public static TimelineEntityReader createSingleEntityReader(String userId,
-      String clusterId, String flowId, Long flowRunId, String appId,
+      String clusterId, String flowName, Long flowRunId, String appId,
       String entityType, String entityId, TimelineFilterList confs,
       TimelineFilterList metrics, EnumSet<Field> fieldsToRetrieve) {
     // currently the types that are handled separate from the generic entity
     // table are application, flow run, and flow activity entities
     if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) {
-      return new ApplicationEntityReader(userId, clusterId, flowId, flowRunId,
+      return new ApplicationEntityReader(userId, clusterId, flowName, flowRunId,
           appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
     } else if (TimelineEntityType.YARN_FLOW_RUN.matches(entityType)) {
-      return new FlowRunEntityReader(userId, clusterId, flowId, flowRunId,
+      return new FlowRunEntityReader(userId, clusterId, flowName, flowRunId,
           appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
     } else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) {
-      return new FlowActivityEntityReader(userId, clusterId, flowId, flowRunId,
+      return new FlowActivityEntityReader(userId, clusterId, flowName, flowRunId,
           appId, entityType, entityId, fieldsToRetrieve);
     } else {
       // assume we're dealing with a generic entity read
-      return new GenericEntityReader(userId, clusterId, flowId, flowRunId,
+      return new GenericEntityReader(userId, clusterId, flowName, flowRunId,
         appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
     }
   }
@@ -60,7 +60,7 @@ class TimelineEntityReaderFactory {
    * the specified input and predicates.
    */
   public static TimelineEntityReader createMultipleEntitiesReader(String userId,
-      String clusterId, String flowId, Long flowRunId, String appId,
+      String clusterId, String flowName, Long flowRunId, String appId,
       String entityType, Long limit, Long createdTimeBegin, Long createdTimeEnd,
       Long modifiedTimeBegin, Long modifiedTimeEnd,
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
@@ -71,26 +71,26 @@ class TimelineEntityReaderFactory {
     // currently the types that are handled separate from the generic entity
     // table are application, flow run, and flow activity entities
     if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) {
-      return new ApplicationEntityReader(userId, clusterId, flowId, flowRunId,
+      return new ApplicationEntityReader(userId, clusterId, flowName, flowRunId,
           appId, entityType, limit, createdTimeBegin, createdTimeEnd,
           modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
           infoFilters, configFilters, metricFilters, eventFilters, confs,
           metrics, fieldsToRetrieve);
     } else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) {
-      return new FlowActivityEntityReader(userId, clusterId, flowId, flowRunId,
+      return new FlowActivityEntityReader(userId, clusterId, flowName, flowRunId,
           appId, entityType, limit, createdTimeBegin, createdTimeEnd,
           modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
           infoFilters, configFilters, metricFilters, eventFilters,
           fieldsToRetrieve);
     } else if (TimelineEntityType.YARN_FLOW_RUN.matches(entityType)) {
-      return new FlowRunEntityReader(userId, clusterId, flowId, flowRunId,
+      return new FlowRunEntityReader(userId, clusterId, flowName, flowRunId,
           appId, entityType, limit, createdTimeBegin, createdTimeEnd,
           modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
           infoFilters, configFilters, metricFilters, eventFilters, confs,
           metrics, fieldsToRetrieve);
     } else {
       // assume we're dealing with a generic entity read
-      return new GenericEntityReader(userId, clusterId, flowId, flowRunId,
+      return new GenericEntityReader(userId, clusterId, flowName, flowRunId,
           appId, entityType, limit, createdTimeBegin, createdTimeEnd,
           modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
           infoFilters, configFilters, metricFilters, eventFilters, confs,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.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/TimelineReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java
index 0ed17da..56bd3a0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java
@@ -62,7 +62,7 @@ public interface TimelineReader extends Service {
    *    Context user Id(optional).
    * @param clusterId
    *    Context cluster Id(mandatory).
-   * @param flowId
+   * @param flowName
    *    Context flow Id (optional).
    * @param flowRunId
    *    Context flow run Id (optional).
@@ -93,7 +93,7 @@ public interface TimelineReader extends Service {
    *    contain the metadata plus the given fields to retrieve.
    * @throws IOException
    */
-  TimelineEntity getEntity(String userId, String clusterId, String flowId,
+  TimelineEntity getEntity(String userId, String clusterId, String flowName,
       Long flowRunId, String appId, String entityType, String entityId,
       TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve) throws IOException;
@@ -113,7 +113,7 @@ public interface TimelineReader extends Service {
    *    Context user Id(optional).
    * @param clusterId
    *    Context cluster Id(mandatory).
-   * @param flowId
+   * @param flowName
    *    Context flow Id (optional).
    * @param flowRunId
    *    Context flow run Id (optional).
@@ -183,7 +183,7 @@ public interface TimelineReader extends Service {
    * @throws IOException
    */
   Set<TimelineEntity> getEntities(String userId, String clusterId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       Long limit, Long createdTimeBegin, Long createdTimeEnd,
       Long modifiedTimeBegin, Long modifiedTimeEnd,
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java
index 1cf6145..3c182b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKey.java
@@ -27,15 +27,15 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStor
 public class ApplicationRowKey {
   private final String clusterId;
   private final String userId;
-  private final String flowId;
+  private final String flowName;
   private final long flowRunId;
   private final String appId;
 
-  public ApplicationRowKey(String clusterId, String userId, String flowId,
+  public ApplicationRowKey(String clusterId, String userId, String flowName,
       long flowRunId, String appId) {
     this.clusterId = clusterId;
     this.userId = userId;
-    this.flowId = flowId;
+    this.flowName = flowName;
     this.flowRunId = flowRunId;
     this.appId = appId;
   }
@@ -48,8 +48,8 @@ public class ApplicationRowKey {
     return userId;
   }
 
-  public String getFlowId() {
-    return flowId;
+  public String getFlowName() {
+    return flowName;
   }
 
   public long getFlowRunId() {
@@ -62,54 +62,54 @@ public class ApplicationRowKey {
 
   /**
    * Constructs a row key prefix for the application table as follows:
-   * {@code clusterId!userName!flowId!}
+   * {@code clusterId!userName!flowName!}
    *
    * @param clusterId
    * @param userId
-   * @param flowId
+   * @param flowName
    * @return byte array with the row key prefix
    */
   public static byte[] getRowKeyPrefix(String clusterId, String userId,
-      String flowId) {
+      String flowName) {
     byte[] first = Bytes.toBytes(
-        Separator.QUALIFIERS.joinEncoded(clusterId, userId, flowId));
+        Separator.QUALIFIERS.joinEncoded(clusterId, userId, flowName));
     return Separator.QUALIFIERS.join(first, new byte[0]);
   }
 
   /**
    * Constructs a row key prefix for the application table as follows:
-   * {@code clusterId!userName!flowId!flowRunId!}
+   * {@code clusterId!userName!flowName!flowRunId!}
    *
    * @param clusterId
    * @param userId
-   * @param flowId
+   * @param flowName
    * @param flowRunId
    * @return byte array with the row key prefix
    */
   public static byte[] getRowKeyPrefix(String clusterId, String userId,
-      String flowId, Long flowRunId) {
+      String flowName, Long flowRunId) {
     byte[] first = Bytes.toBytes(
-        Separator.QUALIFIERS.joinEncoded(clusterId, userId, flowId));
+        Separator.QUALIFIERS.joinEncoded(clusterId, userId, flowName));
     byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
     return Separator.QUALIFIERS.join(first, second, new byte[0]);
   }
 
   /**
    * Constructs a row key for the application table as follows:
-   * {@code clusterId!userName!flowId!flowRunId!AppId}
+   * {@code clusterId!userName!flowName!flowRunId!AppId}
    *
    * @param clusterId
    * @param userId
-   * @param flowId
+   * @param flowName
    * @param flowRunId
    * @param appId
    * @return byte array with the row key
    */
   public static byte[] getRowKey(String clusterId, String userId,
-      String flowId, Long flowRunId, String appId) {
+      String flowName, Long flowRunId, String appId) {
     byte[] first =
         Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId, userId,
-            flowId));
+            flowName));
     // Note that flowRunId is a long, so we can't encode them all at the same
     // time.
     byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
@@ -132,11 +132,11 @@ public class ApplicationRowKey {
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[0]));
     String userId =
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[1]));
-    String flowId =
+    String flowName =
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[2]));
     long flowRunId =
         TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
     String appId = TimelineStorageUtils.decodeAppId(rowKeyComponents[4]);
-    return new ApplicationRowKey(clusterId, userId, flowId, flowRunId, appId);
+    return new ApplicationRowKey(clusterId, userId, flowName, flowRunId, appId);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
index a997997..ee5ff83 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationTable.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBas
  * |-------------------------------------------------------------------------|
  * | clusterId! | id:appId                     | metricId1:   | configKey1:  |
  * | userName!  |                              | metricValue1 | configValue1 |
- * | flowId!    | created_time:                | @timestamp1  |              |
+ * | flowName!  | created_time:                | @timestamp1  |              |
  * | flowRunId! | 1392993084018                |              | configKey2:  |
  * | AppId      |                              | metriciD1:   | configValue2 |
  * |            | modified_time:               | metricValue2 |              |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.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/AppToFlowColumnFamily.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/AppToFlowColumnFamily.java
index e74235f..d722d15 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/AppToFlowColumnFamily.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/AppToFlowColumnFamily.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
  */
 public enum AppToFlowColumnFamily implements ColumnFamily<AppToFlowTable> {
   /**
-   * Mapping column family houses known columns such as flowId and flowRunId
+   * Mapping column family houses known columns such as flowName and flowRunId
    */
   MAPPING("m");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/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 b30f253..868c6f3 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
@@ -34,7 +34,7 @@ import java.io.IOException;
 
 /**
  * The app_flow table as column families mapping. Mapping stores
- * appId to flowId and flowRunId mapping information
+ * appId to flowName and flowRunId mapping information
  *
  * Example app_flow table record:
  *
@@ -43,7 +43,7 @@ import java.io.IOException;
  * |  Row       | Column Family           |
  * |  key       | info                    |
  * |--------------------------------------|
- * | clusterId! | flowId:                 |
+ * | clusterId! | flowName:               |
  * | AppId      | foo@daily_hive_report   |
  * |            |                         |
  * |            | flowRunId:              |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java
index e0413c0..3eaa78c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java
@@ -27,17 +27,17 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStor
 public class EntityRowKey {
   private final String clusterId;
   private final String userId;
-  private final String flowId;
+  private final String flowName;
   private final long flowRunId;
   private final String appId;
   private final String entityType;
   private final String entityId;
 
-  public EntityRowKey(String clusterId, String userId, String flowId,
+  public EntityRowKey(String clusterId, String userId, String flowName,
       long flowRunId, String appId, String entityType, String entityId) {
     this.clusterId = clusterId;
     this.userId = userId;
-    this.flowId = flowId;
+    this.flowName = flowName;
     this.flowRunId = flowRunId;
     this.appId = appId;
     this.entityType = entityType;
@@ -52,8 +52,8 @@ public class EntityRowKey {
     return userId;
   }
 
-  public String getFlowId() {
-    return flowId;
+  public String getFlowName() {
+    return flowName;
   }
 
   public long getFlowRunId() {
@@ -74,20 +74,20 @@ public class EntityRowKey {
 
   /**
    * Constructs a row key prefix for the entity table as follows:
-   * {@code userName!clusterId!flowId!flowRunId!AppId}
+   * {@code userName!clusterId!flowName!flowRunId!AppId}
    *
    * @param clusterId
    * @param userId
-   * @param flowId
+   * @param flowName
    * @param flowRunId
    * @param appId
    * @return byte array with the row key prefix
    */
   public static byte[] getRowKeyPrefix(String clusterId, String userId,
-      String flowId, Long flowRunId, String appId) {
+      String flowName, Long flowRunId, String appId) {
     byte[] first =
         Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(userId, clusterId,
-            flowId));
+            flowName));
     // Note that flowRunId is a long, so we can't encode them all at the same
     // time.
     byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
@@ -97,21 +97,21 @@ public class EntityRowKey {
 
   /**
    * Constructs a row key prefix for the entity table as follows:
-   * {@code userName!clusterId!flowId!flowRunId!AppId!entityType!}
+   * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!}
    *
    * @param clusterId
    * @param userId
-   * @param flowId
+   * @param flowName
    * @param flowRunId
    * @param appId
    * @param entityType
    * @return byte array with the row key prefix
    */
   public static byte[] getRowKeyPrefix(String clusterId, String userId,
-      String flowId, Long flowRunId, String appId, String entityType) {
+      String flowName, Long flowRunId, String appId, String entityType) {
     byte[] first =
         Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(userId, clusterId,
-            flowId));
+            flowName));
     // Note that flowRunId is a long, so we can't encode them all at the same
     // time.
     byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
@@ -123,11 +123,11 @@ public class EntityRowKey {
 
   /**
    * Constructs a row key for the entity table as follows:
-   * {@code userName!clusterId!flowId!flowRunId!AppId!entityType!entityId}
+   * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}
    *
    * @param clusterId
    * @param userId
-   * @param flowId
+   * @param flowName
    * @param flowRunId
    * @param appId
    * @param entityType
@@ -135,11 +135,11 @@ public class EntityRowKey {
    * @return byte array with the row key
    */
   public static byte[] getRowKey(String clusterId, String userId,
-      String flowId, Long flowRunId, String appId, String entityType,
+      String flowName, Long flowRunId, String appId, String entityType,
       String entityId) {
     byte[] first =
         Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(userId, clusterId,
-            flowId));
+            flowName));
     // Note that flowRunId is a long, so we can't encode them all at the same
     // time.
     byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
@@ -164,7 +164,7 @@ public class EntityRowKey {
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[0]));
     String clusterId =
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[1]));
-    String flowId =
+    String flowName =
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[2]));
     long flowRunId =
         TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
@@ -173,7 +173,7 @@ public class EntityRowKey {
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[5]));
     String entityId =
         Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[6]));
-    return new EntityRowKey(clusterId, userId, flowId, flowRunId, appId,
+    return new EntityRowKey(clusterId, userId, flowName, flowRunId, appId,
         entityType, entityId);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.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/entity/EntityTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
index 96773b2..15d4465 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
@@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBas
  * |-------------------------------------------------------------------------|
  * | userName!  | id:entityId                  | metricId1:   | configKey1:  |
  * | clusterId! |                              | metricValue1 | configValue1 |
- * | flowId!    | type:entityType              | @timestamp1  |              |
+ * | flowName!  | type:entityType              | @timestamp1  |              |
  * | flowRunId! |                              |              | configKey2:  |
  * | AppId!     | created_time:                | metriciD1:   | configValue2 |
  * | entityType!| 1392993084018                | metricValue2 |              |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
index fc1aa70..a9598ef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKey.java
@@ -29,14 +29,14 @@ public class FlowActivityRowKey {
   private final String clusterId;
   private final long dayTs;
   private final String userId;
-  private final String flowId;
+  private final String flowName;
 
   public FlowActivityRowKey(String clusterId, long dayTs, String userId,
-      String flowId) {
+      String flowName) {
     this.clusterId = clusterId;
     this.dayTs = dayTs;
     this.userId = userId;
-    this.flowId = flowId;
+    this.flowName = flowName;
   }
 
   public String getClusterId() {
@@ -51,8 +51,8 @@ public class FlowActivityRowKey {
     return userId;
   }
 
-  public String getFlowId() {
-    return flowId;
+  public String getFlowName() {
+    return flowName;
   }
 
   /**
@@ -82,38 +82,38 @@ public class FlowActivityRowKey {
 
   /**
    * Constructs a row key for the flow activity table as follows:
-   * {@code clusterId!dayTimestamp!user!flowId}
+   * {@code clusterId!dayTimestamp!user!flowName}
    *
    * Will insert into current day's record in the table
    * @param clusterId
    * @param userId
-   * @param flowId
+   * @param flowName
    * @return byte array with the row key prefix
    */
   public static byte[] getRowKey(String clusterId, String userId,
-      String flowId) {
+      String flowName) {
     long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(System
         .currentTimeMillis());
-    return getRowKey(clusterId, dayTs, userId, flowId);
+    return getRowKey(clusterId, dayTs, userId, flowName);
   }
 
   /**
    * Constructs a row key for the flow activity table as follows:
-   * {@code clusterId!dayTimestamp!user!flowId}
+   * {@code clusterId!dayTimestamp!user!flowName}
    *
    * @param clusterId
    * @param dayTs
    * @param userId
-   * @param flowId
+   * @param flowName
    * @return byte array for the row key
    */
   public static byte[] getRowKey(String clusterId, long dayTs, String userId,
-      String flowId) {
+      String flowName) {
     return Separator.QUALIFIERS.join(
         Bytes.toBytes(Separator.QUALIFIERS.encode(clusterId)),
         Bytes.toBytes(TimelineStorageUtils.invertLong(dayTs)),
         Bytes.toBytes(Separator.QUALIFIERS.encode(userId)),
-        Bytes.toBytes(Separator.QUALIFIERS.encode(flowId)));
+        Bytes.toBytes(Separator.QUALIFIERS.encode(flowName)));
   }
 
   /**
@@ -133,8 +133,8 @@ public class FlowActivityRowKey {
         TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[1]));
     String userId = Separator.QUALIFIERS.decode(Bytes
         .toString(rowKeyComponents[2]));
-    String flowId = Separator.QUALIFIERS.decode(Bytes
+    String flowName = Separator.QUALIFIERS.decode(Bytes
         .toString(rowKeyComponents[3]));
-    return new FlowActivityRowKey(clusterId, dayTs, userId, flowId);
+    return new FlowActivityRowKey(clusterId, dayTs, userId, flowName);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dfaa7c5f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.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/flow/FlowActivityTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
index af8df99..315281f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
  * | inv Top of |                              |
  * | Day!       | r!runid2:version7            |
  * | userName!  |                              |
- * | flowId     |                              |
+ * | flowName   |                              |
  * |-------------------------------------------|
  * </pre>
  */


[24/50] [abbrv] hadoop git commit: YARN-4210. HBase reader throws NPE if Get returns no rows (Varun Saxena via vrushali)

Posted by gt...@apache.org.
YARN-4210. HBase reader throws NPE if Get returns no rows (Varun Saxena via vrushali)


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

Branch: refs/heads/feature-YARN-2928
Commit: 0839cd149b787d5c731aae7e2ccd47e20ebb14fd
Parents: 008e273
Author: Vrushali Channapattan <vr...@apache.org>
Authored: Wed Sep 30 13:56:07 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:58 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../storage/ApplicationEntityReader.java        | 21 ++--------
 .../storage/FlowActivityEntityReader.java       | 27 +++++++------
 .../storage/FlowRunEntityReader.java            | 11 +++---
 .../storage/GenericEntityReader.java            |  3 +-
 .../storage/TimelineEntityReader.java           | 37 ++++++++++++------
 .../TestTimelineReaderWebServicesFlowRun.java   | 40 ++++++++++++++++++++
 .../storage/TestHBaseTimelineStorage.java       | 13 -------
 8 files changed, 96 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0839cd14/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f9bb187..0d2537e 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -115,6 +115,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4203. Add request/response logging & timing for each REST endpoint
     call (Varun Saxena via vrushali)
 
+    YARN-4210. HBase reader throws NPE if Get returns no rows (Varun Saxena
+    via vrushali)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0839cd14/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/ApplicationEntityReader.java
index dfbc31d..d5b5d63 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/ApplicationEntityReader.java
@@ -27,6 +27,7 @@ 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.hbase.client.ResultScanner;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
@@ -85,24 +86,10 @@ class ApplicationEntityReader extends GenericEntityReader {
   }
 
   @Override
-  protected Iterable<Result> getResults(Configuration hbaseConf,
+  protected ResultScanner getResults(Configuration hbaseConf,
       Connection conn) throws IOException {
-    // If getEntities() is called for an application, there can be at most
-    // one entity. If the entity passes the filter, it is returned. Otherwise,
-    // an empty set is returned.
-    byte[] rowKey = ApplicationRowKey.getRowKey(clusterId, userId, flowId,
-        flowRunId, appId);
-    Get get = new Get(rowKey);
-    get.setMaxVersions(Integer.MAX_VALUE);
-    Result result = table.getResult(hbaseConf, conn, get);
-    TimelineEntity entity = parseEntity(result);
-    Set<Result> set;
-    if (entity != null) {
-      set = Collections.singleton(result);
-    } else {
-      set = Collections.emptySet();
-    }
-    return set;
+    throw new UnsupportedOperationException(
+        "we don't support multiple apps query");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0839cd14/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.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/FlowActivityEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
index d5ece2e..e68ca17 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
@@ -27,6 +27,7 @@ import java.util.TreeSet;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.PageFilter;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
@@ -88,18 +89,22 @@ class FlowActivityEntityReader extends TimelineEntityReader {
     augmentParams(hbaseConf, conn);
 
     NavigableSet<TimelineEntity> entities = new TreeSet<>();
-    Iterable<Result> results = getResults(hbaseConf, conn);
-    for (Result result : results) {
-      TimelineEntity entity = parseEntity(result);
-      if (entity == null) {
-        continue;
-      }
-      entities.add(entity);
-      if (entities.size() == limit) {
-        break;
+    ResultScanner results = getResults(hbaseConf, conn);
+    try {
+      for (Result result : results) {
+        TimelineEntity entity = parseEntity(result);
+        if (entity == null) {
+          continue;
+        }
+        entities.add(entity);
+        if (entities.size() == limit) {
+          break;
+        }
       }
+      return entities;
+    } finally {
+      results.close();
     }
-    return entities;
   }
 
   @Override
@@ -123,7 +128,7 @@ class FlowActivityEntityReader extends TimelineEntityReader {
   }
 
   @Override
-  protected Iterable<Result> getResults(Configuration hbaseConf,
+  protected ResultScanner getResults(Configuration hbaseConf,
       Connection conn) throws IOException {
     Scan scan = new Scan();
     scan.setRowPrefixFilter(FlowActivityRowKey.getRowKeyPrefix(clusterId));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0839cd14/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/FlowRunEntityReader.java
index ced795d..b5d7ae5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/FlowRunEntityReader.java
@@ -26,6 +26,7 @@ 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.hbase.client.ResultScanner;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
@@ -96,7 +97,7 @@ class FlowRunEntityReader extends TimelineEntityReader {
   }
 
   @Override
-  protected Iterable<Result> getResults(Configuration hbaseConf,
+  protected ResultScanner getResults(Configuration hbaseConf,
       Connection conn) throws IOException {
     throw new UnsupportedOperationException(
         "multiple entity query is not supported");
@@ -110,14 +111,14 @@ class FlowRunEntityReader extends TimelineEntityReader {
     flowRun.setRunId(flowRunId);
 
     // read the start time
-    Long startTime = (Long)FlowRunColumn.MIN_START_TIME.readResult(result);
+    Number startTime = (Number)FlowRunColumn.MIN_START_TIME.readResult(result);
     if (startTime != null) {
-      flowRun.setStartTime(startTime);
+      flowRun.setStartTime(startTime.longValue());
     }
     // read the end time if available
-    Long endTime = (Long)FlowRunColumn.MAX_END_TIME.readResult(result);
+    Number endTime = (Number)FlowRunColumn.MAX_END_TIME.readResult(result);
     if (endTime != null) {
-      flowRun.setMaxEndTime(endTime);
+      flowRun.setMaxEndTime(endTime.longValue());
     }
 
     // read the flow version

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0839cd14/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.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/GenericEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
index 466914b..396a02b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
@@ -30,6 +30,7 @@ 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.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
@@ -176,7 +177,7 @@ class GenericEntityReader extends TimelineEntityReader {
   }
 
   @Override
-  protected Iterable<Result> getResults(Configuration hbaseConf,
+  protected ResultScanner getResults(Configuration hbaseConf,
       Connection conn) throws IOException {
     // Scan through part of the table to find the entities belong to one app
     // and one type

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0839cd14/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.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/TimelineEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
index 0d1134c..93be2db 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
@@ -25,9 +25,12 @@ import java.util.NavigableSet;
 import java.util.Set;
 import java.util.TreeSet;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
 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.TimelineReader.Field;
@@ -40,6 +43,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
  * entities that are being requested.
  */
 abstract class TimelineEntityReader {
+  private static final Log LOG = LogFactory.getLog(TimelineEntityReader.class);
   protected final boolean singleEntityRead;
 
   protected String userId;
@@ -131,6 +135,11 @@ abstract class TimelineEntityReader {
     augmentParams(hbaseConf, conn);
 
     Result result = getResult(hbaseConf, conn);
+    if (result == null || result.isEmpty()) {
+      // Could not find a matching row.
+      LOG.info("Cannot find matching entity of type " + entityType);
+      return null;
+    }
     return parseEntity(result);
   }
 
@@ -145,18 +154,22 @@ abstract class TimelineEntityReader {
     augmentParams(hbaseConf, conn);
 
     NavigableSet<TimelineEntity> entities = new TreeSet<>();
-    Iterable<Result> results = getResults(hbaseConf, conn);
-    for (Result result : results) {
-      TimelineEntity entity = parseEntity(result);
-      if (entity == null) {
-        continue;
-      }
-      entities.add(entity);
-      if (entities.size() > limit) {
-        entities.pollLast();
+    ResultScanner results = getResults(hbaseConf, conn);
+    try {
+      for (Result result : results) {
+        TimelineEntity entity = parseEntity(result);
+        if (entity == null) {
+          continue;
+        }
+        entities.add(entity);
+        if (entities.size() > limit) {
+          entities.pollLast();
+        }
       }
+      return entities;
+    } finally {
+      results.close();
     }
-    return entities;
   }
 
   /**
@@ -184,9 +197,9 @@ abstract class TimelineEntityReader {
       throws IOException;
 
   /**
-   * Fetches an iterator for {@link Result} instances for a multi-entity read.
+   * Fetches a {@link ResultScanner} for a multi-entity read.
    */
-  protected abstract Iterable<Result> getResults(Configuration hbaseConf,
+  protected abstract ResultScanner getResults(Configuration hbaseConf,
       Connection conn) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0839cd14/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesFlowRun.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/TestTimelineReaderWebServicesFlowRun.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesFlowRun.java
index ae71e2c..e359f78 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesFlowRun.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesFlowRun.java
@@ -59,6 +59,7 @@ import org.junit.Test;
 import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.GenericType;
+import com.sun.jersey.api.client.ClientResponse.Status;
 import com.sun.jersey.api.client.config.ClientConfig;
 import com.sun.jersey.api.client.config.DefaultClientConfig;
 import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
@@ -281,6 +282,16 @@ public class TestTimelineReaderWebServicesFlowRun {
     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().equals(status));
+  }
+
   @Test
   public void testGetFlowRun() throws Exception {
     Client client = createClient();
@@ -354,6 +365,35 @@ public class TestTimelineReaderWebServicesFlowRun {
     }
   }
 
+  @Test
+  public void testGetFlowRunNotPresent() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowrun/cluster1/flow_name/1002345678929?userid=user1");
+      verifyHttpResponse(client, uri, Status.NOT_FOUND);
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlowsNotPresent() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows/cluster2");
+      ClientResponse resp = getResponse(client, uri);
+      Set<FlowActivityEntity> entities =
+          resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+    } finally {
+      client.destroy();
+    }
+  }
+
   @After
   public void stop() throws Exception {
     if (server != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0839cd14/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.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/TestHBaseTimelineStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index 01920b3..3b0921b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -249,11 +249,7 @@ public class TestHBaseTimelineStorage {
       TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appId,
           entity.getType(), entity.getId(),
           EnumSet.of(TimelineReader.Field.ALL));
-      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
-          appId, entity.getType(), null, null, null, null, null, null, null,
-          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
       assertNotNull(e1);
-      assertEquals(1, es1.size());
 
       // verify attributes
       assertEquals(appId, e1.getId());
@@ -610,18 +606,9 @@ public class TestHBaseTimelineStorage {
       TimelineEntity e2 = hbr.getEntity(user, cluster, null, null, appName,
           entity.getType(), entity.getId(),
           EnumSet.of(TimelineReader.Field.ALL));
-      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
-          appName, entity.getType(), null, null, null, null, null, null, null,
-          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
-      Set<TimelineEntity> es2 = hbr.getEntities(user, cluster, null, null,
-          appName, entity.getType(), null, null, null, null, null, null, null,
-          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
       assertNotNull(e1);
       assertNotNull(e2);
       assertEquals(e1, e2);
-      assertEquals(1, es1.size());
-      assertEquals(1, es2.size());
-      assertEquals(es1, es2);
 
       // check the events
       NavigableSet<TimelineEvent> events = e1.getEvents();


[26/50] [abbrv] hadoop git commit: YARN-4178. [storage implementation] app id as string in row keys can cause incorrect ordering (Varun Saxena via sjlee)

Posted by gt...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestTimelineStorageUtils.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/TestTimelineStorageUtils.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/TestTimelineStorageUtils.java
new file mode 100644
index 0000000..046eda1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestTimelineStorageUtils.java
@@ -0,0 +1,56 @@
+/**
+ * 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.assertTrue;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.junit.Test;
+
+public class TestTimelineStorageUtils {
+
+  @Test
+  public void testEncodeDecodeAppId() {
+    long currentTs = System.currentTimeMillis();
+    ApplicationId appId1 = ApplicationId.newInstance(currentTs, 1);
+    ApplicationId appId2 = ApplicationId.newInstance(currentTs, 2);
+    ApplicationId appId3 = ApplicationId.newInstance(currentTs + 300, 1);
+    String appIdStr1 = appId1.toString();
+    String appIdStr2 = appId2.toString();
+    String appIdStr3 = appId3.toString();
+    byte[] appIdBytes1 = TimelineStorageUtils.encodeAppId(appIdStr1);
+    byte[] appIdBytes2 = TimelineStorageUtils.encodeAppId(appIdStr2);
+    byte[] appIdBytes3 = TimelineStorageUtils.encodeAppId(appIdStr3);
+    // App ids' should be encoded in a manner wherein descending order
+    // is maintained.
+    assertTrue("Ordering of app ids' is incorrect",
+        Bytes.compareTo(appIdBytes1, appIdBytes2) > 0 &&
+        Bytes.compareTo(appIdBytes1, appIdBytes3) > 0 &&
+        Bytes.compareTo(appIdBytes2, appIdBytes3) > 0);
+    String decodedAppId1 = TimelineStorageUtils.decodeAppId(appIdBytes1);
+    String decodedAppId2 = TimelineStorageUtils.decodeAppId(appIdBytes2);
+    String decodedAppId3 = TimelineStorageUtils.decodeAppId(appIdBytes3);
+    assertTrue("Decoded app id is not same as the app id encoded",
+        appIdStr1.equals(decodedAppId1));
+    assertTrue("Decoded app id is not same as the app id encoded",
+        appIdStr2.equals(decodedAppId2));
+    assertTrue("Decoded app id is not same as the app id encoded",
+        appIdStr3.equals(decodedAppId3));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestTimelineWriterUtils.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/TestTimelineWriterUtils.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/TestTimelineWriterUtils.java
deleted file mode 100644
index 4f96f87..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestTimelineWriterUtils.java
+++ /dev/null
@@ -1,29 +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.common;
-
-import org.junit.Test;
-
-public class TestTimelineWriterUtils {
-
-  @Test
-  public void test() {
-    // TODO: implement a test for the remaining method in TimelineWriterUtils.
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fdaa1e4e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
index 6bdec6b..c957dad 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
@@ -49,7 +49,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReader
 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.ColumnHelper;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -166,7 +166,7 @@ public class TestHBaseStorageFlowActivity {
     assertEquals(cluster, flowActivityRowKey.getClusterId());
     assertEquals(user, flowActivityRowKey.getUserId());
     assertEquals(flow, flowActivityRowKey.getFlowId());
-    long dayTs = TimelineWriterUtils.getTopOfTheDayTimestamp(System
+    long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(System
         .currentTimeMillis());
     assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
     assertEquals(1, values.size());
@@ -281,7 +281,7 @@ public class TestHBaseStorageFlowActivity {
       assertEquals(cluster, flowActivityRowKey.getClusterId());
       assertEquals(user, flowActivityRowKey.getUserId());
       assertEquals(flow, flowActivityRowKey.getFlowId());
-      long dayTs = TimelineWriterUtils.getTopOfTheDayTimestamp(System
+      long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(System
           .currentTimeMillis());
       assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
       assertEquals(1, values.size());
@@ -360,7 +360,7 @@ public class TestHBaseStorageFlowActivity {
         assertEquals(cluster, flowActivity.getCluster());
         assertEquals(user, flowActivity.getUser());
         assertEquals(flow, flowActivity.getFlowName());
-        long dayTs = TimelineWriterUtils.getTopOfTheDayTimestamp(System
+        long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(System
             .currentTimeMillis());
         assertEquals(dayTs, flowActivity.getDate().getTime());
         Set<FlowRunEntity> flowRuns = flowActivity.getFlowRuns();
@@ -410,7 +410,7 @@ public class TestHBaseStorageFlowActivity {
       assertEquals(cluster, flowActivityRowKey.getClusterId());
       assertEquals(user, flowActivityRowKey.getUserId());
       assertEquals(flow, flowActivityRowKey.getFlowId());
-      long dayTs = TimelineWriterUtils.getTopOfTheDayTimestamp(System
+      long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(System
           .currentTimeMillis());
       assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
 


[37/50] [abbrv] hadoop git commit: YARN-4450. TestTimelineAuthenticationFilter and TestYarnConfigurationFields fail. Contributed by Sangjin Lee.

Posted by gt...@apache.org.
YARN-4450. TestTimelineAuthenticationFilter and
TestYarnConfigurationFields fail. 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/93f2f13b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/93f2f13b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/93f2f13b

Branch: refs/heads/feature-YARN-2928
Commit: 93f2f13bc1c25fa441a3819c98b7073fca45b07a
Parents: 5591eeb
Author: Li Lu <gt...@apache.org>
Authored: Mon Dec 14 10:48:39 2015 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 18:02:43 2016 -0800

----------------------------------------------------------------------
 .../yarn/conf/TestYarnConfigurationFields.java    |  3 +++
 .../yarn/client/api/impl/TimelineClientImpl.java  | 18 ++++++++++--------
 2 files changed, 13 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/93f2f13b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
index 0e508ed..fe73276 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/conf/TestYarnConfigurationFields.java
@@ -94,6 +94,9 @@ public class TestYarnConfigurationFields extends TestConfigurationFieldsBase {
 
     // Ignore all YARN Application Timeline Service (version 1) properties
     configurationPrefixToSkipCompare.add("yarn.timeline-service.");
+    // skip deprecated RM_SYSTEM_METRICS_PUBLISHER_ENABLED
+    configurationPropsToSkipCompare
+        .add(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED);
 
     // Used as Java command line properties, not XML
     configurationPrefixToSkipCompare.add("yarn.app.container");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/93f2f13b/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 f07c479..61789a3 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
@@ -110,7 +110,6 @@ public class TimelineClientImpl extends TimelineClient {
   private ConnectionConfigurator connConfigurator;
   private DelegationTokenAuthenticator authenticator;
   private DelegationTokenAuthenticatedURL.Token token;
-  private URI resURI;
   private UserGroupInformation authUgi;
   private String doAsUser;
   private Configuration configuration;
@@ -542,8 +541,8 @@ public class TimelineClientImpl extends TimelineClient {
           @Override
           public Long run() throws Exception {
             // If the timeline DT to renew is different than cached, replace it.
-            // Token to set every time for retry, because when exception happens,
-            // DelegationTokenAuthenticatedURL will reset it to null;
+            // Token to set every time for retry, because when exception
+            // happens, DelegationTokenAuthenticatedURL will reset it to null;
             if (!timelineDT.equals(token.getDelegationToken())) {
               token.setDelegationToken((Token) timelineDT);
             }
@@ -552,7 +551,8 @@ public class TimelineClientImpl extends TimelineClient {
                     connConfigurator);
             // If the token service address is not available, fall back to use
             // the configured service address.
-            final URI serviceURI = isTokenServiceAddrEmpty ? resURI
+            final URI serviceURI = isTokenServiceAddrEmpty ?
+                constructResURI(getConfig(), getTimelineServiceAddress(), false)
                 : new URI(scheme, null, address.getHostName(),
                 address.getPort(), RESOURCE_URI_STR_V1, null, null);
             return authUrl
@@ -578,9 +578,10 @@ public class TimelineClientImpl extends TimelineClient {
 
           @Override
           public Void run() throws Exception {
-            // If the timeline DT to cancel is different than cached, replace it.
-            // Token to set every time for retry, because when exception happens,
-            // DelegationTokenAuthenticatedURL will reset it to null;
+            // If the timeline DT to cancel is different than cached, replace
+            // it.
+            // Token to set every time for retry, because when exception
+            // happens, DelegationTokenAuthenticatedURL will reset it to null;
             if (!timelineDT.equals(token.getDelegationToken())) {
               token.setDelegationToken((Token) timelineDT);
             }
@@ -589,7 +590,8 @@ public class TimelineClientImpl extends TimelineClient {
                     connConfigurator);
             // If the token service address is not available, fall back to use
             // the configured service address.
-            final URI serviceURI = isTokenServiceAddrEmpty ? resURI
+            final URI serviceURI = isTokenServiceAddrEmpty ?
+                constructResURI(getConfig(), getTimelineServiceAddress(), false)
                 : new URI(scheme, null, address.getHostName(),
                 address.getPort(), RESOURCE_URI_STR_V1, null, null);
             authUrl.cancelDelegationToken(serviceURI.toURL(), token, doAsUser);


[35/50] [abbrv] hadoop git commit: YARN-3862. Support for fetching specific configs and metrics based on prefixes (Varun Saxena via sjlee)

Posted by gt...@apache.org.
YARN-3862. Support for fetching specific configs and metrics based on prefixes (Varun Saxena via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: 76bc71cc7abe776420223a78f2d885800410ddc6
Parents: 54a529d
Author: Sangjin Lee <sj...@apache.org>
Authored: Tue Dec 1 21:47:43 2015 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:59:39 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../reader/TimelineReaderManager.java           |   4 +-
 .../reader/filter/TimelineCompareFilter.java    |  61 ++
 .../reader/filter/TimelineCompareOp.java        |  36 +
 .../reader/filter/TimelineFilter.java           |  56 ++
 .../reader/filter/TimelineFilterList.java       |  91 +++
 .../reader/filter/TimelineFilterUtils.java      | 120 ++++
 .../reader/filter/TimelinePrefixFilter.java     |  56 ++
 .../reader/filter/package-info.java             |  28 +
 .../storage/ApplicationEntityReader.java        | 123 +++-
 .../storage/FileSystemTimelineReaderImpl.java   |   9 +-
 .../storage/FlowActivityEntityReader.java       |  16 +-
 .../storage/FlowRunEntityReader.java            |  69 +-
 .../storage/GenericEntityReader.java            | 119 +++-
 .../storage/HBaseTimelineReaderImpl.java        |  11 +-
 .../storage/TimelineEntityReader.java           |  32 +-
 .../storage/TimelineEntityReaderFactory.java    |  23 +-
 .../timelineservice/storage/TimelineReader.java |  32 +
 .../application/ApplicationColumnPrefix.java    |  18 +-
 .../storage/common/ColumnPrefix.java            |  29 +-
 .../storage/entity/EntityColumnPrefix.java      |  18 +-
 .../storage/flow/FlowActivityColumnPrefix.java  |  18 +-
 .../storage/flow/FlowRunColumnPrefix.java       |  18 +-
 .../TestFileSystemTimelineReaderImpl.java       |  42 +-
 .../storage/TestHBaseTimelineStorage.java       | 682 +++++++++++++++++--
 .../flow/TestHBaseStorageFlowActivity.java      |   6 +-
 .../storage/flow/TestHBaseStorageFlowRun.java   | 190 +++++-
 27 files changed, 1761 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7636317..78705e1 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -143,6 +143,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4053. Change the way metric values are stored in HBase Storage (Varun
     Saxena via sjlee)
 
+    YARN-3862. Support for fetching specific configs and metrics based on
+    prefixes (Varun Saxena via sjlee)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/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 27a50d5..294b05b 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
@@ -77,7 +77,7 @@ public class TimelineReaderManager extends AbstractService {
     return reader.getEntities(userId, cluster, flowId, flowRunId, appId,
         entityType, limit, createdTimeBegin, createdTimeEnd, modifiedTimeBegin,
         modifiedTimeEnd, relatesTo, isRelatedTo, infoFilters, configFilters,
-        metricFilters, eventFilters, fieldsToRetrieve);
+        metricFilters, eventFilters, null, null, fieldsToRetrieve);
   }
 
   /**
@@ -91,6 +91,6 @@ public class TimelineReaderManager extends AbstractService {
       String entityId, EnumSet<Field> fields) throws IOException {
     String cluster = getClusterID(clusterId, getConfig());
     return reader.getEntity(userId, cluster, flowId, flowRunId, appId,
-        entityType, entityId, fields);
+        entityType, entityId, null, null, fields);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineCompareFilter.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/filter/TimelineCompareFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineCompareFilter.java
new file mode 100644
index 0000000..14e7124
--- /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/filter/TimelineCompareFilter.java
@@ -0,0 +1,61 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Filter class which represents filter to be applied based on key-value pair
+ * and the relation between them represented by different relational operators.
+ */
+@Private
+@Unstable
+public class TimelineCompareFilter extends TimelineFilter {
+
+  private TimelineCompareOp compareOp;
+  private String key;
+  private Object value;
+
+  public TimelineCompareFilter() {
+  }
+
+  public TimelineCompareFilter(TimelineCompareOp op, String key, Object val) {
+    this.compareOp = op;
+    this.key = key;
+    this.value = val;
+  }
+
+  @Override
+  public TimelineFilterType getFilterType() {
+    return TimelineFilterType.COMPARE;
+  }
+
+  public TimelineCompareOp getCompareOp() {
+    return compareOp;
+  }
+
+  public String getKey() {
+    return key;
+  }
+
+  public Object getValue() {
+    return value;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineCompareOp.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/filter/TimelineCompareOp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineCompareOp.java
new file mode 100644
index 0000000..461a7d8
--- /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/filter/TimelineCompareOp.java
@@ -0,0 +1,36 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ *  Comparison Operators.
+ */
+@Private
+@Unstable
+public enum TimelineCompareOp {
+  LESS_THAN,
+  LESS_OR_EQUAL,
+  EQUAL,
+  NOT_EQUAL,
+  GREATER_OR_EQUAL,
+  GREATER_THAN
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilter.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/filter/TimelineFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilter.java
new file mode 100644
index 0000000..d4b4045
--- /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/filter/TimelineFilter.java
@@ -0,0 +1,56 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Abstract base class extended to implement timeline filters.
+ */
+@Private
+@Unstable
+public abstract class TimelineFilter {
+
+  /**
+   * Lists the different filter types.
+   */
+  @Private
+  @Unstable
+  public enum TimelineFilterType {
+    /**
+     * Combines multiple filters.
+     */
+    LIST,
+    /**
+     * Filter which is used for comparison.
+     */
+    COMPARE,
+    /**
+     * Filter which matches prefix for a config or a metric.
+     */
+    PREFIX
+  }
+
+  public abstract TimelineFilterType getFilterType();
+
+  public String toString() {
+    return this.getClass().getSimpleName();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterList.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/filter/TimelineFilterList.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterList.java
new file mode 100644
index 0000000..8727bd7
--- /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/filter/TimelineFilterList.java
@@ -0,0 +1,91 @@
+/**
+ * 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.filter;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Implementation of {@link TimelineFilter} that represents an ordered list of
+ * timeline filters which will then be evaluated with a specified boolean
+ * operator {@link Operator#AND} or {@link Operator#OR}. Since you can use
+ * timeline filter lists as children of timeline filter lists, you can create a
+ * hierarchy of filters to be evaluated.
+ */
+@Private
+@Unstable
+public class TimelineFilterList extends TimelineFilter {
+  /**
+   * Specifies how filters in the filter list will be evaluated. AND means all
+   * the filters should match and OR means atleast one should match.
+   */
+  @Private
+  @Unstable
+  public static enum Operator {
+    AND,
+    OR
+  }
+
+  private Operator operator;
+  private List<TimelineFilter> filterList = new ArrayList<TimelineFilter>();
+
+  public TimelineFilterList(TimelineFilter...filters) {
+    this(Operator.AND, filters);
+  }
+
+  public TimelineFilterList(Operator op, TimelineFilter...filters) {
+    this.operator = op;
+    this.filterList = new ArrayList<TimelineFilter>(Arrays.asList(filters));
+  }
+
+  @Override
+  public TimelineFilterType getFilterType() {
+    return TimelineFilterType.LIST;
+  }
+
+  /**
+   * Get the filter list.
+   *
+   * @return filterList
+   */
+  public List<TimelineFilter> getFilterList() {
+    return filterList;
+  }
+
+  /**
+   * Get the operator.
+   *
+   * @return operator
+   */
+  public Operator getOperator() {
+    return operator;
+  }
+
+  public void setOperator(Operator op) {
+    operator = op;
+  }
+
+  public void addFilter(TimelineFilter filter) {
+    filterList.add(filter);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.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/filter/TimelineFilterUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineFilterUtils.java
new file mode 100644
index 0000000..da3c383
--- /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/filter/TimelineFilterUtils.java
@@ -0,0 +1,120 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.FilterList.Operator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
+
+/**
+ * Set of utility methods used by timeline filter classes.
+ */
+public final class TimelineFilterUtils {
+
+  private TimelineFilterUtils() {
+  }
+
+  /**
+   * Returns the equivalent HBase filter list's {@link Operator}.
+   * @param op
+   * @return HBase filter list's Operator.
+   */
+  private static Operator getHBaseOperator(TimelineFilterList.Operator op) {
+    switch (op) {
+    case AND:
+      return Operator.MUST_PASS_ALL;
+    case OR:
+      return Operator.MUST_PASS_ONE;
+    default:
+      throw new IllegalArgumentException("Invalid operator");
+    }
+  }
+
+  /**
+   * Returns the equivalent HBase compare filter's {@link CompareOp}.
+   * @param op
+   * @return HBase compare filter's CompareOp.
+   */
+  private static CompareOp getHBaseCompareOp(
+      TimelineCompareOp op) {
+    switch (op) {
+    case LESS_THAN:
+      return CompareOp.LESS;
+    case LESS_OR_EQUAL:
+      return CompareOp.LESS_OR_EQUAL;
+    case EQUAL:
+      return CompareOp.EQUAL;
+    case NOT_EQUAL:
+      return CompareOp.NOT_EQUAL;
+    case GREATER_OR_EQUAL:
+      return CompareOp.GREATER_OR_EQUAL;
+    case GREATER_THAN:
+      return CompareOp.GREATER;
+    default:
+      throw new IllegalArgumentException("Invalid compare operator");
+    }
+  }
+
+  /**
+   * Converts a {@link TimelinePrefixFilter} to an equivalent HBase
+   * {@link QualifierFilter}.
+   * @param colPrefix
+   * @param filter
+   * @return a {@link QualifierFilter} object
+   */
+  private static <T> Filter createHBaseColQualPrefixFilter(
+      ColumnPrefix<T> colPrefix, TimelinePrefixFilter filter) {
+    return new QualifierFilter(getHBaseCompareOp(filter.getCompareOp()),
+        new BinaryPrefixComparator(
+            colPrefix.getColumnPrefixBytes(filter.getPrefix())));
+  }
+
+  /**
+   * Creates equivalent HBase {@link FilterList} from {@link TimelineFilterList}
+   * while converting different timeline filters(of type {@link TimelineFilter})
+   * into their equivalent HBase filters.
+   * @param colPrefix
+   * @param filterList
+   * @return a {@link FilterList} object
+   */
+  public static <T> FilterList createHBaseFilterList(ColumnPrefix<T> colPrefix,
+      TimelineFilterList filterList) {
+    FilterList list =
+        new FilterList(getHBaseOperator(filterList.getOperator()));
+    for (TimelineFilter filter : filterList.getFilterList()) {
+      switch(filter.getFilterType()) {
+      case LIST:
+        list.addFilter(
+            createHBaseFilterList(colPrefix, (TimelineFilterList)filter));
+        break;
+      case PREFIX:
+        list.addFilter(createHBaseColQualPrefixFilter(
+            colPrefix, (TimelinePrefixFilter)filter));
+        break;
+      default:
+        break;
+      }
+    }
+    return list;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelinePrefixFilter.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/filter/TimelinePrefixFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelinePrefixFilter.java
new file mode 100644
index 0000000..6233f26
--- /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/filter/TimelinePrefixFilter.java
@@ -0,0 +1,56 @@
+/**
+ * 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.filter;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Filter class which represents filter to be applied based on prefixes.
+ * Prefixes can either match or not match.
+ */
+@Private
+@Unstable
+public class TimelinePrefixFilter extends TimelineFilter {
+
+  private TimelineCompareOp compareOp;
+  private String prefix;
+
+  public TimelinePrefixFilter(TimelineCompareOp op, String prefix) {
+    this.prefix = prefix;
+    if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
+      throw new IllegalArgumentException("CompareOp for prefix filter should " +
+          "be EQUAL or NOT_EQUAL");
+    }
+    this.compareOp = op;
+  }
+
+  @Override
+  public TimelineFilterType getFilterType() {
+    return TimelineFilterType.PREFIX;
+  }
+
+  public String getPrefix() {
+    return prefix;
+  }
+
+  public TimelineCompareOp getCompareOp() {
+    return compareOp;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/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/filter/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/filter/package-info.java
new file mode 100644
index 0000000..f7c0705
--- /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/filter/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.server.timelineservice.reader.filter stores
+ * timeline filter implementations.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.reader.filter;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/ApplicationEntityReader.java
index 8324afd..7082a5e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/ApplicationEntityReader.java
@@ -28,11 +28,21 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.FamilyFilter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.FilterList.Operator;
 import org.apache.hadoop.hbase.filter.PageFilter;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnFamily;
 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;
@@ -56,18 +66,21 @@ class ApplicationEntityReader extends GenericEntityReader {
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve) {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, fieldsToRetrieve, true);
+        eventFilters, confsToRetrieve, metricsToRetrieve, fieldsToRetrieve,
+        true);
   }
 
   public ApplicationEntityReader(String userId, String clusterId,
       String flowId, Long flowRunId, String appId, String entityType,
-      String entityId, EnumSet<Field> fieldsToRetrieve) {
+      String entityId, TimelineFilterList confsToRetrieve,
+      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, entityId,
-        fieldsToRetrieve);
+        confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
   }
 
   /**
@@ -78,13 +91,95 @@ class ApplicationEntityReader extends GenericEntityReader {
   }
 
   @Override
-  protected Result getResult(Configuration hbaseConf, Connection conn)
-      throws IOException {
+  protected FilterList constructFilterListBasedOnFields() {
+    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
+    // Fetch all the columns.
+    if (fieldsToRetrieve.contains(Field.ALL) &&
+        (confsToRetrieve == null ||
+        confsToRetrieve.getFilterList().isEmpty()) &&
+        (metricsToRetrieve == null ||
+        metricsToRetrieve.getFilterList().isEmpty())) {
+      return list;
+    }
+    FilterList infoColFamilyList = new FilterList();
+    // By default fetch everything in INFO column family.
+    FamilyFilter infoColumnFamily =
+        new FamilyFilter(CompareOp.EQUAL,
+           new BinaryComparator(ApplicationColumnFamily.INFO.getBytes()));
+    infoColFamilyList.addFilter(infoColumnFamily);
+    // Events not required.
+    if (!fieldsToRetrieve.contains(Field.EVENTS) &&
+        !fieldsToRetrieve.contains(Field.ALL) && eventFilters == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+          ApplicationColumnPrefix.EVENT.getColumnPrefixBytes(""))));
+    }
+    // info not required.
+    if (!fieldsToRetrieve.contains(Field.INFO) &&
+        !fieldsToRetrieve.contains(Field.ALL) && infoFilters == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+          ApplicationColumnPrefix.INFO.getColumnPrefixBytes(""))));
+    }
+    // is releated to not required.
+    if (!fieldsToRetrieve.contains(Field.IS_RELATED_TO) &&
+        !fieldsToRetrieve.contains(Field.ALL) && isRelatedTo == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+          ApplicationColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes(""))));
+    }
+    // relates to not required.
+    if (!fieldsToRetrieve.contains(Field.RELATES_TO) &&
+        !fieldsToRetrieve.contains(Field.ALL) && relatesTo == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+          ApplicationColumnPrefix.RELATES_TO.getColumnPrefixBytes(""))));
+    }
+    list.addFilter(infoColFamilyList);
+    if ((fieldsToRetrieve.contains(Field.CONFIGS) || configFilters != null) ||
+        (confsToRetrieve != null &&
+        !confsToRetrieve.getFilterList().isEmpty())) {
+      FilterList filterCfg =
+          new FilterList(new FamilyFilter(CompareOp.EQUAL,
+          new BinaryComparator(ApplicationColumnFamily.CONFIGS.getBytes())));
+      if (confsToRetrieve != null &&
+          !confsToRetrieve.getFilterList().isEmpty()) {
+        filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList(
+            ApplicationColumnPrefix.CONFIG, confsToRetrieve));
+      }
+      list.addFilter(filterCfg);
+    }
+    if ((fieldsToRetrieve.contains(Field.METRICS) || metricFilters != null) ||
+        (metricsToRetrieve != null &&
+        !metricsToRetrieve.getFilterList().isEmpty())) {
+      FilterList filterMetrics =
+          new FilterList(new FamilyFilter(CompareOp.EQUAL,
+          new BinaryComparator(ApplicationColumnFamily.METRICS.getBytes())));
+      if (metricsToRetrieve != null &&
+          !metricsToRetrieve.getFilterList().isEmpty()) {
+        filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList(
+            ApplicationColumnPrefix.METRIC, metricsToRetrieve));
+      }
+      list.addFilter(filterMetrics);
+    }
+    return list;
+  }
+
+  @Override
+  protected Result getResult(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException {
     byte[] rowKey =
         ApplicationRowKey.getRowKey(clusterId, userId, flowId, flowRunId,
             appId);
     Get get = new Get(rowKey);
     get.setMaxVersions(Integer.MAX_VALUE);
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      get.setFilter(filterList);
+    }
     return table.getResult(hbaseConf, conn, get);
   }
 
@@ -115,6 +210,15 @@ class ApplicationEntityReader extends GenericEntityReader {
     if (fieldsToRetrieve == null) {
       fieldsToRetrieve = EnumSet.noneOf(Field.class);
     }
+    if (!fieldsToRetrieve.contains(Field.CONFIGS) &&
+        confsToRetrieve != null && !confsToRetrieve.getFilterList().isEmpty()) {
+      fieldsToRetrieve.add(Field.CONFIGS);
+    }
+    if (!fieldsToRetrieve.contains(Field.METRICS) &&
+        metricsToRetrieve != null &&
+        !metricsToRetrieve.getFilterList().isEmpty()) {
+      fieldsToRetrieve.add(Field.METRICS);
+    }
     if (!singleEntityRead) {
       if (limit == null || limit < 0) {
         limit = TimelineReader.DEFAULT_LIMIT;
@@ -136,7 +240,7 @@ class ApplicationEntityReader extends GenericEntityReader {
 
   @Override
   protected ResultScanner getResults(Configuration hbaseConf,
-      Connection conn) throws IOException {
+      Connection conn, FilterList filterList) throws IOException {
     Scan scan = new Scan();
     if (flowRunId != null) {
       scan.setRowPrefixFilter(ApplicationRowKey.
@@ -145,7 +249,12 @@ class ApplicationEntityReader extends GenericEntityReader {
       scan.setRowPrefixFilter(ApplicationRowKey.
           getRowKeyPrefix(clusterId, userId, flowId));
     }
-    scan.setFilter(new PageFilter(limit));
+    FilterList newList = new FilterList();
+    newList.addFilter(new PageFilter(limit));
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      newList.addFilter(filterList);
+    }
+    scan.setFilter(newList);
     return table.getResultScanner(hbaseConf, conn, scan);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/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 30d1d00..48bf844 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
@@ -44,6 +44,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.codehaus.jackson.JsonGenerationException;
@@ -272,6 +273,7 @@ public class FileSystemTimelineReaderImpl extends AbstractService
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve) throws IOException {
     if (limit == null || limit <= 0) {
       limit = DEFAULT_LIMIT;
@@ -386,7 +388,9 @@ public class FileSystemTimelineReaderImpl extends AbstractService
   @Override
   public TimelineEntity getEntity(String userId, String clusterId,
       String flowId, Long flowRunId, String appId, String entityType,
-      String entityId, EnumSet<Field> fieldsToRetrieve) throws IOException {
+      String entityId, TimelineFilterList confsToRetrieve,
+      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve)
+      throws IOException {
     String flowRunPath = getFlowRunPath(userId, clusterId, flowId,
         flowRunId, appId);
     File dir = new File(new File(rootPath, ENTITIES_DIR),
@@ -413,6 +417,7 @@ public class FileSystemTimelineReaderImpl extends AbstractService
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve) throws IOException {
     String flowRunPath =
         getFlowRunPath(userId, clusterId, flowId, flowRunId, appId);
@@ -422,6 +427,6 @@ public class FileSystemTimelineReaderImpl extends AbstractService
     return getEntities(dir, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, fieldsToRetrieve);
+        eventFilters, confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.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/FlowActivityEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
index 3e32128..71dd0a1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.PageFilter;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowActivityEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
@@ -58,14 +59,14 @@ class FlowActivityEntityReader extends TimelineEntityReader {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, fieldsToRetrieve, true);
+        eventFilters, null, null, fieldsToRetrieve, true);
   }
 
   public FlowActivityEntityReader(String userId, String clusterId,
       String flowId, Long flowRunId, String appId, String entityType,
       String entityId, EnumSet<Field> fieldsToRetrieve) {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, entityId,
-        fieldsToRetrieve);
+        null, null, fieldsToRetrieve);
   }
 
   /**
@@ -96,15 +97,20 @@ class FlowActivityEntityReader extends TimelineEntityReader {
   }
 
   @Override
-  protected Result getResult(Configuration hbaseConf, Connection conn)
-      throws IOException {
+  protected FilterList constructFilterListBasedOnFields() {
+    return null;
+  }
+
+  @Override
+  protected Result getResult(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException {
     throw new UnsupportedOperationException(
         "we don't support a single entity query");
   }
 
   @Override
   protected ResultScanner getResults(Configuration hbaseConf,
-      Connection conn) throws IOException {
+      Connection conn, FilterList filterList) throws IOException {
     Scan scan = new Scan();
     if (createdTimeBegin == DEFAULT_BEGIN_TIME &&
         createdTimeEnd == DEFAULT_END_TIME) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/FlowRunEntityReader.java
index ebf2d27..1895fa6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/FlowRunEntityReader.java
@@ -28,12 +28,22 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
+import org.apache.hadoop.hbase.filter.FamilyFilter;
+import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.PageFilter;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.FilterList.Operator;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnFamily;
 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;
@@ -54,18 +64,20 @@ class FlowRunEntityReader extends TimelineEntityReader {
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve) {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, fieldsToRetrieve, true);
+        eventFilters, null, metricsToRetrieve, fieldsToRetrieve, true);
   }
 
   public FlowRunEntityReader(String userId, String clusterId,
       String flowId, Long flowRunId, String appId, String entityType,
-      String entityId, EnumSet<Field> fieldsToRetrieve) {
+      String entityId, TimelineFilterList confsToRetrieve,
+      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, entityId,
-        fieldsToRetrieve);
+        null, metricsToRetrieve, fieldsToRetrieve);
   }
 
   /**
@@ -101,26 +113,69 @@ class FlowRunEntityReader extends TimelineEntityReader {
       if (createdTimeEnd == null) {
         createdTimeEnd = DEFAULT_END_TIME;
       }
+      if (!fieldsToRetrieve.contains(Field.METRICS) &&
+          metricsToRetrieve != null &&
+          !metricsToRetrieve.getFilterList().isEmpty()) {
+        fieldsToRetrieve.add(Field.METRICS);
+      }
+    }
+  }
+
+  @Override
+  protected FilterList constructFilterListBasedOnFields() {
+    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
+
+    // By default fetch everything in INFO column family.
+    FamilyFilter infoColumnFamily =
+        new FamilyFilter(CompareOp.EQUAL,
+           new BinaryComparator(FlowRunColumnFamily.INFO.getBytes()));
+    // Metrics not required.
+    if (!singleEntityRead && !fieldsToRetrieve.contains(Field.METRICS) &&
+        !fieldsToRetrieve.contains(Field.ALL)) {
+      FilterList infoColFamilyList = new FilterList(Operator.MUST_PASS_ONE);
+      infoColFamilyList.addFilter(infoColumnFamily);
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(""))));
+      list.addFilter(infoColFamilyList);
+    }
+    if (metricsToRetrieve != null &&
+        !metricsToRetrieve.getFilterList().isEmpty()) {
+      FilterList infoColFamilyList = new FilterList();
+      infoColFamilyList.addFilter(infoColumnFamily);
+      infoColFamilyList.addFilter(TimelineFilterUtils.createHBaseFilterList(
+          FlowRunColumnPrefix.METRIC, metricsToRetrieve));
+      list.addFilter(infoColFamilyList);
     }
+    return list;
   }
 
   @Override
-  protected Result getResult(Configuration hbaseConf, Connection conn)
-      throws IOException {
+  protected Result getResult(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException {
     byte[] rowKey =
         FlowRunRowKey.getRowKey(clusterId, userId, flowId, flowRunId);
     Get get = new Get(rowKey);
     get.setMaxVersions(Integer.MAX_VALUE);
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      get.setFilter(filterList);
+    }
     return table.getResult(hbaseConf, conn, get);
   }
 
   @Override
   protected ResultScanner getResults(Configuration hbaseConf,
-      Connection conn) throws IOException {
+      Connection conn, FilterList filterList) throws IOException {
     Scan scan = new Scan();
     scan.setRowPrefixFilter(
         FlowRunRowKey.getRowKeyPrefix(clusterId, userId, flowId));
-    scan.setFilter(new PageFilter(limit));
+    FilterList newList = new FilterList();
+    newList.addFilter(new PageFilter(limit));
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      newList.addFilter(filterList);
+    }
+    scan.setFilter(newList);
     return table.getResultScanner(hbaseConf, conn, scan);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.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/GenericEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
index 04fc8ee..dcb8b89 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
@@ -32,9 +32,18 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
+import org.apache.hadoop.hbase.filter.FamilyFilter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.FilterList.Operator;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
@@ -46,6 +55,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
@@ -72,18 +82,21 @@ class GenericEntityReader extends TimelineEntityReader {
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve, boolean sortedKeys) {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, limit,
         createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
         relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, fieldsToRetrieve, sortedKeys);
+        eventFilters, confsToRetrieve, metricsToRetrieve, fieldsToRetrieve,
+        sortedKeys);
   }
 
   public GenericEntityReader(String userId, String clusterId,
       String flowId, Long flowRunId, String appId, String entityType,
-      String entityId, EnumSet<Field> fieldsToRetrieve) {
+      String entityId, TimelineFilterList confsToRetrieve,
+      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
     super(userId, clusterId, flowId, flowRunId, appId, entityType, entityId,
-        fieldsToRetrieve);
+        confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
   }
 
   /**
@@ -93,6 +106,85 @@ class GenericEntityReader extends TimelineEntityReader {
     return ENTITY_TABLE;
   }
 
+  @Override
+  protected FilterList constructFilterListBasedOnFields() {
+    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
+    // Fetch all the columns.
+    if (fieldsToRetrieve.contains(Field.ALL) &&
+        (confsToRetrieve == null ||
+        confsToRetrieve.getFilterList().isEmpty()) &&
+        (metricsToRetrieve == null ||
+        metricsToRetrieve.getFilterList().isEmpty())) {
+      return list;
+    }
+    FilterList infoColFamilyList = new FilterList();
+    // By default fetch everything in INFO column family.
+    FamilyFilter infoColumnFamily =
+        new FamilyFilter(CompareOp.EQUAL,
+           new BinaryComparator(EntityColumnFamily.INFO.getBytes()));
+    infoColFamilyList.addFilter(infoColumnFamily);
+    // Events not required.
+    if (!fieldsToRetrieve.contains(Field.EVENTS) &&
+        !fieldsToRetrieve.contains(Field.ALL) && eventFilters == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+          EntityColumnPrefix.EVENT.getColumnPrefixBytes(""))));
+    }
+    // info not required.
+    if (!fieldsToRetrieve.contains(Field.INFO) &&
+        !fieldsToRetrieve.contains(Field.ALL) && infoFilters == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+              EntityColumnPrefix.INFO.getColumnPrefixBytes(""))));
+    }
+    // is related to not required.
+    if (!fieldsToRetrieve.contains(Field.IS_RELATED_TO) &&
+        !fieldsToRetrieve.contains(Field.ALL) && isRelatedTo == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+              EntityColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes(""))));
+    }
+    // relates to not required.
+    if (!fieldsToRetrieve.contains(Field.RELATES_TO) &&
+        !fieldsToRetrieve.contains(Field.ALL) && relatesTo == null) {
+      infoColFamilyList.addFilter(
+          new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(
+              EntityColumnPrefix.RELATES_TO.getColumnPrefixBytes(""))));
+    }
+    list.addFilter(infoColFamilyList);
+    if ((fieldsToRetrieve.contains(Field.CONFIGS) || configFilters != null) ||
+        (confsToRetrieve != null &&
+        !confsToRetrieve.getFilterList().isEmpty())) {
+      FilterList filterCfg =
+          new FilterList(new FamilyFilter(CompareOp.EQUAL,
+              new BinaryComparator(EntityColumnFamily.CONFIGS.getBytes())));
+      if (confsToRetrieve != null &&
+          !confsToRetrieve.getFilterList().isEmpty()) {
+        filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList(
+            EntityColumnPrefix.CONFIG, confsToRetrieve));
+      }
+      list.addFilter(filterCfg);
+    }
+    if ((fieldsToRetrieve.contains(Field.METRICS) || metricFilters != null) ||
+        (metricsToRetrieve != null &&
+        !metricsToRetrieve.getFilterList().isEmpty())) {
+      FilterList filterMetrics =
+          new FilterList(new FamilyFilter(CompareOp.EQUAL,
+              new BinaryComparator(EntityColumnFamily.METRICS.getBytes())));
+      if (metricsToRetrieve != null &&
+          !metricsToRetrieve.getFilterList().isEmpty()) {
+        filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList(
+            EntityColumnPrefix.METRIC, metricsToRetrieve));
+      }
+      list.addFilter(filterMetrics);
+    }
+    return list;
+  }
+
   protected FlowContext lookupFlowContext(String clusterId, String appId,
       Configuration hbaseConf, Connection conn) throws IOException {
     byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
@@ -145,6 +237,15 @@ class GenericEntityReader extends TimelineEntityReader {
     if (fieldsToRetrieve == null) {
       fieldsToRetrieve = EnumSet.noneOf(Field.class);
     }
+    if (!fieldsToRetrieve.contains(Field.CONFIGS) &&
+        confsToRetrieve != null && !confsToRetrieve.getFilterList().isEmpty()) {
+      fieldsToRetrieve.add(Field.CONFIGS);
+    }
+    if (!fieldsToRetrieve.contains(Field.METRICS) &&
+        metricsToRetrieve != null &&
+        !metricsToRetrieve.getFilterList().isEmpty()) {
+      fieldsToRetrieve.add(Field.METRICS);
+    }
     if (!singleEntityRead) {
       if (limit == null || limit < 0) {
         limit = TimelineReader.DEFAULT_LIMIT;
@@ -165,25 +266,31 @@ class GenericEntityReader extends TimelineEntityReader {
   }
 
   @Override
-  protected Result getResult(Configuration hbaseConf, Connection conn)
-      throws IOException {
+  protected Result getResult(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException {
     byte[] rowKey =
         EntityRowKey.getRowKey(clusterId, userId, flowId, flowRunId, appId,
             entityType, entityId);
     Get get = new Get(rowKey);
     get.setMaxVersions(Integer.MAX_VALUE);
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      get.setFilter(filterList);
+    }
     return table.getResult(hbaseConf, conn, get);
   }
 
   @Override
   protected ResultScanner getResults(Configuration hbaseConf,
-      Connection conn) throws IOException {
+      Connection conn, FilterList filterList) throws IOException {
     // Scan through part of the table to find the entities belong to one app
     // and one type
     Scan scan = new Scan();
     scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
         clusterId, userId, flowId, flowRunId, appId, entityType));
     scan.setMaxVersions(Integer.MAX_VALUE);
+    if (filterList != null && !filterList.getFilters().isEmpty()) {
+      scan.setFilter(filterList);
+    }
     return table.getResultScanner(hbaseConf, conn, scan);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.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/HBaseTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
index 889ae19..9e4b26a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
 
 public class HBaseTimelineReaderImpl
     extends AbstractService implements TimelineReader {
@@ -64,11 +65,13 @@ public class HBaseTimelineReaderImpl
   @Override
   public TimelineEntity getEntity(String userId, String clusterId,
       String flowId, Long flowRunId, String appId, String entityType,
-      String entityId, EnumSet<Field> fieldsToRetrieve)
+      String entityId, TimelineFilterList confsToRetrieve,
+      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve)
       throws IOException {
     TimelineEntityReader reader =
         TimelineEntityReaderFactory.createSingleEntityReader(userId, clusterId,
-            flowId, flowRunId, appId, entityType, entityId, fieldsToRetrieve);
+            flowId, flowRunId, appId, entityType, entityId, confsToRetrieve,
+            metricsToRetrieve, fieldsToRetrieve);
     return reader.readEntity(hbaseConf, conn);
   }
 
@@ -80,13 +83,15 @@ public class HBaseTimelineReaderImpl
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve) throws IOException {
     TimelineEntityReader reader =
         TimelineEntityReaderFactory.createMultipleEntitiesReader(userId,
             clusterId, flowId, flowRunId, appId, entityType, limit,
             createdTimeBegin, createdTimeEnd, modifiedTimeBegin,
             modifiedTimeEnd, relatesTo, isRelatedTo, infoFilters, configFilters,
-            metricFilters, eventFilters, fieldsToRetrieve);
+            metricFilters, eventFilters, confsToRetrieve, metricsToRetrieve,
+            fieldsToRetrieve);
     return reader.readEntities(hbaseConf, conn);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.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/TimelineEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
index adaf42e..7178aab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
@@ -31,8 +31,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.filter.FilterList;
 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.reader.filter.TimelineFilterList;
 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.ColumnPrefix;
@@ -70,6 +72,8 @@ abstract class TimelineEntityReader {
   protected Map<String, String> configFilters;
   protected Set<String> metricFilters;
   protected Set<String> eventFilters;
+  protected TimelineFilterList confsToRetrieve;
+  protected TimelineFilterList metricsToRetrieve;
 
   /**
    * Main table the entity reader uses.
@@ -94,6 +98,7 @@ abstract class TimelineEntityReader {
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve, boolean sortedKeys) {
     this.singleEntityRead = false;
     this.sortedKeys = sortedKeys;
@@ -115,6 +120,8 @@ abstract class TimelineEntityReader {
     this.configFilters = configFilters;
     this.metricFilters = metricFilters;
     this.eventFilters = eventFilters;
+    this.confsToRetrieve = confsToRetrieve;
+    this.metricsToRetrieve = metricsToRetrieve;
 
     this.table = getTable();
   }
@@ -124,7 +131,8 @@ abstract class TimelineEntityReader {
    */
   protected TimelineEntityReader(String userId, String clusterId,
       String flowId, Long flowRunId, String appId, String entityType,
-      String entityId, EnumSet<Field> fieldsToRetrieve) {
+      String entityId, TimelineFilterList confsToRetrieve,
+      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
     this.singleEntityRead = true;
     this.userId = userId;
     this.clusterId = clusterId;
@@ -134,11 +142,21 @@ abstract class TimelineEntityReader {
     this.entityType = entityType;
     this.fieldsToRetrieve = fieldsToRetrieve;
     this.entityId = entityId;
+    this.confsToRetrieve = confsToRetrieve;
+    this.metricsToRetrieve = metricsToRetrieve;
 
     this.table = getTable();
   }
 
   /**
+   * Creates a {@link FilterList} based on fields, confs and metrics to
+   * retrieve. This filter list will be set in Scan/Get objects to trim down
+   * results fetched from HBase back-end storage.
+   * @return a {@link FilterList} object.
+   */
+  protected abstract FilterList constructFilterListBasedOnFields();
+
+  /**
    * Reads and deserializes a single timeline entity from the HBase storage.
    */
   public TimelineEntity readEntity(Configuration hbaseConf, Connection conn)
@@ -146,7 +164,8 @@ abstract class TimelineEntityReader {
     validateParams();
     augmentParams(hbaseConf, conn);
 
-    Result result = getResult(hbaseConf, conn);
+    FilterList filterList = constructFilterListBasedOnFields();
+    Result result = getResult(hbaseConf, conn, filterList);
     if (result == null || result.isEmpty()) {
       // Could not find a matching row.
       LOG.info("Cannot find matching entity of type " + entityType);
@@ -166,7 +185,8 @@ abstract class TimelineEntityReader {
     augmentParams(hbaseConf, conn);
 
     NavigableSet<TimelineEntity> entities = new TreeSet<>();
-    ResultScanner results = getResults(hbaseConf, conn);
+    FilterList filterList = constructFilterListBasedOnFields();
+    ResultScanner results = getResults(hbaseConf, conn, filterList);
     try {
       for (Result result : results) {
         TimelineEntity entity = parseEntity(result);
@@ -211,14 +231,14 @@ abstract class TimelineEntityReader {
    *
    * @return the {@link Result} instance or null if no such record is found.
    */
-  protected abstract Result getResult(Configuration hbaseConf, Connection conn)
-      throws IOException;
+  protected abstract Result getResult(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException;
 
   /**
    * Fetches a {@link ResultScanner} for a multi-entity read.
    */
   protected abstract ResultScanner getResults(Configuration hbaseConf,
-      Connection conn) throws IOException;
+      Connection conn, FilterList filterList) throws IOException;
 
   /**
    * Given a {@link Result} instance, deserializes and creates a

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.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/TimelineEntityReaderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.java
index f5341c2..16204c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.java
@@ -22,6 +22,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 
 /**
@@ -34,22 +35,23 @@ class TimelineEntityReaderFactory {
    */
   public static TimelineEntityReader createSingleEntityReader(String userId,
       String clusterId, String flowId, Long flowRunId, String appId,
-      String entityType, String entityId, EnumSet<Field> fieldsToRetrieve) {
+      String entityType, String entityId, TimelineFilterList confs,
+      TimelineFilterList metrics, EnumSet<Field> fieldsToRetrieve) {
     // currently the types that are handled separate from the generic entity
     // table are application, flow run, and flow activity entities
     if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) {
       return new ApplicationEntityReader(userId, clusterId, flowId, flowRunId,
-          appId, entityType, entityId, fieldsToRetrieve);
+          appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
     } else if (TimelineEntityType.YARN_FLOW_RUN.matches(entityType)) {
       return new FlowRunEntityReader(userId, clusterId, flowId, flowRunId,
-          appId, entityType, entityId, fieldsToRetrieve);
+          appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
     } else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) {
       return new FlowActivityEntityReader(userId, clusterId, flowId, flowRunId,
           appId, entityType, entityId, fieldsToRetrieve);
     } else {
       // assume we're dealing with a generic entity read
       return new GenericEntityReader(userId, clusterId, flowId, flowRunId,
-        appId, entityType, entityId, fieldsToRetrieve);
+        appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
     }
   }
 
@@ -64,6 +66,7 @@ class TimelineEntityReaderFactory {
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String> metricFilters, Set<String> eventFilters,
+      TimelineFilterList confs, TimelineFilterList metrics,
       EnumSet<Field> fieldsToRetrieve) {
     // currently the types that are handled separate from the generic entity
     // table are application, flow run, and flow activity entities
@@ -71,8 +74,8 @@ class TimelineEntityReaderFactory {
       return new ApplicationEntityReader(userId, clusterId, flowId, flowRunId,
           appId, entityType, limit, createdTimeBegin, createdTimeEnd,
           modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
-          infoFilters, configFilters, metricFilters, eventFilters,
-          fieldsToRetrieve);
+          infoFilters, configFilters, metricFilters, eventFilters, confs,
+          metrics, fieldsToRetrieve);
     } else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) {
       return new FlowActivityEntityReader(userId, clusterId, flowId, flowRunId,
           appId, entityType, limit, createdTimeBegin, createdTimeEnd,
@@ -83,15 +86,15 @@ class TimelineEntityReaderFactory {
       return new FlowRunEntityReader(userId, clusterId, flowId, flowRunId,
           appId, entityType, limit, createdTimeBegin, createdTimeEnd,
           modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
-          infoFilters, configFilters, metricFilters, eventFilters,
-          fieldsToRetrieve);
+          infoFilters, configFilters, metricFilters, eventFilters, confs,
+          metrics, fieldsToRetrieve);
     } else {
       // assume we're dealing with a generic entity read
       return new GenericEntityReader(userId, clusterId, flowId, flowRunId,
           appId, entityType, limit, createdTimeBegin, createdTimeEnd,
           modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
-          infoFilters, configFilters, metricFilters, eventFilters,
-          fieldsToRetrieve, false);
+          infoFilters, configFilters, metricFilters, eventFilters, confs,
+          metrics, fieldsToRetrieve, false);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.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/TimelineReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java
index e4e305e..0ed17da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineReader.java
@@ -27,6 +27,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.service.Service;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
 
 /** ATSv2 reader interface. */
 @Private
@@ -70,6 +72,18 @@ public interface TimelineReader extends Service {
    *    Entity type (mandatory)
    * @param entityId
    *    Entity Id (mandatory)
+   * @param confsToRetrieve
+   *    Used for deciding which configs to return in response. This is
+   *    represented as a {@link TimelineFilterList} object containing
+   *    {@link TimelinePrefixFilter} objects. These can either be exact config
+   *    keys' or prefixes which are then compared against config keys' to decide
+   *    configs to return in response.
+   * @param metricsToRetrieve
+   *    Used for deciding which metrics to return in response. This is
+   *    represented as a {@link TimelineFilterList} object containing
+   *    {@link TimelinePrefixFilter} objects. These can either be exact metric
+   *    ids' or prefixes which are then compared against metric ids' to decide
+   *    metrics to return in response.
    * @param fieldsToRetrieve
    *    Specifies which fields of the entity object to retrieve(optional), see
    *    {@link Field}. If null, retrieves 4 fields namely entity id,
@@ -81,6 +95,7 @@ public interface TimelineReader extends Service {
    */
   TimelineEntity getEntity(String userId, String clusterId, String flowId,
       Long flowRunId, String appId, String entityType, String entityId,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve) throws IOException;
 
   /**
@@ -139,6 +154,22 @@ public interface TimelineReader extends Service {
    * @param eventFilters
    *    Matched entities should contain the given events (optional). If null
    *    or empty, the filter is not applied.
+   * @param confsToRetrieve
+   *    Used for deciding which configs to return in response. This is
+   *    represented as a {@link TimelineFilterList} object containing
+   *    {@link TimelinePrefixFilter} objects. These can either be exact config
+   *    keys' or prefixes which are then compared against config keys' to decide
+   *    configs(inside entities) to return in response. This should not be
+   *    confused with configFilters which is used to decide which entities to
+   *    return instead.
+   * @param metricsToRetrieve
+   *    Used for deciding which metrics to return in response. This is
+   *    represented as a {@link TimelineFilterList} object containing
+   *    {@link TimelinePrefixFilter} objects. These can either be exact metric
+   *    ids' or prefixes which are then compared against metric ids' to decide
+   *    metrics(inside entities) to return in response. This should not be
+   *    confused with metricFilters which is used to decide which entities to
+   *    return instead.
    * @param fieldsToRetrieve
    *    Specifies which fields of the entity object to retrieve(optional), see
    *    {@link Field}. If null, retrieves 4 fields namely entity id,
@@ -158,5 +189,6 @@ public interface TimelineReader extends Service {
       Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String>  metricFilters, Set<String> eventFilters,
+      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
       EnumSet<Field> fieldsToRetrieve) throws IOException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
index b06f5c1..056e51f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationColumnPrefix.java
@@ -119,6 +119,18 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
     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);
+  }
+
   /*
    * (non-Javadoc)
    *
@@ -139,8 +151,7 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
           + tableMutator.getName().getNameAsString());
     }
 
-    byte[] columnQualifier =
-        ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
+    byte[] columnQualifier = getColumnPrefixBytes(qualifier);
 
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         attributes);
@@ -166,8 +177,7 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
           + tableMutator.getName().getNameAsString());
     }
 
-    byte[] columnQualifier =
-        ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
+    byte[] columnQualifier = getColumnPrefixBytes(qualifier);
 
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         attributes);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.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/ColumnPrefix.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/ColumnPrefix.java
index db49098..0f3ac4e 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/ColumnPrefix.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/ColumnPrefix.java
@@ -44,13 +44,13 @@ public interface ColumnPrefix<T> {
    * @param qualifier column qualifier. Nothing gets written when null.
    * @param timestamp version timestamp. When null the server timestamp will be
    *          used.
-   *@param attributes attributes for the mutation that are used by the coprocessor
-   *          to set/read the cell tags
+   * @param attributes attributes for the mutation that are used by the
+   *          coprocessor to set/read the cell tags.
    * @param inputValue the value to write to the rowKey and column qualifier.
    *          Nothing gets written when null.
    * @throws IOException
    */
-  public void store(byte[] rowKey, TypedBufferedMutator<T> tableMutator,
+  void store(byte[] rowKey, TypedBufferedMutator<T> tableMutator,
       byte[] qualifier, Long timestamp, Object inputValue,
       Attribute... attributes) throws IOException;
 
@@ -65,13 +65,13 @@ public interface ColumnPrefix<T> {
    * @param qualifier column qualifier. Nothing gets written when null.
    * @param timestamp version timestamp. When null the server timestamp will be
    *          used.
-   *@param attributes attributes for the mutation that are used by the coprocessor
-   *          to set/read the cell tags
+   * @param attributes attributes for the mutation that are used by the
+   *          coprocessor to set/read the cell tags.
    * @param inputValue the value to write to the rowKey and column qualifier.
    *          Nothing gets written when null.
    * @throws IOException
    */
-  public void store(byte[] rowKey, TypedBufferedMutator<T> tableMutator,
+  void store(byte[] rowKey, TypedBufferedMutator<T> tableMutator,
       String qualifier, Long timestamp, Object inputValue,
       Attribute... attributes) throws IOException;
 
@@ -86,7 +86,7 @@ public interface ColumnPrefix<T> {
    *         in the result.
    * @throws IOException
    */
-  public Object readResult(Result result, String qualifier) throws IOException;
+  Object readResult(Result result, String qualifier) throws IOException;
 
   /**
    * @param result from which to read columns
@@ -94,7 +94,7 @@ public interface ColumnPrefix<T> {
    *         (or all of them if the prefix value is null).
    * @throws IOException
    */
-  public Map<String, Object> readResults(Result result) throws IOException;
+  Map<String, Object> readResults(Result result) throws IOException;
 
   /**
    * @param result from which to reads data with timestamps
@@ -104,7 +104,18 @@ public interface ColumnPrefix<T> {
    *         idB={timestamp3->value3}, idC={timestamp1->value4}}
    * @throws IOException
    */
-  public <V> NavigableMap<String, NavigableMap<Long, V>>
+  <V> NavigableMap<String, NavigableMap<Long, V>>
       readResultsWithTimestamps(Result result) throws IOException;
 
+  /**
+   * @param qualifierPrefix Column qualifier or prefix of qualifier.
+   * @return a byte array encoding column prefix and qualifier/prefix passed.
+   */
+  byte[] getColumnPrefixBytes(String qualifierPrefix);
+
+  /**
+   * @param qualifierPrefix Column qualifier or prefix of qualifier.
+   * @return a byte array encoding column prefix and qualifier/prefix passed.
+   */
+  byte[] getColumnPrefixBytes(byte[] qualifierPrefix);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
index abede9c..5b71228 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
@@ -119,6 +119,18 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
     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);
+  }
+
   /*
    * (non-Javadoc)
    *
@@ -140,8 +152,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
           + tableMutator.getName().getNameAsString());
     }
 
-    byte[] columnQualifier =
-        ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
+    byte[] columnQualifier = getColumnPrefixBytes(qualifier);
 
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         attributes);
@@ -167,8 +178,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
           + tableMutator.getName().getNameAsString());
     }
 
-    byte[] columnQualifier =
-        ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
+    byte[] columnQualifier = getColumnPrefixBytes(qualifier);
 
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         attributes);


[40/50] [abbrv] hadoop git commit: YARN-4392. ApplicationCreatedEvent event time resets after RM restart/failover. Contributed by Naganarasimha G R and Xuan Gong

Posted by gt...@apache.org.
YARN-4392. ApplicationCreatedEvent event time resets after RM
restart/failover. Contributed by Naganarasimha G R and Xuan Gong

(cherry picked from commit 4546c7582b6762c18ba150d80a8976eb51a8290c)

Conflicts:
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java


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

Branch: refs/heads/feature-YARN-2928
Commit: ee5ebf0539ba56d99747061d254f0045fd7f506a
Parents: 93f2f13
Author: Xuan <xg...@apache.org>
Authored: Mon Dec 7 12:24:55 2015 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 18:03:30 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                             | 3 +++
 .../hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java | 9 +++++++++
 .../server/resourcemanager/rmapp/TestRMAppTransitions.java  | 2 --
 3 files changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee5ebf05/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 78705e1..c291689 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -1471,6 +1471,9 @@ Release 2.8.0 - UNRELEASED
     YARN-4565. Fix a bug that leads to AM resource limit not hornored when
     sizeBasedWeight enabled for FairOrderingPolicy. (wtan via jianhe)
 
+    YARN-4392. ApplicationCreatedEvent event time resets after RM restart/failover.
+    (Naganarasimha G R and Xuan Gong via xgong)
+
 Release 2.7.3 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee5ebf05/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.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/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 51dd8b3..0872553 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -896,6 +896,9 @@ public class RMAppImpl implements RMApp, Recoverable {
     //TODO recover collector address.
     //this.collectorAddr = appState.getCollectorAddr();
 
+    // send the ATS create Event
+    sendATSCreateEvent(this, this.startTime);
+
     for(int i=0; i<appState.getAttemptCount(); ++i) {
       // create attempt
       createNewAttempt();
@@ -1838,16 +1841,20 @@ public class RMAppImpl implements RMApp, Recoverable {
     }
     return amNodeLabelExpression;
   }
+<<<<<<< 93f2f13bc1c25fa441a3819c98b7073fca45b07a
   
   @Override
   public CallerContext getCallerContext() {
     return callerContext;
   }
+=======
+>>>>>>> YARN-4392. ApplicationCreatedEvent event time resets after RM
 
   private void sendATSCreateEvent(RMApp app, long startTime) {
     rmContext.getRMApplicationHistoryWriter().applicationStarted(app);
     rmContext.getSystemMetricsPublisher().appCreated(app, startTime);
   }
+<<<<<<< 93f2f13bc1c25fa441a3819c98b7073fca45b07a
 
   @VisibleForTesting
   public boolean isAmBlacklistingEnabled() {
@@ -1858,4 +1865,6 @@ public class RMAppImpl implements RMApp, Recoverable {
   public float getAmBlacklistingDisableThreshold() {
     return blacklistDisableThreshold;
   }
+=======
+>>>>>>> YARN-4392. ApplicationCreatedEvent event time resets after RM
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ee5ebf05/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.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/rmapp/TestRMAppTransitions.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
index 2036c41..e04117b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
 
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.ApplicationStateData;
-
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Mockito.doReturn;


[43/50] [abbrv] hadoop git commit: YARN-4460. [Bug fix] RM fails to start when SMP is enabled. (Li Lu via Varun Saxena)

Posted by gt...@apache.org.
YARN-4460. [Bug fix] RM fails to start when SMP is enabled. (Li Lu via Varun Saxena)


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

Branch: refs/heads/feature-YARN-2928
Commit: 5de8b14c74b3d6e62f6315b204996df229b7d161
Parents: ee5ebf0
Author: Varun Saxena <va...@apache.org>
Authored: Wed Dec 16 15:24:57 2015 +0530
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 18:03:30 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                             | 2 ++
 .../resourcemanager/metrics/TimelineServiceV2Publisher.java | 9 +++++----
 .../metrics/TestSystemMetricsPublisherForV2.java            | 5 ++++-
 3 files changed, 11 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5de8b14c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c291689..1fc0bf6 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -188,6 +188,8 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4297. TestJobHistoryEventHandler and TestRMContainerAllocator failing
     on YARN-2928 branch (Varun Saxena via sjlee)
 
+    YARN-4460. [Bug fix] RM fails to start when SMP is enabled. (Li Lu via Varun Saxena)
+
 Trunk - Unreleased
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5de8b14c/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 b96114e..1954783 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
@@ -25,6 +25,7 @@ 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.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -75,13 +76,13 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
   }
 
   @Override
-  protected void serviceStart() throws Exception {
-    super.serviceStart();
+  protected void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    getDispatcher().register(SystemMetricsEventType.class,
+        new TimelineV2EventHandler());
     publishContainerMetrics = getConfig().getBoolean(
         YarnConfiguration.RM_PUBLISH_CONTAINER_METRICS_ENABLED,
         YarnConfiguration.DEFAULT_RM_PUBLISH_CONTAINER_METRICS_ENABLED);
-    getDispatcher().register(SystemMetricsEventType.class,
-        new TimelineV2EventHandler());
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5de8b14c/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 baaa566..57258d5 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
@@ -156,7 +156,7 @@ public class TestSystemMetricsPublisherForV2 {
     try {
       Configuration conf = getTimelineV2Conf();
       conf.setBoolean(YarnConfiguration.RM_PUBLISH_CONTAINER_METRICS_ENABLED,
-          false);
+          YarnConfiguration.DEFAULT_RM_PUBLISH_CONTAINER_METRICS_ENABLED);
       metricsPublisher.init(conf);
       assertFalse(
           "Default configuration should not publish container Metrics from RM",
@@ -167,6 +167,9 @@ public class TestSystemMetricsPublisherForV2 {
       metricsPublisher = new TimelineServiceV2Publisher(mock(RMContext.class));
       conf = getTimelineV2Conf();
       metricsPublisher.init(conf);
+      assertTrue("Expected to have registered event handlers and set ready to "
+          + "publish events after init",
+          metricsPublisher.isPublishContainerMetrics());
       metricsPublisher.start();
       assertTrue("Expected to publish container Metrics from RM",
           metricsPublisher.isPublishContainerMetrics());


[13/50] [abbrv] hadoop git commit: YARN-3901. Populate flow run data in the flow_run & flow activity tables (Vrushali C via sjlee)

Posted by gt...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.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/flow/FlowActivityTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityTable.java
new file mode 100644
index 0000000..af8df99
--- /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/flow/FlowActivityTable.java
@@ -0,0 +1,107 @@
+/**
+ * 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.flow;
+
+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;
+
+/**
+ * The flow activity table has column family info
+ * Stores the daily activity record for flows
+ * Useful as a quick lookup of what flows were
+ * running on a given day
+ *
+ * Example flow activity table record:
+ *
+ * </pre>
+ * |-------------------------------------------|
+ * |  Row key   | Column Family                |
+ * |            | info                         |
+ * |-------------------------------------------|
+ * | clusterId! | r!runid1:version1            |
+ * | inv Top of |                              |
+ * | Day!       | r!runid2:version7            |
+ * | userName!  |                              |
+ * | flowId     |                              |
+ * |-------------------------------------------|
+ * </pre>
+ */
+public class FlowActivityTable extends BaseTable<FlowActivityTable> {
+  /** flow activity table prefix */
+  private static final String PREFIX =
+      YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowactivity";
+
+  /** config param name that specifies the flowactivity table name */
+  public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name";
+
+  /** default value for flowactivity table name */
+  public static final String DEFAULT_TABLE_NAME = "timelineservice.flowactivity";
+
+  private static final Log LOG = LogFactory.getLog(FlowActivityTable.class);
+
+  /** default max number of versions */
+  public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE;
+
+  public FlowActivityTable() {
+    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 FlowActivityTableDescp = new HTableDescriptor(table);
+    HColumnDescriptor infoCF =
+        new HColumnDescriptor(FlowActivityColumnFamily.INFO.getBytes());
+    infoCF.setBloomFilterType(BloomType.ROWCOL);
+    FlowActivityTableDescp.addFamily(infoCF);
+    infoCF.setMinVersions(1);
+    infoCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS);
+
+    // TODO: figure the split policy before running in production
+    admin.createTable(FlowActivityTableDescp);
+    LOG.info("Status of table creation for " + table.getNameAsString() + "="
+        + admin.tableExists(table));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumn.java
new file mode 100644
index 0000000..ad30add
--- /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/flow/FlowRunColumn.java
@@ -0,0 +1,161 @@
+/**
+ * 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.flow;
+
+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.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+
+/**
+ * Identifies fully qualified columns for the {@link FlowRunTable}.
+ */
+public enum FlowRunColumn implements Column<FlowRunTable> {
+
+  /**
+   * When the flow was started. This is the minimum of currently known
+   * application start times.
+   */
+  MIN_START_TIME(FlowRunColumnFamily.INFO, "min_start_time",
+      AggregationOperation.MIN),
+
+  /**
+   * When the flow ended. This is the maximum of currently known application end
+   * times.
+   */
+  MAX_END_TIME(FlowRunColumnFamily.INFO, "max_end_time",
+      AggregationOperation.MAX),
+
+  /**
+   * The version of the flow that this flow belongs to.
+   */
+  FLOW_VERSION(FlowRunColumnFamily.INFO, "flow_version", null);
+
+  private final ColumnHelper<FlowRunTable> column;
+  private final ColumnFamily<FlowRunTable> columnFamily;
+  private final String columnQualifier;
+  private final byte[] columnQualifierBytes;
+  private final AggregationOperation aggOp;
+
+  private FlowRunColumn(ColumnFamily<FlowRunTable> columnFamily,
+      String columnQualifier, AggregationOperation aggOp) {
+    this.columnFamily = columnFamily;
+    this.columnQualifier = columnQualifier;
+    this.aggOp = aggOp;
+    // Future-proof by ensuring the right column prefix hygiene.
+    this.columnQualifierBytes = Bytes.toBytes(Separator.SPACE
+        .encode(columnQualifier));
+    this.column = new ColumnHelper<FlowRunTable>(columnFamily);
+  }
+
+  /**
+   * @return the column name value
+   */
+  private String getColumnQualifier() {
+    return columnQualifier;
+  }
+
+  public byte[] getColumnQualifierBytes() {
+    return columnQualifierBytes.clone();
+  }
+
+  public AggregationOperation getAggregationOperation() {
+    return aggOp;
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.Column#store
+   * (byte[], org.apache.hadoop.yarn.server.timelineservice.storage.common.
+   * TypedBufferedMutator, java.lang.Long, java.lang.Object,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute[])
+   */
+  public void store(byte[] rowKey,
+      TypedBufferedMutator<FlowRunTable> tableMutator, Long timestamp,
+      Object inputValue, Attribute... attributes) throws IOException {
+
+    Attribute[] combinedAttributes = TimelineWriterUtils.combineAttributes(
+        attributes, aggOp);
+    column.store(rowKey, tableMutator, columnQualifierBytes, timestamp,
+        inputValue, combinedAttributes);
+  }
+
+  public Object readResult(Result result) throws IOException {
+    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;
+  }
+
+  /**
+   * 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/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.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/flow/FlowRunColumnFamily.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnFamily.java
new file mode 100644
index 0000000..8faf5f8
--- /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/flow/FlowRunColumnFamily.java
@@ -0,0 +1,54 @@
+/**
+ * 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.flow;
+
+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 flow run table column families.
+ */
+public enum FlowRunColumnFamily implements ColumnFamily<FlowRunTable> {
+
+  /**
+   * Info column family houses known columns, specifically ones included in
+   * columnfamily filters.
+   */
+  INFO("i");
+
+  /**
+   * 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.
+   */
+  private FlowRunColumnFamily(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/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java
new file mode 100644
index 0000000..d55f510
--- /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/flow/FlowRunColumnPrefix.java
@@ -0,0 +1,239 @@
+/**
+ * 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.flow;
+
+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.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+
+/**
+ * Identifies partially qualified columns for the {@link FlowRunTable}.
+ */
+public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
+
+  /**
+   * To store flow run info values.
+   */
+  METRIC(FlowRunColumnFamily.INFO, "m", AggregationOperation.SUM);
+
+  private final ColumnHelper<FlowRunTable> column;
+  private final ColumnFamily<FlowRunTable> 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 final AggregationOperation aggOp;
+
+  /**
+   * Private constructor, meant to be used by the enum definition.
+   *
+   * @param columnFamily
+   *          that this column is stored in.
+   * @param columnPrefix
+   *          for this column.
+   */
+  private FlowRunColumnPrefix(ColumnFamily<FlowRunTable> columnFamily,
+      String columnPrefix, AggregationOperation fra) {
+    column = new ColumnHelper<FlowRunTable>(columnFamily);
+    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.aggOp = fra;
+  }
+
+  /**
+   * @return the column name value
+   */
+  public String getColumnPrefix() {
+    return columnPrefix;
+  }
+
+  public byte[] getColumnPrefixBytes() {
+    return columnPrefixBytes.clone();
+  }
+
+  public AggregationOperation getAttribute() {
+    return aggOp;
+  }
+
+  /*
+   * (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<FlowRunTable> 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 = ColumnHelper.getColumnQualifier(
+        this.columnPrefixBytes, qualifier);
+    Attribute[] combinedAttributes = TimelineWriterUtils.combineAttributes(
+        attributes, this.aggOp);
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
+        combinedAttributes);
+  }
+
+  /*
+   * (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<FlowRunTable> 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 = ColumnHelper.getColumnQualifier(
+        this.columnPrefixBytes, qualifier);
+    Attribute[] combinedAttributes = TimelineWriterUtils.combineAttributes(
+        attributes, this.aggOp);
+    column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
+        combinedAttributes);
+  }
+
+  /*
+   * (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)
+   */
+  public Map<String, Object> readResults(Result result) throws IOException {
+    return column.readResults(result, columnPrefixBytes);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
+   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result)
+   */
+  public <T> NavigableMap<String, NavigableMap<Long, T>> readResultsWithTimestamps(
+      Result result) throws IOException {
+    return column.readResultsWithTimestamps(result, columnPrefixBytes);
+  }
+
+  /**
+   * 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;
+  }
+
+  /**
+   * 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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.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/flow/FlowRunCoprocessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunCoprocessor.java
new file mode 100644
index 0000000..f743e5e
--- /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/flow/FlowRunCoprocessor.java
@@ -0,0 +1,210 @@
+/**
+ * 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.flow;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator;
+
+public class FlowRunCoprocessor extends BaseRegionObserver {
+
+  @SuppressWarnings("unused")
+  private static final Log LOG = LogFactory.getLog(FlowRunCoprocessor.class);
+
+  private HRegion region;
+  /**
+   * generate a timestamp that is unique per row in a region this is per region
+   */
+  private final TimestampGenerator timestampGenerator = new TimestampGenerator();
+
+  @Override
+  public void start(CoprocessorEnvironment e) throws IOException {
+    if (e instanceof RegionCoprocessorEnvironment) {
+      RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) e;
+      this.region = env.getRegion();
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * This method adds the tags onto the cells in the Put. It is presumed that
+   * all the cells in one Put have the same set of Tags. The existing cell
+   * timestamp is overwritten for non-metric cells and each such cell gets a new
+   * unique timestamp generated by {@link TimestampGenerator}
+   *
+   * @see
+   * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#prePut(org.apache
+   * .hadoop.hbase.coprocessor.ObserverContext,
+   * org.apache.hadoop.hbase.client.Put,
+   * org.apache.hadoop.hbase.regionserver.wal.WALEdit,
+   * org.apache.hadoop.hbase.client.Durability)
+   */
+  @Override
+  public void prePut(ObserverContext<RegionCoprocessorEnvironment> e, Put put,
+      WALEdit edit, Durability durability) throws IOException {
+    Map<String, byte[]> attributes = put.getAttributesMap();
+
+    // Assumption is that all the cells in a put are the same operation.
+    List<Tag> tags = new ArrayList<>();
+    if ((attributes != null) && (attributes.size() > 0)) {
+      for (Map.Entry<String, byte[]> attribute : attributes.entrySet()) {
+        Tag t = TimelineWriterUtils.getTagFromAttribute(attribute);
+        tags.add(t);
+      }
+      byte[] tagByteArray = Tag.fromList(tags);
+      NavigableMap<byte[], List<Cell>> newFamilyMap = new TreeMap<>(
+          Bytes.BYTES_COMPARATOR);
+      for (Map.Entry<byte[], List<Cell>> entry : put.getFamilyCellMap()
+          .entrySet()) {
+        List<Cell> newCells = new ArrayList<>(entry.getValue().size());
+        for (Cell cell : entry.getValue()) {
+          // for each cell in the put add the tags
+          // Assumption is that all the cells in
+          // one put are the same operation
+          // also, get a unique cell timestamp for non-metric cells
+          // this way we don't inadvertently overwrite cell versions
+          long cellTimestamp = getCellTimestamp(cell.getTimestamp(), tags);
+          newCells.add(CellUtil.createCell(CellUtil.cloneRow(cell),
+              CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell),
+              cellTimestamp, KeyValue.Type.Put, CellUtil.cloneValue(cell),
+              tagByteArray));
+        }
+        newFamilyMap.put(entry.getKey(), newCells);
+      } // for each entry
+      // Update the family map for the Put
+      put.setFamilyCellMap(newFamilyMap);
+    }
+  }
+
+  /**
+   * Determines if the current cell's timestamp is to be used or a new unique
+   * cell timestamp is to be used. The reason this is done is to inadvertently
+   * overwrite cells when writes come in very fast. But for metric cells, the
+   * cell timestamp signifies the metric timestamp. Hence we don't want to
+   * overwrite it.
+   *
+   * @param timestamp
+   * @param tags
+   * @return cell timestamp
+   */
+  private long getCellTimestamp(long timestamp, List<Tag> tags) {
+    // if ts not set (hbase sets to HConstants.LATEST_TIMESTAMP by default)
+    // then use the generator
+    if (timestamp == HConstants.LATEST_TIMESTAMP) {
+      return timestampGenerator.getUniqueTimestamp();
+    } else {
+      return timestamp;
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * Creates a {@link FlowScanner} Scan so that it can correctly process the
+   * contents of {@link FlowRunTable}.
+   *
+   * @see
+   * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#preGetOp(org.apache
+   * .hadoop.hbase.coprocessor.ObserverContext,
+   * org.apache.hadoop.hbase.client.Get, java.util.List)
+   */
+  @Override
+  public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e,
+      Get get, List<Cell> results) throws IOException {
+    Scan scan = new Scan(get);
+    scan.setMaxVersions();
+    RegionScanner scanner = null;
+    try {
+      scanner = new FlowScanner(region, scan.getBatch(),
+          region.getScanner(scan));
+      scanner.next(results);
+      e.bypass();
+    } finally {
+      if (scanner != null) {
+        scanner.close();
+      }
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * Ensures that max versions are set for the Scan so that metrics can be
+   * correctly aggregated and min/max can be correctly determined.
+   *
+   * @see
+   * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#preScannerOpen(org
+   * .apache.hadoop.hbase.coprocessor.ObserverContext,
+   * org.apache.hadoop.hbase.client.Scan,
+   * org.apache.hadoop.hbase.regionserver.RegionScanner)
+   */
+  @Override
+  public RegionScanner preScannerOpen(
+      ObserverContext<RegionCoprocessorEnvironment> e, Scan scan,
+      RegionScanner s) throws IOException {
+    // set max versions for scan to see all
+    // versions to aggregate for metrics
+    scan.setMaxVersions();
+    return s;
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * Creates a {@link FlowScanner} Scan so that it can correctly process the
+   * contents of {@link FlowRunTable}.
+   *
+   * @see
+   * org.apache.hadoop.hbase.coprocessor.BaseRegionObserver#postScannerOpen(
+   * org.apache.hadoop.hbase.coprocessor.ObserverContext,
+   * org.apache.hadoop.hbase.client.Scan,
+   * org.apache.hadoop.hbase.regionserver.RegionScanner)
+   */
+  @Override
+  public RegionScanner postScannerOpen(
+      ObserverContext<RegionCoprocessorEnvironment> e, Scan scan,
+      RegionScanner scanner) throws IOException {
+    return new FlowScanner(region, scan.getBatch(), scanner);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKey.java
new file mode 100644
index 0000000..e133241
--- /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/flow/FlowRunRowKey.java
@@ -0,0 +1,50 @@
+/**
+ * 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.flow;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+
+/**
+ * Represents a rowkey for the flow run table.
+ */
+public class FlowRunRowKey {
+  // TODO: more methods are needed for this class like parse row key
+
+  /**
+   * Constructs a row key for the entity table as follows: {
+   * clusterId!userI!flowId!Inverted Flow Run Id}
+   *
+   * @param clusterId
+   * @param userId
+   * @param flowId
+   * @param flowRunId
+   * @return byte array with the row key
+   */
+  public static byte[] getRowKey(String clusterId, String userId,
+      String flowId, Long flowRunId) {
+    byte[] first = Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId,
+        userId, flowId));
+    // Note that flowRunId is a long, so we can't encode them all at the same
+    // time.
+    byte[] second = Bytes.toBytes(TimelineWriterUtils.invert(flowRunId));
+    return Separator.QUALIFIERS.join(first, second);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.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/flow/FlowRunTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunTable.java
new file mode 100644
index 0000000..b1b93c1
--- /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/flow/FlowRunTable.java
@@ -0,0 +1,141 @@
+/**
+ * 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.flow;
+
+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;
+
+/**
+ * The flow run table has column family info
+ * Stores per flow run information
+ * aggregated across applications.
+ *
+ * Metrics are also stored in the info column family.
+ *
+ * Example flow run table record:
+ *
+ * <pre>
+ * flow_run table
+ * |-------------------------------------------|
+ * |  Row key   | Column Family                |
+ * |            | info                         |
+ * |-------------------------------------------|
+ * | clusterId! | flow_version:version7        |
+ * | userName!  |                              |
+ * | flowId!    | running_apps:1               |
+ * | flowRunId  |                              |
+ * |            | min_start_time:1392995080000 |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            | min_start_time:1392995081012 |
+ * |            | #0:appId2                    |
+ * |            |                              |
+ * |            | min_start_time:1392993083210 |
+ * |            | #0:appId3                    |
+ * |            |                              |
+ * |            |                              |
+ * |            | max_end_time:1392993084018   |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            |                              |
+ * |            | m!mapInputRecords:127        |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            | m!mapInputRecords:31         |
+ * |            | #2:appId2                    |
+ * |            |                              |
+ * |            | m!mapInputRecords:37         |
+ * |            | #1:appId3                    |
+ * |            |                              |
+ * |            |                              |
+ * |            | m!mapOutputRecords:181       |
+ * |            | #0:""                        |
+ * |            |                              |
+ * |            | m!mapOutputRecords:37        |
+ * |            | #1:appId3                    |
+ * |            |                              |
+ * |            |                              |
+ * |-------------------------------------------|
+ * </pre>
+ */
+public class FlowRunTable extends BaseTable<FlowRunTable> {
+  /** entity prefix */
+  private static final String PREFIX =
+      YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".flowrun";
+
+  /** config param name that specifies the flowrun table name */
+  public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name";
+
+  /** default value for flowrun table name */
+  public static final String DEFAULT_TABLE_NAME = "timelineservice.flowrun";
+
+  private static final Log LOG = LogFactory.getLog(FlowRunTable.class);
+
+  /** default max number of versions */
+  public static final int DEFAULT_METRICS_MAX_VERSIONS = Integer.MAX_VALUE;
+
+  public FlowRunTable() {
+    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 flowRunTableDescp = new HTableDescriptor(table);
+    HColumnDescriptor infoCF =
+        new HColumnDescriptor(FlowRunColumnFamily.INFO.getBytes());
+    infoCF.setBloomFilterType(BloomType.ROWCOL);
+    flowRunTableDescp.addFamily(infoCF);
+    infoCF.setMinVersions(1);
+    infoCF.setMaxVersions(DEFAULT_METRICS_MAX_VERSIONS);
+
+    // TODO: figure the split policy
+    flowRunTableDescp.addCoprocessor(FlowRunCoprocessor.class
+        .getCanonicalName());
+    admin.createTable(flowRunTableDescp);
+    LOG.info("Status of table creation for " + table.getNameAsString() + "="
+        + admin.tableExists(table));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.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/flow/FlowScanner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowScanner.java
new file mode 100644
index 0000000..a1948aa
--- /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/flow/FlowScanner.java
@@ -0,0 +1,486 @@
+/**
+ * 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.flow;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
+import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+/**
+ * Invoked via the coprocessor when a Get or a Scan is issued for flow run
+ * table. Looks through the list of cells per row, checks their tags and does
+ * operation on those cells as per the cell tags. Transforms reads of the stored
+ * metrics into calculated sums for each column Also, finds the min and max for
+ * start and end times in a flow run.
+ */
+class FlowScanner implements RegionScanner, Closeable {
+
+  private static final Log LOG = LogFactory.getLog(FlowScanner.class);
+
+  private final HRegion region;
+  private final InternalScanner flowRunScanner;
+  private RegionScanner regionScanner;
+  private final int limit;
+  private boolean hasMore;
+  private byte[] currentRow;
+  private List<Cell> availableCells = new ArrayList<>();
+  private int currentIndex;
+
+  FlowScanner(HRegion region, int limit, InternalScanner internalScanner) {
+    this.region = region;
+    this.limit = limit;
+    this.flowRunScanner = internalScanner;
+    if (internalScanner instanceof RegionScanner) {
+      this.regionScanner = (RegionScanner) internalScanner;
+    }
+    // TODO: note if it's compaction/flush
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see org.apache.hadoop.hbase.regionserver.RegionScanner#getRegionInfo()
+   */
+  @Override
+  public HRegionInfo getRegionInfo() {
+    return region.getRegionInfo();
+  }
+
+  @Override
+  public boolean nextRaw(List<Cell> cells) throws IOException {
+    return nextRaw(cells, limit);
+  }
+
+  @Override
+  public boolean nextRaw(List<Cell> cells, int limit) throws IOException {
+    return nextInternal(cells, limit);
+  }
+
+  @Override
+  public boolean next(List<Cell> cells) throws IOException {
+    return next(cells, limit);
+  }
+
+  @Override
+  public boolean next(List<Cell> cells, int limit) throws IOException {
+    return nextInternal(cells, limit);
+  }
+
+  private String getAggregationCompactionDimension(List<Tag> tags) {
+    String appId = null;
+    for (Tag t : tags) {
+      if (AggregationCompactionDimension.APPLICATION_ID.getTagType() == t
+          .getType()) {
+        appId = Bytes.toString(t.getValue());
+      }
+    }
+    return appId;
+  }
+
+  /**
+   * This method loops through the cells in a given row of the
+   * {@link FlowRunTable}. It looks at the tags of each cell to figure out how
+   * to process the contents. It then calculates the sum or min or max for each
+   * column or returns the cell as is.
+   *
+   * @param cells
+   * @param limit
+   * @return true if next row is available for the scanner, false otherwise
+   * @throws IOException
+   */
+  private boolean nextInternal(List<Cell> cells, int limit) throws IOException {
+    Cell cell = null;
+    startNext();
+    // Loop through all the cells in this row
+    // For min/max/metrics we do need to scan the entire set of cells to get the
+    // right one
+    // But with flush/compaction, the number of cells being scanned will go down
+    // cells are grouped per column qualifier then sorted by cell timestamp
+    // (latest to oldest) per column qualifier
+    // So all cells in one qualifier come one after the other before we see the
+    // next column qualifier
+    ByteArrayComparator comp = new ByteArrayComparator();
+    byte[] currentColumnQualifier = TimelineWriterUtils.EMPTY_BYTES;
+    AggregationOperation currentAggOp = null;
+    SortedSet<Cell> currentColumnCells = new TreeSet<>(KeyValue.COMPARATOR);
+    Set<String> alreadySeenAggDim = new HashSet<>();
+    int addedCnt = 0;
+    while (((cell = peekAtNextCell(limit)) != null)
+        && (limit <= 0 || addedCnt < limit)) {
+      byte[] newColumnQualifier = CellUtil.cloneQualifier(cell);
+      if (comp.compare(currentColumnQualifier, newColumnQualifier) != 0) {
+        addedCnt += emitCells(cells, currentColumnCells, currentAggOp);
+        resetState(currentColumnCells, alreadySeenAggDim);
+        currentColumnQualifier = newColumnQualifier;
+        currentAggOp = getCurrentAggOp(cell);
+      }
+      collectCells(currentColumnCells, currentAggOp, cell, alreadySeenAggDim);
+      nextCell(limit);
+    }
+    if (!currentColumnCells.isEmpty()) {
+      emitCells(cells, currentColumnCells, currentAggOp);
+    }
+    return hasMore();
+  }
+
+  private AggregationOperation getCurrentAggOp(Cell cell) {
+    List<Tag> tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
+        cell.getTagsLength());
+    // We assume that all the operations for a particular column are the same
+    return TimelineWriterUtils.getAggregationOperationFromTagsList(tags);
+  }
+
+  /**
+   * resets the parameters to an intialized state for next loop iteration
+   *
+   * @param cell
+   * @param currentAggOp
+   * @param currentColumnCells
+   * @param alreadySeenAggDim
+   * @param collectedButNotEmitted
+   */
+  private void resetState(SortedSet<Cell> currentColumnCells,
+      Set<String> alreadySeenAggDim) {
+    currentColumnCells.clear();
+    alreadySeenAggDim.clear();
+  }
+
+  private void collectCells(SortedSet<Cell> currentColumnCells,
+      AggregationOperation currentAggOp, Cell cell,
+      Set<String> alreadySeenAggDim) throws IOException {
+    if (currentAggOp == null) {
+      // not a min/max/metric cell, so just return it as is
+      currentColumnCells.add(cell);
+      nextCell(limit);
+      return;
+    }
+
+    switch (currentAggOp) {
+    case MIN:
+      if (currentColumnCells.size() == 0) {
+        currentColumnCells.add(cell);
+      } else {
+        Cell currentMinCell = currentColumnCells.first();
+        Cell newMinCell = compareCellValues(currentMinCell, cell, currentAggOp);
+        if (!currentMinCell.equals(newMinCell)) {
+          currentColumnCells.remove(currentMinCell);
+          currentColumnCells.add(newMinCell);
+        }
+      }
+      break;
+    case MAX:
+      if (currentColumnCells.size() == 0) {
+        currentColumnCells.add(cell);
+      } else {
+        Cell currentMaxCell = currentColumnCells.first();
+        Cell newMaxCell = compareCellValues(currentMaxCell, cell, currentAggOp);
+        if (!currentMaxCell.equals(newMaxCell)) {
+          currentColumnCells.remove(currentMaxCell);
+          currentColumnCells.add(newMaxCell);
+        }
+      }
+      break;
+    case SUM:
+    case SUM_FINAL:
+      // only if this app has not been seen yet, add to current column cells
+      List<Tag> tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
+          cell.getTagsLength());
+      String aggDim = getAggregationCompactionDimension(tags);
+      if (alreadySeenAggDim.contains(aggDim)) {
+        // if this agg dimension has already been seen,
+        // since they show up in sorted order
+        // we drop the rest which are older
+        // in other words, this cell is older than previously seen cells
+        // for that agg dim
+      } else {
+        // not seen this agg dim, hence consider this cell in our working set
+        currentColumnCells.add(cell);
+        alreadySeenAggDim.add(aggDim);
+      }
+      break;
+    default:
+      break;
+    } // end of switch case
+  }
+
+  /*
+   * Processes the cells in input param currentColumnCells and populates
+   * List<Cell> cells as the output based on the input AggregationOperation
+   * parameter.
+   */
+  private int emitCells(List<Cell> cells, SortedSet<Cell> currentColumnCells,
+      AggregationOperation currentAggOp) throws IOException {
+    if ((currentColumnCells == null) || (currentColumnCells.size() == 0)) {
+      return 0;
+    }
+    if (currentAggOp == null) {
+      cells.addAll(currentColumnCells);
+      return currentColumnCells.size();
+    }
+
+    switch (currentAggOp) {
+    case MIN:
+    case MAX:
+      cells.addAll(currentColumnCells);
+      return currentColumnCells.size();
+    case SUM:
+    case SUM_FINAL:
+      Cell sumCell = processSummation(currentColumnCells);
+      cells.add(sumCell);
+      return 1;
+    default:
+      cells.addAll(currentColumnCells);
+      return currentColumnCells.size();
+    }
+  }
+
+  /*
+   * Returns a cell whose value is the sum of all cell values in the input set.
+   * The new cell created has the timestamp of the most recent metric cell. The
+   * sum of a metric for a flow run is the summation at the point of the last
+   * metric update in that flow till that time.
+   */
+  private Cell processSummation(SortedSet<Cell> currentColumnCells)
+      throws IOException {
+    Number sum = 0;
+    Number currentValue = 0;
+    long ts = 0L;
+    long mostCurrentTimestamp = 0l;
+    Cell mostRecentCell = null;
+    for (Cell cell : currentColumnCells) {
+      currentValue = (Number) GenericObjectMapper.read(CellUtil
+          .cloneValue(cell));
+      ts = cell.getTimestamp();
+      if (mostCurrentTimestamp < ts) {
+        mostCurrentTimestamp = ts;
+        mostRecentCell = cell;
+      }
+      sum = sum.longValue() + currentValue.longValue();
+    }
+    Cell sumCell = createNewCell(mostRecentCell, sum);
+    return sumCell;
+  }
+
+  /**
+   * Determines which cell is to be returned based on the values in each cell
+   * and the comparison operation MIN or MAX.
+   *
+   * @param previouslyChosenCell
+   * @param currentCell
+   * @param currentAggOp
+   * @return the cell which is the min (or max) cell
+   * @throws IOException
+   */
+  private Cell compareCellValues(Cell previouslyChosenCell, Cell currentCell,
+      AggregationOperation currentAggOp) throws IOException {
+    if (previouslyChosenCell == null) {
+      return currentCell;
+    }
+    try {
+      long previouslyChosenCellValue = ((Number) GenericObjectMapper
+          .read(CellUtil.cloneValue(previouslyChosenCell))).longValue();
+      long currentCellValue = ((Number) GenericObjectMapper.read(CellUtil
+          .cloneValue(currentCell))).longValue();
+      switch (currentAggOp) {
+      case MIN:
+        if (currentCellValue < previouslyChosenCellValue) {
+          // new value is minimum, hence return this cell
+          return currentCell;
+        } else {
+          // previously chosen value is miniumum, hence return previous min cell
+          return previouslyChosenCell;
+        }
+      case MAX:
+        if (currentCellValue > previouslyChosenCellValue) {
+          // new value is max, hence return this cell
+          return currentCell;
+        } else {
+          // previously chosen value is max, hence return previous max cell
+          return previouslyChosenCell;
+        }
+      default:
+        return currentCell;
+      }
+    } catch (IllegalArgumentException iae) {
+      LOG.error("caught iae during conversion to long ", iae);
+      return currentCell;
+    }
+  }
+
+  private Cell createNewCell(Cell origCell, Number number) throws IOException {
+    byte[] newValue = GenericObjectMapper.write(number);
+    return CellUtil.createCell(CellUtil.cloneRow(origCell),
+        CellUtil.cloneFamily(origCell), CellUtil.cloneQualifier(origCell),
+        origCell.getTimestamp(), KeyValue.Type.Put.getCode(), newValue);
+  }
+
+  @Override
+  public void close() throws IOException {
+    flowRunScanner.close();
+  }
+
+  /**
+   * Called to signal the start of the next() call by the scanner.
+   */
+  public void startNext() {
+    currentRow = null;
+  }
+
+  /**
+   * Returns whether or not the underlying scanner has more rows.
+   */
+  public boolean hasMore() {
+    return currentIndex < availableCells.size() ? true : hasMore;
+  }
+
+  /**
+   * Returns the next available cell for the current row and advances the
+   * pointer to the next cell. This method can be called multiple times in a row
+   * to advance through all the available cells.
+   *
+   * @param limit
+   *          the limit of number of cells to return if the next batch must be
+   *          fetched by the wrapped scanner
+   * @return the next available cell or null if no more cells are available for
+   *         the current row
+   * @throws IOException
+   */
+  public Cell nextCell(int limit) throws IOException {
+    Cell cell = peekAtNextCell(limit);
+    if (cell != null) {
+      currentIndex++;
+    }
+    return cell;
+  }
+
+  /**
+   * Returns the next available cell for the current row, without advancing the
+   * pointer. Calling this method multiple times in a row will continue to
+   * return the same cell.
+   *
+   * @param limit
+   *          the limit of number of cells to return if the next batch must be
+   *          fetched by the wrapped scanner
+   * @return the next available cell or null if no more cells are available for
+   *         the current row
+   * @throws IOException
+   */
+  public Cell peekAtNextCell(int limit) throws IOException {
+    if (currentIndex >= availableCells.size()) {
+      // done with current batch
+      availableCells.clear();
+      currentIndex = 0;
+      hasMore = flowRunScanner.next(availableCells, limit);
+    }
+    Cell cell = null;
+    if (currentIndex < availableCells.size()) {
+      cell = availableCells.get(currentIndex);
+      if (currentRow == null) {
+        currentRow = CellUtil.cloneRow(cell);
+      } else if (!CellUtil.matchingRow(cell, currentRow)) {
+        // moved on to the next row
+        // don't use the current cell
+        // also signal no more cells for this row
+        return null;
+      }
+    }
+    return cell;
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see org.apache.hadoop.hbase.regionserver.RegionScanner#getMaxResultSize()
+   */
+  @Override
+  public long getMaxResultSize() {
+    if (regionScanner == null) {
+      throw new IllegalStateException(
+          "RegionScanner.isFilterDone() called when the flow "
+              + "scanner's scanner is not a RegionScanner");
+    }
+    return regionScanner.getMaxResultSize();
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see org.apache.hadoop.hbase.regionserver.RegionScanner#getMvccReadPoint()
+   */
+  @Override
+  public long getMvccReadPoint() {
+    if (regionScanner == null) {
+      throw new IllegalStateException(
+          "RegionScanner.isFilterDone() called when the flow "
+              + "scanner's internal scanner is not a RegionScanner");
+    }
+    return regionScanner.getMvccReadPoint();
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see org.apache.hadoop.hbase.regionserver.RegionScanner#isFilterDone()
+   */
+  @Override
+  public boolean isFilterDone() throws IOException {
+    if (regionScanner == null) {
+      throw new IllegalStateException(
+          "RegionScanner.isFilterDone() called when the flow "
+              + "scanner's internal scanner is not a RegionScanner");
+    }
+    return regionScanner.isFilterDone();
+
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see org.apache.hadoop.hbase.regionserver.RegionScanner#reseek(byte[])
+   */
+  @Override
+  public boolean reseek(byte[] bytes) throws IOException {
+    if (regionScanner == null) {
+      throw new IllegalStateException(
+          "RegionScanner.reseek() called when the flow "
+              + "scanner's internal scanner is not a RegionScanner");
+    }
+    return regionScanner.reseek(bytes);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.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/TestHBaseTimelineStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index 2875e01..3962341 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -53,7 +53,6 @@ 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.AppToFlowTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
@@ -88,20 +87,15 @@ public class TestHBaseTimelineStorage {
   }
 
   private static void createSchema() throws IOException {
-    new EntityTable()
-        .createTable(util.getHBaseAdmin(), util.getConfiguration());
-    new AppToFlowTable()
-        .createTable(util.getHBaseAdmin(), util.getConfiguration());
-    new ApplicationTable()
-        .createTable(util.getHBaseAdmin(), util.getConfiguration());
+    TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
   }
 
   @Test
   public void testWriteApplicationToHBase() throws Exception {
     TimelineEntities te = new TimelineEntities();
     ApplicationEntity entity = new ApplicationEntity();
-    String id = "hello";
-    entity.setId(id);
+    String appId = "application_1000178881110_2002";
+    entity.setId(appId);
     long cTime = 1425016501000L;
     long mTime = 1425026901000L;
     entity.setCreatedTime(cTime);
@@ -173,12 +167,12 @@ public class TestHBaseTimelineStorage {
       String flow = "some_flow_name";
       String flowVersion = "AB7822C10F1111";
       long runid = 1002345678919L;
-      hbi.write(cluster, user, flow, flowVersion, runid, id, te);
+      hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
       hbi.stop();
 
       // retrieve the row
       byte[] rowKey =
-          ApplicationRowKey.getRowKey(cluster, user, flow, runid, id);
+          ApplicationRowKey.getRowKey(cluster, user, flow, runid, appId);
       Get get = new Get(rowKey);
       get.setMaxVersions(Integer.MAX_VALUE);
       Connection conn = ConnectionFactory.createConnection(c1);
@@ -190,11 +184,11 @@ public class TestHBaseTimelineStorage {
       // check the row key
       byte[] row1 = result.getRow();
       assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
-          id));
+          appId));
 
       // check info column family
       String id1 = ApplicationColumn.ID.readResult(result).toString();
-      assertEquals(id, id1);
+      assertEquals(appId, id1);
 
       Number val =
           (Number) ApplicationColumn.CREATED_TIME.readResult(result);
@@ -252,17 +246,17 @@ public class TestHBaseTimelineStorage {
       assertEquals(metricValues, metricMap);
 
       // read the timeline entity using the reader this time
-      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, id,
+      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appId,
           entity.getType(), entity.getId(),
           EnumSet.of(TimelineReader.Field.ALL));
       Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
-          id, entity.getType(), null, null, null, null, null, null, null,
+          appId, entity.getType(), null, null, null, null, null, null, null,
           null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
       assertNotNull(e1);
       assertEquals(1, es1.size());
 
       // verify attributes
-      assertEquals(id, e1.getId());
+      assertEquals(appId, e1.getId());
       assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
           e1.getType());
       assertEquals(cTime, e1.getCreatedTime());
@@ -576,7 +570,7 @@ public class TestHBaseTimelineStorage {
       String flow = "other_flow_name";
       String flowVersion = "1111F01C2287BA";
       long runid = 1009876543218L;
-      String appName = "some app name";
+      String appName = "application_123465899910_1001";
       hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
       hbi.stop();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a469bfe7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.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/flow/TestFlowDataGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java
new file mode 100644
index 0000000..f8331fa
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java
@@ -0,0 +1,213 @@
+/**
+ * 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.flow;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+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.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;
+
+/**
+ * Generates the data/entities for the FlowRun and FlowActivity Tables
+ */
+class TestFlowDataGenerator {
+
+  private final static String metric1 = "MAP_SLOT_MILLIS";
+  private final static String metric2 = "HDFS_BYTES_READ";
+
+
+  static TimelineEntity getEntityMetricsApp1() {
+    TimelineEntity entity = new TimelineEntity();
+    String id = "flowRunMetrics_test";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    Long cTime = 1425016501000L;
+    entity.setCreatedTime(cTime);
+
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    TimelineMetric m1 = new TimelineMetric();
+    m1.setId(metric1);
+    Map<Long, Number> metricValues = new HashMap<Long, Number>();
+    long ts = System.currentTimeMillis();
+    metricValues.put(ts - 100000, 2);
+    metricValues.put(ts - 80000, 40);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+
+    TimelineMetric m2 = new TimelineMetric();
+    m2.setId(metric2);
+    metricValues = new HashMap<Long, Number>();
+    ts = System.currentTimeMillis();
+    metricValues.put(ts - 100000, 31);
+    metricValues.put(ts - 80000, 57);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues);
+    metrics.add(m2);
+
+    entity.addMetrics(metrics);
+    return entity;
+  }
+
+  static TimelineEntity getEntityMetricsApp2() {
+    TimelineEntity entity = new TimelineEntity();
+    String id = "flowRunMetrics_test";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    Long cTime = 1425016501000L;
+    entity.setCreatedTime(cTime);
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    TimelineMetric m1 = new TimelineMetric();
+    m1.setId(metric1);
+    Map<Long, Number> metricValues = new HashMap<Long, Number>();
+    long ts = System.currentTimeMillis();
+    metricValues.put(ts - 100000, 5L);
+    metricValues.put(ts - 80000, 101L);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    entity.addMetrics(metrics);
+    return entity;
+  }
+
+  static TimelineEntity getEntity1() {
+    TimelineEntity entity = new TimelineEntity();
+    String id = "flowRunHello";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    Long cTime = 20000000000000L;
+    Long mTime = 1425026901000L;
+    entity.setCreatedTime(cTime);
+    entity.setModifiedTime(mTime);
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    TimelineMetric m1 = new TimelineMetric();
+    m1.setId(metric1);
+    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);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    entity.addMetrics(metrics);
+
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    Long expTs = 1436512802000L;
+    event.setTimestamp(expTs);
+    String expKey = "foo_event";
+    Object expVal = "test";
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+
+    event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+    event.setTimestamp(1436512801000L);
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+
+    return entity;
+  }
+
+  static TimelineEntity getEntityGreaterStartTime() {
+    TimelineEntity entity = new TimelineEntity();
+    entity.setCreatedTime(30000000000000L);
+    entity.setId("flowRunHello with greater start time");
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setType(type);
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    long endTs = 1439379885000L;
+    event.setTimestamp(endTs);
+    String expKey = "foo_event_greater";
+    String expVal = "test_app_greater";
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+    return entity;
+  }
+
+  static TimelineEntity getEntityMaxEndTime(long endTs) {
+    TimelineEntity entity = new TimelineEntity();
+    entity.setId("flowRunHello Max End time");
+    entity.setType(TimelineEntityType.YARN_APPLICATION.toString());
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+    event.setTimestamp(endTs);
+    String expKey = "foo_even_max_ finished";
+    String expVal = "test_app_max_finished";
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+    return entity;
+  }
+
+  static TimelineEntity getEntityMinStartTime() {
+    TimelineEntity entity = new TimelineEntity();
+    String id = "flowRunHelloMInStartTime";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    Long cTime = 10000000000000L;
+    entity.setCreatedTime(cTime);
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    event.setTimestamp(System.currentTimeMillis());
+    entity.addEvent(event);
+    return entity;
+  }
+
+
+  static TimelineEntity getFlowApp1() {
+    TimelineEntity entity = new TimelineEntity();
+    String id = "flowActivity_test";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    Long cTime = 1425016501000L;
+    entity.setCreatedTime(cTime);
+
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    Long expTs = 1436512802000L;
+    event.setTimestamp(expTs);
+    String expKey = "foo_event";
+    Object expVal = "test";
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+
+    return entity;
+  }
+
+}


[20/50] [abbrv] hadoop git commit: YARN-4102. Add a "skip existing table" mode for timeline schema creator (Li Lu via sjlee)

Posted by gt...@apache.org.
YARN-4102. Add a "skip existing table" mode for timeline schema creator (Li Lu via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: 983e7291b1e3ebb1d6335306c2507bd8c64e99bf
Parents: c59a6ac
Author: Sangjin Lee <sj...@apache.org>
Authored: Fri Sep 11 09:46:13 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:57 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../storage/TimelineSchemaCreator.java          | 81 ++++++++++++++++----
 2 files changed, 69 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/983e7291/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 9c1ad80..667efd7 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -100,6 +100,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3814. REST API implementation for getting raw entities in
     TimelineReader (Varun Saxena via sjlee)
 
+    YARN-4102. Add a "skip existing table" mode for timeline schema creator (Li
+    Lu via sjlee)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/983e7291/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
index 5120856..e7e51a7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
@@ -54,6 +54,11 @@ public class TimelineSchemaCreator {
   final static String NAME = TimelineSchemaCreator.class.getSimpleName();
   private static final Log LOG = LogFactory.getLog(TimelineSchemaCreator.class);
   private static final String PHOENIX_OPTION_SHORT = "p";
+  private static final String SKIP_EXISTING_TABLE_OPTION_SHORT = "s";
+  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_TABLE_NAME_SHORT = "e";
 
   public static void main(String[] args) throws Exception {
 
@@ -66,22 +71,25 @@ public class TimelineSchemaCreator {
     CommandLine commandLine = parseArgs(otherArgs);
 
     // Grab the entityTableName argument
-    String entityTableName = commandLine.getOptionValue("e");
+    String entityTableName
+        = commandLine.getOptionValue(ENTITY_TABLE_NAME_SHORT);
     if (StringUtils.isNotBlank(entityTableName)) {
       hbaseConf.set(EntityTable.TABLE_NAME_CONF_NAME, entityTableName);
     }
-    String entityTableTTLMetrics = commandLine.getOptionValue("m");
+    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("a2f");
+    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("a");
+    String applicationTableName = commandLine.getOptionValue(
+        APP_TABLE_NAME_SHORT);
     if (StringUtils.isNotBlank(applicationTableName)) {
       hbaseConf.set(ApplicationTable.TABLE_NAME_CONF_NAME,
           applicationTableName);
@@ -89,7 +97,13 @@ public class TimelineSchemaCreator {
 
     List<Exception> exceptions = new ArrayList<>();
     try {
-      createAllTables(hbaseConf);
+      boolean skipExisting
+          = commandLine.hasOption(SKIP_EXISTING_TABLE_OPTION_SHORT);
+      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());
@@ -135,26 +149,39 @@ public class TimelineSchemaCreator {
     Options options = new Options();
 
     // Input
-    Option o = new Option("e", "entityTableName", true, "entity table name");
+    Option o = new Option(ENTITY_TABLE_NAME_SHORT, "entityTableName", true,
+        "entity table name");
     o.setArgName("entityTableName");
     o.setRequired(false);
     options.addOption(o);
 
-    o = new Option("m", "metricsTTL", true, "TTL for metrics column family");
+    o = new Option(TTL_OPTION_SHORT, "metricsTTL", true,
+        "TTL for metrics column family");
     o.setArgName("metricsTTL");
     o.setRequired(false);
     options.addOption(o);
 
-    o = new Option("a2f", "appToflowTableName", true, "app to flow table name");
+    o = new Option(APP_TO_FLOW_TABLE_NAME_SHORT, "appToflowTableName", true,
+        "app to flow table name");
     o.setArgName("appToflowTableName");
-    o = new Option("a", "applicationTableName", true, "application table name");
+    o.setRequired(false);
+    options.addOption(o);
+
+    o = new Option(APP_TABLE_NAME_SHORT, "applicationTableName", true,
+        "application table name");
     o.setArgName("applicationTableName");
     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(PHOENIX_OPTION_SHORT, "usePhoenix", false,
         "create Phoenix offline aggregation tables");
-    // No need to set arg name since we do not need an argument here
+    o.setRequired(false);
+    options.addOption(o);
+
+    o = new Option(SKIP_EXISTING_TABLE_OPTION_SHORT, "skipExistingTable",
+        false, "skip existing Hbase tables and continue to create new tables");
     o.setRequired(false);
     options.addOption(o);
 
@@ -172,8 +199,8 @@ public class TimelineSchemaCreator {
     return commandLine;
   }
 
-  private static void createAllTables(Configuration hbaseConf)
-      throws IOException {
+  private static void createAllTables(Configuration hbaseConf,
+      boolean skipExisting) throws IOException {
 
     Connection conn = null;
     try {
@@ -182,9 +209,33 @@ public class TimelineSchemaCreator {
       if (admin == null) {
         throw new IOException("Cannot create table since admin is null");
       }
-      new EntityTable().createTable(admin, hbaseConf);
-      new AppToFlowTable().createTable(admin, hbaseConf);
-      new ApplicationTable().createTable(admin, hbaseConf);
+      try {
+        new EntityTable().createTable(admin, hbaseConf);
+      } catch (IOException e) {
+        if (skipExisting) {
+          LOG.warn("Skip and continue on: " + e.getMessage());
+        } else {
+          throw e;
+        }
+      }
+      try {
+        new AppToFlowTable().createTable(admin, hbaseConf);
+      } catch (IOException e) {
+        if (skipExisting) {
+          LOG.warn("Skip and continue on: " + e.getMessage());
+        } else {
+          throw e;
+        }
+      }
+      try {
+        new ApplicationTable().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();


[38/50] [abbrv] hadoop git commit: YARN-4356. Ensure the timeline service v.2 is disabled cleanly and has no impact when it's turned off. Contributed by Sangjin Lee.

Posted by gt...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 2c5c300..69de433 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
@@ -38,7 +38,6 @@ 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.client.api.TimelineClient;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -56,12 +55,16 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.Contai
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 
+/**
+ * Metrics publisher service that publishes data to the timeline service v.2. It
+ * is used only if the timeline service v.2 is enabled and the system publishing
+ * of events and metrics is enabled.
+ */
 public class NMTimelinePublisher extends CompositeService {
 
   private static final Log LOG = LogFactory.getLog(NMTimelinePublisher.class);
 
   private Dispatcher dispatcher;
-  private boolean publishSystemMetrics;
 
   private Context context;
 
@@ -76,24 +79,16 @@ public class NMTimelinePublisher extends CompositeService {
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
-    publishSystemMetrics =
-        YarnConfiguration.systemMetricsPublisherEnabled(conf);
-
-    if (publishSystemMetrics) {
-      dispatcher = new AsyncDispatcher();
-      dispatcher.register(NMTimelineEventType.class,
-          new ForwardingEventHandler());
-      dispatcher
-          .register(ContainerEventType.class, new ContainerEventHandler());
-      dispatcher.register(ApplicationEventType.class,
-          new ApplicationEventHandler());
-      dispatcher.register(LocalizationEventType.class,
-          new LocalizationEventDispatcher());
-      addIfService(dispatcher);
-      LOG.info("YARN system metrics publishing service is enabled");
-    } else {
-      LOG.info("YARN system metrics publishing service is not enabled");
-    }
+    dispatcher = new AsyncDispatcher();
+    dispatcher.register(NMTimelineEventType.class,
+        new ForwardingEventHandler());
+    dispatcher
+        .register(ContainerEventType.class, new ContainerEventHandler());
+    dispatcher.register(ApplicationEventType.class,
+        new ApplicationEventHandler());
+    dispatcher.register(LocalizationEventType.class,
+        new LocalizationEventDispatcher());
+    addIfService(dispatcher);
     super.serviceInit(conf);
   }
 
@@ -121,8 +116,9 @@ public class NMTimelinePublisher extends CompositeService {
   public void reportContainerResourceUsage(Container container,
       long createdTime, String pId, Long pmemUsage,
       Float cpuUsageTotalCoresPercentage) {
-    if (publishSystemMetrics
-        && (pmemUsage != ResourceCalculatorProcessTree.UNAVAILABLE || cpuUsageTotalCoresPercentage != ResourceCalculatorProcessTree.UNAVAILABLE)) {
+    if (pmemUsage != ResourceCalculatorProcessTree.UNAVAILABLE ||
+        cpuUsageTotalCoresPercentage !=
+            ResourceCalculatorProcessTree.UNAVAILABLE) {
       ContainerEntity entity =
           createContainerEntity(container.getContainerId());
       long currentTimeMillis = System.currentTimeMillis();
@@ -219,9 +215,6 @@ public class NMTimelinePublisher extends CompositeService {
   }
 
   public void publishApplicationEvent(ApplicationEvent event) {
-    if (!publishSystemMetrics) {
-      return;
-    }
     // publish only when the desired event is received
     switch (event.getType()) {
     case INIT_APPLICATION:
@@ -242,9 +235,6 @@ public class NMTimelinePublisher extends CompositeService {
   }
 
   public void publishContainerEvent(ContainerEvent event) {
-    if (!publishSystemMetrics) {
-      return;
-    }
     // publish only when the desired event is received
     switch (event.getType()) {
     case INIT_CONTAINER:
@@ -262,9 +252,6 @@ public class NMTimelinePublisher extends CompositeService {
   }
 
   public void publishLocalizationEvent(LocalizationEvent event) {
-    if (!publishSystemMetrics) {
-      return;
-    }
     // publish only when the desired event is received
     switch (event.getType()) {
     case CONTAINER_RESOURCES_LOCALIZED:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 f29b791..c43777c 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
@@ -50,7 +50,6 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@@ -95,7 +94,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.LogHandler;
-import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMMemoryStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
 import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
@@ -106,7 +104,6 @@ import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.Mockito;
 
 public class TestContainerManagerRecovery extends BaseContainerManagerTest {
 
@@ -473,7 +470,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
       NMStateStoreService stateStore) {
     NMContext context = new NMContext(new NMContainerTokenSecretManager(
         conf), new NMTokenSecretManagerInNM(), null,
-        new ApplicationACLsManager(conf), stateStore, conf){
+        new ApplicationACLsManager(conf), stateStore, conf) {
       public int getHttpPort() {
         return HTTP_PORT;
       }
@@ -638,9 +635,9 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
           }
 
           @Override
-          public NMTimelinePublisher createNMTimelinePublisher(Context context) {
-            NMTimelinePublisher timelinePublisher = mock(NMTimelinePublisher.class);
-            return timelinePublisher;
+          public NMTimelinePublisher
+              createNMTimelinePublisher(Context context) {
+            return null;
           }
     };
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.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/application/TestApplication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java
index 38b3172f..f31a98c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/application/TestApplication.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
@@ -535,7 +534,7 @@ public class TestApplication {
       this.appId = BuilderUtils.newApplicationId(timestamp, id);
 
       app = new ApplicationImpl(
-          dispatcher, this.user, null, null, 0, appId, null, context);
+          dispatcher, this.user, appId, null, context);
       containers = new ArrayList<Container>();
       for (int i = 0; i < numContainers; i++) {
         Container container = createMockedContainer(this.appId, i);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.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/webapp/TestNMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
index ef5eb65..a6818ec 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServices.java
@@ -31,17 +31,14 @@ import javax.ws.rs.core.MediaType;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 
-import org.junit.Assert;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.NodeHealthScriptRunner;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
@@ -50,7 +47,6 @@ import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.ResourceView;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
 import org.apache.hadoop.yarn.server.nodemanager.webapp.WebServer.NMWebApp;
@@ -64,6 +60,7 @@ import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.w3c.dom.Document;
@@ -327,7 +324,7 @@ public class TestNMWebServices extends JerseyTestBase {
     final String filename = "logfile1";
     final String logMessage = "log message\n";
     nmContext.getApplications().put(appId, new ApplicationImpl(null, "user",
-        null, null, 0, appId, null, nmContext));
+        appId, null, nmContext));
     
     MockContainer container = new MockContainer(appAttemptId,
         new AsyncDispatcher(), new Configuration(), "user", appId, 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.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/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
index dd817d0..9ace1fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
@@ -293,8 +293,11 @@ public class ApplicationMasterService extends AbstractService implements
 
     RMApp rmApp =
         rmContext.getRMApps().get(applicationAttemptId.getApplicationId());
+
     // Remove collector address when app get finished.
-    rmApp.removeCollectorAddr();
+    if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
+      rmApp.removeCollectorAddr();
+    }
     // checking whether the app exits in RMStateStore at first not to throw
     // ApplicationDoesNotExistInCacheException before and after
     // RM work-preserving restart.
@@ -562,8 +565,10 @@ public class ApplicationMasterService extends AbstractService implements
       allocateResponse.setNumClusterNodes(this.rScheduler.getNumClusterNodes());
 
       // add collector address for this application
-      allocateResponse.setCollectorAddr(
-          this.rmContext.getRMApps().get(applicationId).getCollectorAddr());
+      if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
+        allocateResponse.setCollectorAddr(
+            this.rmContext.getRMApps().get(applicationId).getCollectorAddr());
+      }
 
       // add preemption to the allocateResponse message (if any)
       allocateResponse

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.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/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index 42d07a1..962d483 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -574,24 +574,27 @@ public class ClientRMService extends AbstractService implements
       throw RPCUtil.getRemoteException(ie);
     }
 
-    // Sanity check for flow run
-    String value = null;
-    try {
-      for (String tag : submissionContext.getApplicationTags()) {
-        if (tag.startsWith(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX + ":") ||
-            tag.startsWith(
-                TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.toLowerCase() + ":")) {
-          value = tag.substring(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.length() + 1);
-          Long.valueOf(value);
+    if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
+      // Sanity check for flow run
+      String value = null;
+      try {
+        for (String tag : submissionContext.getApplicationTags()) {
+          if (tag.startsWith(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX + ":") ||
+              tag.startsWith(
+                  TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.toLowerCase() + ":")) {
+            value = tag.substring(TimelineUtils.FLOW_RUN_ID_TAG_PREFIX.length()
+                + 1);
+            Long.valueOf(value);
+          }
         }
+      } catch (NumberFormatException e) {
+        LOG.warn("Invalid to flow run: " + value +
+            ". Flow run should be a long integer", e);
+        RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_APP_REQUEST,
+            e.getMessage(), "ClientRMService",
+            "Exception in submitting application", applicationId);
+        throw RPCUtil.getRemoteException(e);
       }
-    } catch (NumberFormatException e) {
-      LOG.warn("Invalid to flow run: " + value +
-          ". Flow run should be a long integer", e);
-      RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_APP_REQUEST,
-          e.getMessage(), "ClientRMService",
-          "Exception in submitting application", applicationId);
-      throw RPCUtil.getRemoteException(e);
     }
 
     // Check whether app has already been put into rmContext,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.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/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
index 36614f9..4aa7533 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
@@ -396,8 +396,11 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       LOG.warn(message);
       throw new YarnException(message);
     }
-    // Start timeline collector for the submitted app
-    application.startTimelineCollector();
+
+    if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      // Start timeline collector for the submitted app
+      application.startTimelineCollector();
+    }
     // Inform the ACLs Manager
     this.applicationACLsManager.addApplication(applicationId,
         submissionContext.getAMContainerSpec().getApplicationACLs());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 e939ba9..a627c4d 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
@@ -383,18 +383,19 @@ public class ResourceManager extends CompositeService implements Recoverable {
   }
 
   protected SystemMetricsPublisher createSystemMetricsPublisher() {
-    boolean timelineServiceEnabled =
-        conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED);
-    SystemMetricsPublisher publisher = null;
-    if (timelineServiceEnabled) {
-      if (conf.getBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
-          YarnConfiguration.DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_ENABLED)) {
-        LOG.info("TimelineService V1 is configured");
-        publisher = new TimelineServiceV1Publisher();
-      } else {
-        LOG.info("TimelineService V2 is configured");
+    SystemMetricsPublisher publisher;
+    if (YarnConfiguration.timelineServiceEnabled(conf) &&
+        YarnConfiguration.systemMetricsPublisherEnabled(conf)) {
+      if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+        // we're dealing with the v.2.x publisher
+        LOG.info("system metrics publisher with the timeline service V2 is " +
+            "configured");
         publisher = new TimelineServiceV2Publisher(rmContext);
+      } else {
+        // we're dealing with the v.1.x publisher
+        LOG.info("system metrics publisher with the timeline service V1 is " +
+            "configured");
+        publisher = new TimelineServiceV1Publisher();
       }
     } else {
       LOG.info("TimelineServicePublisher is not configured");
@@ -523,10 +524,12 @@ public class ResourceManager extends CompositeService implements Recoverable {
       addService(rmApplicationHistoryWriter);
       rmContext.setRMApplicationHistoryWriter(rmApplicationHistoryWriter);
 
-      RMTimelineCollectorManager timelineCollectorManager =
-          createRMTimelineCollectorManager();
-      addService(timelineCollectorManager);
-      rmContext.setRMTimelineCollectorManager(timelineCollectorManager);
+      if (YarnConfiguration.timelineServiceV2Enabled(configuration)) {
+        RMTimelineCollectorManager timelineCollectorManager =
+            createRMTimelineCollectorManager();
+        addService(timelineCollectorManager);
+        rmContext.setRMTimelineCollectorManager(timelineCollectorManager);
+      }
 
       // Register event handler for NodesListManager
       nodesListManager = new NodesListManager(rmContext);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 083c6bb..b386f0a 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
@@ -450,10 +450,15 @@ public class ResourceTrackerService extends AbstractService implements
           message);
     }
 
-    // Check & update collectors info from request.
-    // TODO make sure it won't have race condition issue for AM failed over case
-    // that the older registration could possible override the newer one.
-    updateAppCollectorsMap(request);
+    boolean timelineV2Enabled =
+        YarnConfiguration.timelineServiceV2Enabled(getConfig());
+    if (timelineV2Enabled) {
+      // Check & update collectors info from request.
+      // TODO make sure it won't have race condition issue for AM failed over
+      // case that the older registration could possible override the newer
+      // one.
+      updateAppCollectorsMap(request);
+    }
 
     // Heartbeat response
     NodeHeartbeatResponse nodeHeartBeatResponse = YarnServerBuilderUtils
@@ -472,12 +477,12 @@ public class ResourceTrackerService extends AbstractService implements
       nodeHeartBeatResponse.setSystemCredentialsForApps(systemCredentials);
     }
 
-    // Return collectors' map that NM needs to know
-    // TODO we should optimize this to only include collector info that NM
-    // doesn't know yet.
     List<ApplicationId> keepAliveApps =
         remoteNodeStatus.getKeepAliveApplications();
-    if (keepAliveApps != null) {
+    if (timelineV2Enabled && keepAliveApps != null) {
+      // Return collectors' map that NM needs to know
+      // TODO we should optimize this to only include collector info that NM
+      // doesn't know yet.
       setAppCollectorsMapToResponse(keepAliveApps, nodeHeartBeatResponse);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.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/amlauncher/AMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
index df760a3..9a7638c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.client.NMProxy;
+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.ipc.YarnRPC;
@@ -215,12 +216,14 @@ public class AMLauncher implements Runnable {
             .get(applicationId)
             .getSubmitTime()));
 
-    // Set flow context info
-    for (String tag :
-        rmContext.getRMApps().get(applicationId).getApplicationTags()) {
-      setFlowTags(environment, TimelineUtils.FLOW_NAME_TAG_PREFIX, tag);
-      setFlowTags(environment, TimelineUtils.FLOW_VERSION_TAG_PREFIX, tag);
-      setFlowTags(environment, TimelineUtils.FLOW_RUN_ID_TAG_PREFIX, tag);
+    if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      // Set flow context info
+      for (String tag :
+          rmContext.getRMApps().get(applicationId).getApplicationTags()) {
+        setFlowTags(environment, TimelineUtils.FLOW_NAME_TAG_PREFIX, tag);
+        setFlowTags(environment, TimelineUtils.FLOW_VERSION_TAG_PREFIX, tag);
+        setFlowTags(environment, TimelineUtils.FLOW_RUN_ID_TAG_PREFIX, tag);
+      }
     }
     Credentials credentials = new Credentials();
     DataInputByteBuffer dibb = new DataInputByteBuffer();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 0105495..b96114e 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
@@ -58,7 +58,7 @@ import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import com.google.common.annotations.VisibleForTesting;
 
 /**
- * This class is responsible for posting application, appattempt & Container
+ * This class is responsible for posting application, appattempt &amp; Container
  * lifecycle related events to timeline service V2
  */
 @Private

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.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/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index e04c499..51dd8b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -955,15 +955,17 @@ public class RMAppImpl implements RMApp, Recoverable {
       extends RMAppTransition {
 
     public void transition(RMAppImpl app, RMAppEvent event) {
-      LOG.info("Updating collector info for app: " + app.getApplicationId());
+      if (YarnConfiguration.timelineServiceV2Enabled(app.conf)) {
+        LOG.info("Updating collector info for app: " + app.getApplicationId());
 
-      RMAppCollectorUpdateEvent appCollectorUpdateEvent =
-          (RMAppCollectorUpdateEvent) event;
-      // Update collector address
-      app.setCollectorAddr(appCollectorUpdateEvent.getAppCollectorAddr());
+        RMAppCollectorUpdateEvent appCollectorUpdateEvent =
+            (RMAppCollectorUpdateEvent) event;
+        // Update collector address
+        app.setCollectorAddr(appCollectorUpdateEvent.getAppCollectorAddr());
 
-      // TODO persistent to RMStateStore for recover
-      // Save to RMStateStore
+        // TODO persistent to RMStateStore for recover
+        // Save to RMStateStore
+      }
     };
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.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/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
index 52f9dc0..0903b0c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
@@ -646,6 +646,7 @@ public class TestClientRMService {
     ClientRMService rmService =
         new ClientRMService(rmContext, yarnScheduler, appManager,
             mockAclsManager, mockQueueACLsManager, null);
+    rmService.init(new Configuration());
 
     // without name and queue
 
@@ -738,6 +739,7 @@ public class TestClientRMService {
     ClientRMService rmService =
         new ClientRMService(rmContext, yarnScheduler, appManager,
             mockAclsManager, mockQueueACLsManager, null);
+    rmService.init(new Configuration());
 
     // Initialize appnames and queues
     String[] queues = {QUEUE_1, QUEUE_2};
@@ -901,6 +903,7 @@ public class TestClientRMService {
     final ClientRMService rmService =
         new ClientRMService(rmContext, yarnScheduler, appManager, null, null,
             null);
+    rmService.init(new Configuration());
 
     // submit an app and wait for it to block while in app submission
     Thread t = new Thread() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.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/TestSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
index 2652acf..78435b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
@@ -75,7 +75,7 @@ public class TestSystemMetricsPublisher {
   public static void setup() throws Exception {
     YarnConfiguration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
-    conf.setBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true);
     conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STORE,
         MemoryTimelineStore.class, TimelineStore.class);
     conf.setClass(YarnConfiguration.TIMELINE_SERVICE_STATE_STORE_CLASS,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 20a5b13..baaa566 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
@@ -131,6 +131,7 @@ public class TestSystemMetricsPublisherForV2 {
   private static Configuration getTimelineV2Conf() {
     Configuration conf = new Configuration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
     conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true);
     conf.setInt(
         YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE, 2);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 5672759..a734340 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
@@ -20,41 +20,55 @@ package org.apache.hadoop.yarn.server.timelineservice;
 
 
 import static org.junit.Assert.fail;
-import static org.mockito.Mockito.any;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ExitUtil;
 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.api.records.timelineservice.*;
+import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationAttemptEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.ClusterEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.ContainerEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
+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.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.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
 import org.apache.hadoop.yarn.server.timelineservice.collector.NodeTimelineCollectorManager;
+import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.IOException;
-
 public class TestTimelineServiceClientIntegration {
   private static NodeTimelineCollectorManager collectorManager;
   private static PerNodeTimelineCollectorsAuxService auxService;
+  private static Configuration conf;
 
   @BeforeClass
   public static void setupClass() throws Exception {
     try {
       collectorManager = new MockNodeTimelineCollectorManager();
+      conf = new YarnConfiguration();
+      // enable timeline service v.2
+      conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+      conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
       auxService =
           PerNodeTimelineCollectorsAuxService.launchServer(new String[0],
-              collectorManager);
+              collectorManager, conf);
       auxService.addApplication(ApplicationId.newInstance(0, 1));
     } catch (ExitUtil.ExitException e) {
       fail();
@@ -76,7 +90,7 @@ public class TestTimelineServiceClientIntegration {
       // set the timeline service address manually
       client.setTimelineServiceAddress(
           collectorManager.getRestServerBindAddress());
-      client.init(new YarnConfiguration());
+      client.init(conf);
       client.start();
       TimelineEntity entity = new TimelineEntity();
       entity.setType("test entity type");
@@ -103,7 +117,7 @@ public class TestTimelineServiceClientIntegration {
       // set the timeline service address manually
       client.setTimelineServiceAddress(
           collectorManager.getRestServerBindAddress());
-      client.init(new YarnConfiguration());
+      client.init(conf);
       client.start();
       ClusterEntity cluster = new ClusterEntity();
       cluster.setId(YarnConfiguration.DEFAULT_RM_CLUSTER_ID);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 4147d42..0319e34 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
@@ -30,12 +30,11 @@ import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
 import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
 import org.apache.hadoop.yarn.server.api.AuxiliaryService;
-import org.apache.hadoop.yarn.server.api.ContainerContext;
 import org.apache.hadoop.yarn.server.api.ContainerInitializationContext;
 import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
 import org.apache.hadoop.yarn.server.api.ContainerType;
@@ -68,6 +67,9 @@ 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");
+    }
     collectorManager.init(conf);
     super.serviceInit(conf);
   }
@@ -175,7 +177,8 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
 
   @VisibleForTesting
   public static PerNodeTimelineCollectorsAuxService
-      launchServer(String[] args, NodeTimelineCollectorManager collectorManager) {
+      launchServer(String[] args, NodeTimelineCollectorManager collectorManager,
+      Configuration conf) {
     Thread
       .setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
     StringUtils.startupShutdownMessage(
@@ -187,7 +190,6 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
           new PerNodeTimelineCollectorsAuxService(collectorManager);
       ShutdownHookManager.get().addShutdownHook(new ShutdownHook(auxService),
           SHUTDOWN_HOOK_PRIORITY);
-      YarnConfiguration conf = new YarnConfiguration();
       auxService.init(conf);
       auxService.start();
     } catch (Throwable t) {
@@ -210,6 +212,9 @@ public class PerNodeTimelineCollectorsAuxService extends AuxiliaryService {
   }
 
   public static void main(String[] args) {
-    launchServer(args, null);
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+    launchServer(args, null, conf);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 afe1536..6b4213d 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
@@ -39,6 +39,7 @@ import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
 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.storage.FileSystemTimelineReaderImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
@@ -66,6 +67,10 @@ public class TimelineReaderServer extends CompositeService {
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
+    if (!YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      throw new YarnException("timeline service v.2 is not enabled");
+    }
+
     TimelineReader timelineReaderStore = createTimelineReaderStore(conf);
     addService(timelineReaderStore);
     timelineReaderManager = createTimelineReaderManager(timelineReaderStore);
@@ -143,7 +148,8 @@ public class TimelineReaderServer extends CompositeService {
     return readerWebServer.getConnectorAddress(0).getPort();
   }
 
-  static TimelineReaderServer startTimelineReaderServer(String[] args) {
+  static TimelineReaderServer startTimelineReaderServer(String[] args,
+      Configuration conf) {
     Thread.setDefaultUncaughtExceptionHandler(
         new YarnUncaughtExceptionHandler());
     StringUtils.startupShutdownMessage(TimelineReaderServer.class,
@@ -154,7 +160,6 @@ public class TimelineReaderServer extends CompositeService {
       ShutdownHookManager.get().addShutdownHook(
           new CompositeServiceShutdownHook(timelineReaderServer),
           SHUTDOWN_HOOK_PRIORITY);
-      YarnConfiguration conf = new YarnConfiguration();
       timelineReaderServer.init(conf);
       timelineReaderServer.start();
     } catch (Throwable t) {
@@ -165,6 +170,9 @@ public class TimelineReaderServer extends CompositeService {
   }
 
   public static void main(String[] args) {
-    startTimelineReaderServer(args);
+    Configuration conf = new YarnConfiguration();
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+    startTimelineReaderServer(args, conf);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.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/TestPerNodeTimelineCollectorsAuxService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
index dafc76e..4fdf47e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestPerNodeTimelineCollectorsAuxService.java
@@ -50,11 +50,16 @@ import java.io.IOException;
 public class TestPerNodeTimelineCollectorsAuxService {
   private ApplicationAttemptId appAttemptId;
   private PerNodeTimelineCollectorsAuxService auxService;
+  private Configuration conf;
 
   public TestPerNodeTimelineCollectorsAuxService() {
     ApplicationId appId =
         ApplicationId.newInstance(System.currentTimeMillis(), 1);
     appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+    conf = new YarnConfiguration();
+    // enable timeline service v.2
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
   }
 
   @After
@@ -134,7 +139,7 @@ public class TestPerNodeTimelineCollectorsAuxService {
     try {
       auxService =
           PerNodeTimelineCollectorsAuxService.launchServer(new String[0],
-              createCollectorManager());
+              createCollectorManager(), conf);
     } catch (ExitUtil.ExitException e) {
       assertEquals(0, e.status);
       ExitUtil.resetFirstExitException();
@@ -160,7 +165,7 @@ public class TestPerNodeTimelineCollectorsAuxService {
     NodeTimelineCollectorManager collectorManager = createCollectorManager();
     PerNodeTimelineCollectorsAuxService auxService =
         spy(new PerNodeTimelineCollectorsAuxService(collectorManager));
-    auxService.init(new YarnConfiguration());
+    auxService.init(conf);
     auxService.start();
     return auxService;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.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/TestTimelineReaderServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java
index 7098814..b42488c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderServer.java
@@ -30,8 +30,11 @@ public class TestTimelineReaderServer {
 
   @Test(timeout = 60000)
   public void testStartStopServer() throws Exception {
+    @SuppressWarnings("resource")
     TimelineReaderServer server = new TimelineReaderServer();
     Configuration config = new YarnConfiguration();
+    config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
     config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
         "localhost:0");
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 45bce2f..91f6ee5 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
@@ -72,6 +72,8 @@ public class TestTimelineReaderWebServices {
   public void init() throws Exception {
     try {
       Configuration config = new YarnConfiguration();
+      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");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
index 3b285aa..818cd89 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -238,6 +238,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
   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");


[39/50] [abbrv] hadoop git commit: YARN-4356. Ensure the timeline service v.2 is disabled cleanly and has no impact when it's turned off. Contributed by Sangjin Lee.

Posted by gt...@apache.org.
YARN-4356. Ensure the timeline service v.2 is disabled cleanly and has no
impact when it's turned off. 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/5591eebd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5591eebd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5591eebd

Branch: refs/heads/feature-YARN-2928
Commit: 5591eebd814a0c7202f00a2f62b2703195e5670d
Parents: cb46808
Author: Li Lu <gt...@apache.org>
Authored: Fri Dec 11 11:17:34 2015 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 18:02:43 2016 -0800

----------------------------------------------------------------------
 .../jobhistory/JobHistoryEventHandler.java      |  64 ++++---
 .../hadoop/mapreduce/v2/app/MRAppMaster.java    |  11 +-
 .../apache/hadoop/mapreduce/MRJobConfig.java    |   5 -
 .../src/main/resources/mapred-default.xml       |   7 -
 .../mapred/TestMRTimelineEventHandling.java     |   5 +-
 .../hadoop/mapreduce/v2/MiniMRYarnCluster.java  |   2 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |  58 +++++-
 .../distributedshell/ApplicationMaster.java     | 192 ++++++++-----------
 .../applications/distributedshell/Client.java   |  16 --
 .../distributedshell/TestDistributedShell.java  |  14 +-
 .../hadoop/yarn/client/api/TimelineClient.java  |  18 +-
 .../client/api/impl/TimelineClientImpl.java     |   3 +
 .../src/main/resources/yarn-default.xml         |   5 +-
 .../impl/pb/NodeHeartbeatRequestPBImpl.java     |  10 +-
 .../impl/pb/NodeHeartbeatResponsePBImpl.java    |  10 +-
 .../hadoop/yarn/server/nodemanager/Context.java |   3 +-
 .../yarn/server/nodemanager/NodeManager.java    |  23 ++-
 .../nodemanager/NodeStatusUpdaterImpl.java      |  48 +++--
 .../collectormanager/NMCollectorService.java    |  10 +-
 .../containermanager/ContainerManagerImpl.java  |  59 ++++--
 .../application/ApplicationImpl.java            |  70 +++++--
 .../monitor/ContainersMonitorImpl.java          |  11 +-
 .../timelineservice/NMTimelinePublisher.java    |  49 ++---
 .../TestContainerManagerRecovery.java           |  11 +-
 .../application/TestApplication.java            |   3 +-
 .../nodemanager/webapp/TestNMWebServices.java   |   9 +-
 .../ApplicationMasterService.java               |  11 +-
 .../server/resourcemanager/ClientRMService.java |  35 ++--
 .../server/resourcemanager/RMAppManager.java    |   7 +-
 .../server/resourcemanager/ResourceManager.java |  33 ++--
 .../resourcemanager/ResourceTrackerService.java |  21 +-
 .../resourcemanager/amlauncher/AMLauncher.java  |  15 +-
 .../metrics/TimelineServiceV2Publisher.java     |   2 +-
 .../server/resourcemanager/rmapp/RMAppImpl.java |  16 +-
 .../resourcemanager/TestClientRMService.java    |   3 +
 .../metrics/TestSystemMetricsPublisher.java     |   2 +-
 .../TestSystemMetricsPublisherForV2.java        |   1 +
 .../TestTimelineServiceClientIntegration.java   |  30 ++-
 .../PerNodeTimelineCollectorsAuxService.java    |  15 +-
 .../reader/TimelineReaderServer.java            |  14 +-
 ...TestPerNodeTimelineCollectorsAuxService.java |   9 +-
 .../reader/TestTimelineReaderServer.java        |   3 +
 .../reader/TestTimelineReaderWebServices.java   |   2 +
 ...stTimelineReaderWebServicesHBaseStorage.java |   2 +
 44 files changed, 534 insertions(+), 403 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 f52e654..19699fb 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
@@ -19,9 +19,6 @@
 package org.apache.hadoop.mapreduce.jobhistory;
 
 import java.io.IOException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -30,7 +27,11 @@ import java.util.Map;
 import java.util.Timer;
 import java.util.TimerTask;
 import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -46,7 +47,6 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.TaskStatus;
 import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.CounterGroup;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -56,9 +56,9 @@ import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.MRAppMaster;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
-import org.apache.hadoop.mapreduce.v2.app.MRAppMaster;
 import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
@@ -74,9 +74,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 /**
  * The job history events get routed to this class. This class writes the Job
  * history events to the DFS directly into a staging dir and then moved to a
@@ -122,20 +121,17 @@ public class JobHistoryEventHandler extends AbstractService
 
   protected static final Map<JobId, MetaInfo> fileMap =
     Collections.<JobId,MetaInfo>synchronizedMap(new HashMap<JobId,MetaInfo>());
-  
-  // For posting entities in new timeline service in a non-blocking way
-  // TODO YARN-3367 replace with event loop in TimelineClient.
-  private static ExecutorService threadPool =
-      Executors.newCachedThreadPool(
-          new ThreadFactoryBuilder().setNameFormat("TimelineService #%d")
-          .build());
 
   // should job completion be force when the AM shuts down?
   protected volatile boolean forceJobCompletion = false;
 
   protected TimelineClient timelineClient;
   
-  private boolean newTimelineServiceEnabled = false;
+  private boolean timelineServiceV2Enabled = false;
+
+  // For posting entities in new timeline service in a non-blocking way
+  // TODO YARN-3367 replace with event loop in TimelineClient.
+  private ExecutorService threadPool;
 
   private static String MAPREDUCE_JOB_ENTITY_TYPE = "MAPREDUCE_JOB";
   private static String MAPREDUCE_TASK_ENTITY_TYPE = "MAPREDUCE_TASK";
@@ -265,22 +261,26 @@ public class JobHistoryEventHandler extends AbstractService
     // configuration status: off, on_with_v1 or on_with_v2.
     if (conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA,
         MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA)) {
-      if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
-        
+      LOG.info("Emitting job history data to the timeline service is enabled");
+      if (YarnConfiguration.timelineServiceEnabled(conf)) {
+
         timelineClient = 
             ((MRAppMaster.RunningAppContext)context).getTimelineClient();
         timelineClient.init(conf);
-        newTimelineServiceEnabled = conf.getBoolean(
-            MRJobConfig.MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED,
-            MRJobConfig.DEFAULT_MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED);
-        LOG.info("Timeline service is enabled: " + (newTimelineServiceEnabled? "v2" : "v1"));
-        LOG.info("Emitting job history data to the timeline server is enabled");
+        timelineServiceV2Enabled =
+            YarnConfiguration.timelineServiceV2Enabled(conf);
+        LOG.info("Timeline service is enabled; version: " +
+            YarnConfiguration.getTimelineServiceVersion(conf));
+        if (timelineServiceV2Enabled) {
+          // initialize the thread pool for v.2 timeline service
+          threadPool = createThreadPool();
+        }
       } else {
         LOG.info("Timeline service is not enabled");
       }
     } else {
-      LOG.info("Emitting job history data to the timeline server is not enabled");
+      LOG.info("Emitting job history data to the timeline server is not " +
+          "enabled");
     }
 
     // Flag for setting
@@ -448,19 +448,27 @@ public class JobHistoryEventHandler extends AbstractService
     if (timelineClient != null) {
       timelineClient.stop();
     }
-    shutdownAndAwaitTermination();
+    if (threadPool != null) {
+      shutdownAndAwaitTermination();
+    }
     LOG.info("Stopped JobHistoryEventHandler. super.stop()");
     super.serviceStop();
   }
   
   // TODO remove threadPool after adding non-blocking call in TimelineClient
-  private static void shutdownAndAwaitTermination() {
+  private ExecutorService createThreadPool() {
+    return Executors.newCachedThreadPool(
+      new ThreadFactoryBuilder().setNameFormat("TimelineService #%d")
+      .build());
+  }
+
+  private void shutdownAndAwaitTermination() {
     threadPool.shutdown();
     try {
       if (!threadPool.awaitTermination(60, TimeUnit.SECONDS)) {
         threadPool.shutdownNow(); 
         if (!threadPool.awaitTermination(60, TimeUnit.SECONDS))
-            LOG.error("ThreadPool did not terminate");
+          LOG.error("ThreadPool did not terminate");
       }
     } catch (InterruptedException ie) {
       threadPool.shutdownNow();
@@ -622,7 +630,7 @@ public class JobHistoryEventHandler extends AbstractService
         processEventForJobSummary(event.getHistoryEvent(), mi.getJobSummary(),
             event.getJobID());
         if (timelineClient != null) {
-          if (newTimelineServiceEnabled) {
+          if (timelineServiceV2Enabled) {
             processEventForNewTimelineService(historyEvent, event.getJobID(),
                 event.getTimestamp());
           } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 3279d03..897e2aa 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
@@ -1058,14 +1058,9 @@ public class MRAppMaster extends CompositeService {
       this.taskAttemptFinishingMonitor = taskAttemptFinishingMonitor;
       if (conf.getBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA,
               MRJobConfig.DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA)
-            && conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-                YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
-
-        boolean newTimelineServiceEnabled = conf.getBoolean(
-            MRJobConfig.MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED,
-            MRJobConfig.DEFAULT_MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED);
-            
-        if (newTimelineServiceEnabled) {
+            && YarnConfiguration.timelineServiceEnabled(conf)) {
+
+        if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
           // create new version TimelineClient
           timelineClient = TimelineClient.createTimelineClient(
               appAttemptID.getApplicationId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index bc8aeda..c98746a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -467,11 +467,6 @@ public interface MRJobConfig {
     "mapreduce.job.emit-timeline-data";
   public static final boolean DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA =
       false;
-  
-  public static final String MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED =
-      "mapreduce.job.new-timeline-service.enabled";
-  public static final boolean DEFAULT_MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED =
-      false;
 
   public static final String MR_PREFIX = "yarn.app.mapreduce.";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
index 23118dd..962584c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml
@@ -618,13 +618,6 @@
     </description>
 </property>
 
- <property>
-    <name>mapreduce.job.new-timeline-service.enabled</name>
-    <value>false</value>
-    <description>Specifies if posting job and task events to new timeline service.
-    </description>
-</property>
-
 <property>
   <name>mapreduce.input.fileinputformat.split.minsize</name>
   <value>0</value>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
index a254bad..cf6d64f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMRTimelineEventHandling.java
@@ -165,11 +165,10 @@ public class TestMRTimelineEventHandling {
     LOG.info("testMRNewTimelineServiceEventHandling start.");
     Configuration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    // enable new timeline service
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
     conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, true);
 
-    // enable new timeline serivce in MR side
-    conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_NEW_TIMELINE_SERVICE_ENABLED, true);
-
     // enable aux-service based timeline collectors
     conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME);
     conf.set(YarnConfiguration.NM_AUX_SERVICES + "." + TIMELINE_AUX_SERVICE_NAME

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
index 18a4c14..edb825d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
@@ -173,7 +173,7 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
     boolean enableTimelineAuxService = false;
     if (nmAuxServices != null) {
       for (String nmAuxService: nmAuxServices) {
-        if (nmAuxService == TIMELINE_AUX_SERVICE_NAME) {
+        if (nmAuxService.equals(TIMELINE_AUX_SERVICE_NAME)) {
           enableTimelineAuxService = true;
           break;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 2946240..61b37c6 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
@@ -83,6 +83,10 @@ public class YarnConfiguration extends Configuration {
         new DeprecationDelta("yarn.client.max-nodemanagers-proxies",
             NM_CLIENT_MAX_NM_PROXIES)
     });
+    Configuration.addDeprecations(new DeprecationDelta[] {
+        new DeprecationDelta(RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
+            SYSTEM_METRICS_PUBLISHER_ENABLED)
+    });
   }
 
   //Configurations
@@ -392,7 +396,8 @@ public class YarnConfiguration extends Configuration {
 
   /**
    *  The setting that controls whether yarn system metrics is published on the
-   *  timeline server or not by RM. This configuration setting is for ATS V1
+   *  timeline server or not by RM. This configuration setting is for ATS V1.
+   *  This is now deprecated in favor of SYSTEM_METRICS_PUBLISHER_ENABLED.
    */
   public static final String RM_SYSTEM_METRICS_PUBLISHER_ENABLED = RM_PREFIX
       + "system-metrics-publisher.enabled";
@@ -2508,13 +2513,52 @@ public class YarnConfiguration extends Configuration {
     }
     return clusterId;
   }
-  
-  public static boolean systemMetricsPublisherEnabled(Configuration conf) {
+
+  // helper methods for timeline service configuration
+  /**
+   * Returns whether the timeline service is enabled via configuration.
+   *
+   * @param conf the configuration
+   * @return whether the timeline service is enabled.
+   */
+  public static boolean timelineServiceEnabled(Configuration conf) {
     return conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-        YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)
-        && conf.getBoolean(
-            YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED,
-            YarnConfiguration.DEFAULT_SYSTEM_METRICS_PUBLISHER_ENABLED);  
+      YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED);
+  }
+
+  /**
+   * Returns the timeline service version. It does not check whether the
+   * timeline service itself is enabled.
+   *
+   * @param conf the configuration
+   * @return the timeline service version as a float.
+   */
+  public static float getTimelineServiceVersion(Configuration conf) {
+    return conf.getFloat(TIMELINE_SERVICE_VERSION,
+        DEFAULT_TIMELINE_SERVICE_VERSION);
+  }
+
+  /**
+   * Returns whether the timeline service v.2 is enabled via configuration.
+   *
+   * @param conf the configuration
+   * @return whether the timeline service v.2 is enabled. V.2 refers to a
+   * version greater than equal to 2 but smaller than 3.
+   */
+  public static boolean timelineServiceV2Enabled(Configuration conf) {
+    return timelineServiceEnabled(conf) &&
+        (int)getTimelineServiceVersion(conf) == 2;
+  }
+
+  /**
+   * Returns whether the system publisher is enabled.
+   *
+   * @param conf the configuration
+   * @return whether the system publisher is enabled.
+   */
+  public static boolean systemMetricsPublisherEnabled(Configuration conf) {
+    return conf.getBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED,
+        YarnConfiguration.DEFAULT_SYSTEM_METRICS_PUBLISHER_ENABLED);
   }
 
   /* For debugging. mp configurations to system output as XML format. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 f185e66..1c68086 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
@@ -217,14 +217,11 @@ public class ApplicationMaster {
   // Tracking url to which app master publishes info for clients to monitor
   private String appMasterTrackingUrl = "";
 
-  private boolean newTimelineService = false;
+  private boolean timelineServiceV2 = false;
 
   // For posting entities in new timeline service in a non-blocking way
   // TODO replace with event loop in TimelineClient.
-  private static ExecutorService threadPool =
-      Executors.newCachedThreadPool(
-          new ThreadFactoryBuilder().setNameFormat("TimelineService #%d")
-          .build());
+  private ExecutorService threadPool;
 
   // App Master configuration
   // No. of containers to run shell command on
@@ -314,8 +311,10 @@ public class ApplicationMaster {
       }
       appMaster.run();
       result = appMaster.finish();
-      
-      shutdownAndAwaitTermination();
+
+      if (appMaster.threadPool != null) {
+        appMaster.shutdownAndAwaitTermination();
+      }
     } catch (Throwable t) {
       LOG.fatal("Error running ApplicationMaster", t);
       LogManager.shutdown();
@@ -329,16 +328,22 @@ public class ApplicationMaster {
       System.exit(2);
     }
   }
-  
+
   //TODO remove threadPool after adding non-blocking call in TimelineClient
-  private static void shutdownAndAwaitTermination() {
+  private ExecutorService createThreadPool() {
+    return Executors.newCachedThreadPool(
+        new ThreadFactoryBuilder().setNameFormat("TimelineService #%d")
+        .build());
+  }
+
+  private void shutdownAndAwaitTermination() {
     threadPool.shutdown();
     try {
       // Wait a while for existing tasks to terminate
       if (!threadPool.awaitTermination(60, TimeUnit.SECONDS)) {
         threadPool.shutdownNow();
         if (!threadPool.awaitTermination(60, TimeUnit.SECONDS))
-            LOG.error("ThreadPool did not terminate");
+          LOG.error("ThreadPool did not terminate");
       }
     } catch (InterruptedException ie) {
       threadPool.shutdownNow();
@@ -404,8 +409,7 @@ public class ApplicationMaster {
         "No. of containers on which the shell command needs to be executed");
     opts.addOption("priority", true, "Application Priority. Default 0");
     opts.addOption("debug", false, "Dump out debug information");
-    opts.addOption("timeline_service_version", true,
-        "Version for timeline service");
+
     opts.addOption("help", false, "Print usage");
     CommandLine cliParser = new GnuParser().parse(opts, args);
 
@@ -542,27 +546,15 @@ public class ApplicationMaster {
     requestPriority = Integer.parseInt(cliParser
         .getOptionValue("priority", "0"));
 
-    if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-      YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
-      if (cliParser.hasOption("timeline_service_version")) {
-        String timelineServiceVersion =
-            cliParser.getOptionValue("timeline_service_version", "v1");
-        if (timelineServiceVersion.trim().equalsIgnoreCase("v1")) {
-          newTimelineService = false;
-        } else if (timelineServiceVersion.trim().equalsIgnoreCase("v2")) {
-          newTimelineService = true;
-        } else {
-          throw new IllegalArgumentException(
-              "timeline_service_version is not set properly, should be 'v1' or 'v2'");
-        }
+    if (YarnConfiguration.timelineServiceEnabled(conf)) {
+      timelineServiceV2 =
+          YarnConfiguration.timelineServiceV2Enabled(conf);
+      if (timelineServiceV2) {
+        threadPool = createThreadPool();
       }
     } else {
       timelineClient = null;
       LOG.warn("Timeline service is not enabled");
-      if (cliParser.hasOption("timeline_service_version")) {
-        throw new IllegalArgumentException(
-            "Timeline service is not enabled");
-      }
     }
 
     return true;
@@ -624,16 +616,16 @@ public class ApplicationMaster {
     nmClientAsync.start();
 
     startTimelineClient(conf);
-    // need to bind timelineClient
-    amRMClient.registerTimelineClient(timelineClient);
+    if (timelineServiceV2) {
+      // need to bind timelineClient
+      amRMClient.registerTimelineClient(timelineClient);
+    }
     if(timelineClient != null) {
-      if (newTimelineService) {
-        publishApplicationAttemptEventOnNewTimelineService(timelineClient, 
-            appAttemptID.toString(), DSEvent.DS_APP_ATTEMPT_START, domainId, 
-            appSubmitterUgi);
+      if (timelineServiceV2) {
+        publishApplicationAttemptEventOnTimelineServiceV2(
+            DSEvent.DS_APP_ATTEMPT_START);
       } else {
-        publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
-            DSEvent.DS_APP_ATTEMPT_START, domainId, appSubmitterUgi);
+        publishApplicationAttemptEvent(DSEvent.DS_APP_ATTEMPT_START);
       }
     }
 
@@ -704,10 +696,9 @@ public class ApplicationMaster {
       appSubmitterUgi.doAs(new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
-          if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-              YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
+          if (YarnConfiguration.timelineServiceEnabled(conf)) {
             // Creating the Timeline Client
-            if (newTimelineService) {
+            if (timelineServiceV2) {
               timelineClient = TimelineClient.createTimelineClient(
                   appAttemptID.getApplicationId());
             } else {
@@ -743,13 +734,11 @@ public class ApplicationMaster {
     }
 
     if (timelineClient != null) {
-      if (newTimelineService) {
-        publishApplicationAttemptEventOnNewTimelineService(timelineClient,
-          appAttemptID.toString(), DSEvent.DS_APP_ATTEMPT_END, domainId,
-          appSubmitterUgi);
+      if (timelineServiceV2) {
+        publishApplicationAttemptEventOnTimelineServiceV2(
+            DSEvent.DS_APP_ATTEMPT_END);
       } else {
-        publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
-          DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
+        publishApplicationAttemptEvent(DSEvent.DS_APP_ATTEMPT_END);
       }
     }
 
@@ -856,12 +845,10 @@ public class ApplicationMaster {
               + containerStatus.getContainerId());
         }
         if(timelineClient != null) {
-          if (newTimelineService) {
-            publishContainerEndEventOnNewTimelineService(
-                timelineClient, containerStatus, domainId, appSubmitterUgi);
+          if (timelineServiceV2) {
+            publishContainerEndEventOnTimelineServiceV2(containerStatus);
           } else {
-            publishContainerEndEvent(
-                timelineClient, containerStatus, domainId, appSubmitterUgi);
+            publishContainerEndEvent(containerStatus);
           }
         }
       }
@@ -984,14 +971,11 @@ public class ApplicationMaster {
         applicationMaster.nmClientAsync.getContainerStatusAsync(containerId, container.getNodeId());
       }
       if(applicationMaster.timelineClient != null) {
-        if (applicationMaster.newTimelineService) {
-            ApplicationMaster.publishContainerStartEventOnNewTimelineService(
-                applicationMaster.timelineClient, container,
-                applicationMaster.domainId, applicationMaster.appSubmitterUgi);
+        if (applicationMaster.timelineServiceV2) {
+            applicationMaster.publishContainerStartEventOnTimelineServiceV2(
+                container);
         } else {
-          ApplicationMaster.publishContainerStartEvent(
-              applicationMaster.timelineClient, container,
-              applicationMaster.domainId, applicationMaster.appSubmitterUgi);
+          applicationMaster.publishContainerStartEvent(container);
         }
       }
     }
@@ -1207,14 +1191,12 @@ public class ApplicationMaster {
     }
   }
   
-  private static void publishContainerStartEvent(
-      final TimelineClient timelineClient, Container container, String domainId,
-      UserGroupInformation ugi) {
+  private void publishContainerStartEvent(Container container) {
     final TimelineEntity entity = new TimelineEntity();
     entity.setEntityId(container.getId().toString());
     entity.setEntityType(DSEntity.DS_CONTAINER.toString());
     entity.setDomainId(domainId);
-    entity.addPrimaryFilter("user", ugi.getShortUserName());
+    entity.addPrimaryFilter("user", appSubmitterUgi.getShortUserName());
     TimelineEvent event = new TimelineEvent();
     event.setTimestamp(System.currentTimeMillis());
     event.setEventType(DSEvent.DS_CONTAINER_START.toString());
@@ -1223,13 +1205,13 @@ public class ApplicationMaster {
     entity.addEvent(event);
 
     try {
-      ugi.doAs(new PrivilegedExceptionAction<TimelinePutResponse>() {
-        @Override
-        public TimelinePutResponse run() throws Exception {
-          return processTimelineResponseErrors(
-              timelineClient.putEntities(entity));
-        }
-      });
+      appSubmitterUgi.doAs(
+          new PrivilegedExceptionAction<TimelinePutResponse>() {
+            @Override
+            public TimelinePutResponse run() throws Exception {
+              return timelineClient.putEntities(entity);
+            }
+        });
     } catch (Exception e) {
       LOG.error("Container start event could not be published for "
           + container.getId().toString(),
@@ -1237,14 +1219,12 @@ public class ApplicationMaster {
     }
   }
 
-  private static void publishContainerEndEvent(
-      final TimelineClient timelineClient, ContainerStatus container,
-      String domainId, UserGroupInformation ugi) {
+  private void publishContainerEndEvent(ContainerStatus container) {
     final TimelineEntity entity = new TimelineEntity();
     entity.setEntityId(container.getContainerId().toString());
     entity.setEntityType(DSEntity.DS_CONTAINER.toString());
     entity.setDomainId(domainId);
-    entity.addPrimaryFilter("user", ugi.getShortUserName());
+    entity.addPrimaryFilter("user", appSubmitterUgi.getShortUserName());
     TimelineEvent event = new TimelineEvent();
     event.setTimestamp(System.currentTimeMillis());
     event.setEventType(DSEvent.DS_CONTAINER_END.toString());
@@ -1260,14 +1240,12 @@ public class ApplicationMaster {
     }
   }
 
-  private static void publishApplicationAttemptEvent(
-      final TimelineClient timelineClient, String appAttemptId,
-      DSEvent appEvent, String domainId, UserGroupInformation ugi) {
+  private void publishApplicationAttemptEvent(DSEvent appEvent) {
     final TimelineEntity entity = new TimelineEntity();
-    entity.setEntityId(appAttemptId);
+    entity.setEntityId(appAttemptID.toString());
     entity.setEntityType(DSEntity.DS_APP_ATTEMPT.toString());
     entity.setDomainId(domainId);
-    entity.addPrimaryFilter("user", ugi.getShortUserName());
+    entity.addPrimaryFilter("user", appSubmitterUgi.getShortUserName());
     TimelineEvent event = new TimelineEvent();
     event.setEventType(appEvent.toString());
     event.setTimestamp(System.currentTimeMillis());
@@ -1279,7 +1257,7 @@ public class ApplicationMaster {
       LOG.error("App Attempt "
           + (appEvent.equals(DSEvent.DS_APP_ATTEMPT_START) ? "start" : "end")
           + " event could not be published for "
-          + appAttemptId.toString(), e);
+          + appAttemptID, e);
     }
   }
 
@@ -1327,27 +1305,24 @@ public class ApplicationMaster {
     return new Thread(runnableLaunchContainer);
   }
   
-  private static void publishContainerStartEventOnNewTimelineService(
-      final TimelineClient timelineClient, final Container container,
-      final String domainId, final UserGroupInformation ugi) {
+  private void publishContainerStartEventOnTimelineServiceV2(
+      final Container container) {
     Runnable publishWrapper = new Runnable() {
       public void run() {
-        publishContainerStartEventOnNewTimelineServiceBase(timelineClient,
-            container, domainId, ugi);
+        publishContainerStartEventOnTimelineServiceV2Base(container);
       }
     };
     threadPool.execute(publishWrapper);
   }
 
-  private static void publishContainerStartEventOnNewTimelineServiceBase(
-      final TimelineClient timelineClient, Container container, String domainId,
-      UserGroupInformation ugi) {
+  private void publishContainerStartEventOnTimelineServiceV2Base(
+      Container container) {
     final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
     entity.setId(container.getId().toString());
     entity.setType(DSEntity.DS_CONTAINER.toString());
     //entity.setDomainId(domainId);
-    entity.addInfo("user", ugi.getShortUserName());
+    entity.addInfo("user", appSubmitterUgi.getShortUserName());
 
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
@@ -1358,7 +1333,7 @@ public class ApplicationMaster {
     entity.addEvent(event);
 
     try {
-      ugi.doAs(new PrivilegedExceptionAction<Object>() {
+      appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
         @Override
         public TimelinePutResponse run() throws Exception {
           timelineClient.putEntities(entity);
@@ -1372,27 +1347,24 @@ public class ApplicationMaster {
     }
   }
 
-  private static void publishContainerEndEventOnNewTimelineService(
-      final TimelineClient timelineClient, final ContainerStatus container,
-      final String domainId, final UserGroupInformation ugi) {
+  private void publishContainerEndEventOnTimelineServiceV2(
+      final ContainerStatus container) {
     Runnable publishWrapper = new Runnable() {
       public void run() {
-          publishContainerEndEventOnNewTimelineServiceBase(timelineClient,
-              container, domainId, ugi);
+          publishContainerEndEventOnTimelineServiceV2Base(container);
       }
     };
     threadPool.execute(publishWrapper);
   }
 
-  private static void publishContainerEndEventOnNewTimelineServiceBase(
-      final TimelineClient timelineClient, final ContainerStatus container,
-      final String domainId, final UserGroupInformation ugi) {
+  private void publishContainerEndEventOnTimelineServiceV2Base(
+      final ContainerStatus container) {
     final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
     entity.setId(container.getContainerId().toString());
     entity.setType(DSEntity.DS_CONTAINER.toString());
     //entity.setDomainId(domainId);
-    entity.addInfo("user", ugi.getShortUserName());
+    entity.addInfo("user", appSubmitterUgi.getShortUserName());
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
         new  org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
     event.setTimestamp(System.currentTimeMillis());
@@ -1402,7 +1374,7 @@ public class ApplicationMaster {
     entity.addEvent(event);
 
     try {
-      ugi.doAs(new PrivilegedExceptionAction<Object>() {
+      appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
         @Override
         public TimelinePutResponse run() throws Exception {
           timelineClient.putEntities(entity);
@@ -1416,29 +1388,25 @@ public class ApplicationMaster {
     }
   }
 
-  private static void publishApplicationAttemptEventOnNewTimelineService(
-      final TimelineClient timelineClient, final String appAttemptId,
-      final DSEvent appEvent, final String domainId,
-      final UserGroupInformation ugi) {
+  private void publishApplicationAttemptEventOnTimelineServiceV2(
+      final DSEvent appEvent) {
 
     Runnable publishWrapper = new Runnable() {
       public void run() {
-        publishApplicationAttemptEventOnNewTimelineServiceBase(timelineClient,
-            appAttemptId, appEvent, domainId, ugi);
+        publishApplicationAttemptEventOnTimelineServiceV2Base(appEvent);
       }
     };
     threadPool.execute(publishWrapper);
   }
 
-  private static void publishApplicationAttemptEventOnNewTimelineServiceBase(
-      final TimelineClient timelineClient, String appAttemptId,
-      DSEvent appEvent, String domainId, UserGroupInformation ugi) {
+  private void publishApplicationAttemptEventOnTimelineServiceV2Base(
+      DSEvent appEvent) {
     final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
-    entity.setId(appAttemptId);
+    entity.setId(appAttemptID.toString());
     entity.setType(DSEntity.DS_APP_ATTEMPT.toString());
     //entity.setDomainId(domainId);
-    entity.addInfo("user", ugi.getShortUserName());
+    entity.addInfo("user", appSubmitterUgi.getShortUserName());
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent event =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent();
     event.setId(appEvent.toString());
@@ -1446,7 +1414,7 @@ public class ApplicationMaster {
     entity.addEvent(event);
 
     try {
-      ugi.doAs(new PrivilegedExceptionAction<Object>() {
+      appSubmitterUgi.doAs(new PrivilegedExceptionAction<Object>() {
         @Override
         public TimelinePutResponse run() throws Exception {
           timelineClient.putEntities(entity);
@@ -1457,7 +1425,7 @@ public class ApplicationMaster {
       LOG.error("App Attempt "
           + (appEvent.equals(DSEvent.DS_APP_ATTEMPT_START) ? "start" : "end")
           + " event could not be published for "
-          + appAttemptId.toString(),
+          + appAttemptID,
           e instanceof UndeclaredThrowableException ? e.getCause() : e);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.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/Client.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
index 2819c91..e66005e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
@@ -192,8 +192,6 @@ public class Client {
   // Command line options
   private Options opts;
 
-  private String timelineServiceVersion;
-
   private static final String shellCommandPath = "shellCommands";
   private static final String shellArgsPath = "shellArgs";
   private static final String appMasterJarPath = "AppMaster.jar";
@@ -269,7 +267,6 @@ public class Client {
     opts.addOption("container_vcores", true, "Amount of virtual cores to be requested to run the shell command");
     opts.addOption("num_containers", true, "No. of containers on which the shell command needs to be executed");
     opts.addOption("log_properties", true, "log4j.properties file");
-    opts.addOption("timeline_service_version", true, "Version for timeline service");
     opts.addOption("keep_containers_across_application_attempts", false,
       "Flag to indicate whether to keep containers across application attempts." +
       " If the flag is true, running containers will not be killed when" +
@@ -371,16 +368,6 @@ public class Client {
           + " Specified virtual cores=" + amVCores);
     }
 
-    if (cliParser.hasOption("timeline_service_version")) {
-      timelineServiceVersion =
-        cliParser.getOptionValue("timeline_service_version", "v1");
-      if (! (timelineServiceVersion.trim().equalsIgnoreCase("v1") ||
-          timelineServiceVersion.trim().equalsIgnoreCase("v2"))) {
-        throw new IllegalArgumentException(
-              "timeline_service_version is not set properly, should be 'v1' or 'v2'");
-      }
-    }
-
     if (!cliParser.hasOption("jar")) {
       throw new IllegalArgumentException("No jar file specified for application master");
     }		
@@ -690,9 +677,6 @@ public class Client {
       vargs.add("--debug");
     }
 
-    if (timelineServiceVersion != null) {
-      vargs.add("--timeline_service_version " + timelineServiceVersion);
-    }
     vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/AppMaster.stdout");
     vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/AppMaster.stderr");
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index fe817c3..b3ff9b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -99,22 +99,19 @@ public class TestDistributedShell {
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128);
     conf.set("yarn.log.dir", "target");
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true);
 
     if (!testName.getMethodName().toLowerCase().contains("v2")) {
       // disable aux-service based timeline collectors
       conf.set(YarnConfiguration.NM_AUX_SERVICES, "");
-      conf.setBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
-          true);
-      conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, false);
     } else {
+      // set version to 2
+      conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
       // enable aux-service based timeline collectors
       conf.set(YarnConfiguration.NM_AUX_SERVICES, TIMELINE_AUX_SERVICE_NAME);
       conf.set(YarnConfiguration.NM_AUX_SERVICES + "."
           + TIMELINE_AUX_SERVICE_NAME + ".class",
           PerNodeTimelineCollectorsAuxService.class.getName());
-      conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true);
-      conf.setBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
-          false);
     }
     conf.set(YarnConfiguration.NM_VMEM_PMEM_RATIO, "8");
     conf.set(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class.getName());
@@ -245,12 +242,7 @@ public class TestDistributedShell {
     }
     boolean isTestingTimelineV2 = false;
     if (timelineVersion.equalsIgnoreCase("v2")) {
-      String[] timelineArgs = {
-          "--timeline_service_version",
-          "v2"
-      };
       isTestingTimelineV2 = true;
-      args = mergeArgs(args, timelineArgs);
       if (!defaultFlow) {
         String[] flowArgs = {
             "--flow_name",

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 01548ee..59ac5b3 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
@@ -52,17 +52,21 @@ public abstract class TimelineClient extends AbstractService {
    * 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 a timeline client
    */
   protected ApplicationId contextAppId;
 
+  /**
+   * Creates an instance of the timeline v.1.x client.
+   */
   @Public
   public static TimelineClient createTimelineClient() {
     TimelineClient client = new TimelineClientImpl();
     return client;
   }
 
+  /**
+   * Creates an instance of the timeline v.2 client.
+   */
   @Public
   public static TimelineClient createTimelineClient(ApplicationId appId) {
     TimelineClient client = new TimelineClientImpl(appId);
@@ -201,8 +205,9 @@ public abstract class TimelineClient extends AbstractService {
   /**
    * <p>
    * Send the information of a number of conceptual entities to the timeline
-   * aggregator. It is a blocking API. The method will not return until all the
-   * put entities have been persisted.
+   * 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
@@ -218,8 +223,9 @@ public abstract class TimelineClient extends AbstractService {
   /**
    * <p>
    * Send the information of a number of conceptual entities to the timeline
-   * aggregator. It is an asynchronous API. The method will return once all the
-   * entities are received.
+   * 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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 0dabab9..f07c479 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
@@ -378,6 +378,9 @@ public class TimelineClientImpl extends TimelineClient {
   private void putEntities(boolean async,
       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");
+    }
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
         entitiesContainer =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 a748135..077fb5d 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
@@ -771,14 +771,15 @@
   <property>
     <description>The setting that controls whether yarn system metrics is
     published to the Timeline server (version one) or not, by RM. 
-    This configuration is deprecated.</description>
+    This configuration is now deprecated in favor of
+    yarn.system-metrics-publisher.enabled.</description>
     <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>
     <value>false</value>
   </property>
 
   <property>
     <description>The setting that controls whether yarn system metrics is
-    published on the Timeline server (version two) or not by RM And NM.</description>
+    published on the Timeline service or not by RM And NM.</description>
     <name>yarn.system-metrics-publisher.enabled</name>
     <value>false</value>
   </property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
index fa0cf5c..066abfc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatRequestPBImpl.java
@@ -261,10 +261,12 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
   private void initRegisteredCollectors() {
     NodeHeartbeatRequestProtoOrBuilder p = viaProto ? proto : builder;
     List<AppCollectorsMapProto> list = p.getRegisteredCollectorsList();
-    this.registeredCollectors = new HashMap<ApplicationId, String> ();
-    for (AppCollectorsMapProto c : list) {
-      ApplicationId appId = convertFromProtoFormat(c.getAppId());
-      this.registeredCollectors.put(appId, c.getAppCollectorAddr());
+    if (!list.isEmpty()) {
+      this.registeredCollectors = new HashMap<>();
+      for (AppCollectorsMapProto c : list) {
+        ApplicationId appId = convertFromProtoFormat(c.getAppId());
+        this.registeredCollectors.put(appId, c.getAppCollectorAddr());
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
index 2521b9c..151006b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/NodeHeartbeatResponsePBImpl.java
@@ -531,10 +531,12 @@ public class NodeHeartbeatResponsePBImpl extends
   private void initAppCollectorsMap() {
     NodeHeartbeatResponseProtoOrBuilder p = viaProto ? proto : builder;
     List<AppCollectorsMapProto> list = p.getAppCollectorsMapList();
-    this.appCollectorsMap = new HashMap<ApplicationId, String> ();
-    for (AppCollectorsMapProto c : list) {
-      ApplicationId appId = convertFromProtoFormat(c.getAppId());
-      this.appCollectorsMap.put(appId, c.getAppCollectorAddr());
+    if (!list.isEmpty()) {
+      this.appCollectorsMap = new HashMap<>();
+      for (AppCollectorsMapProto c : list) {
+        ApplicationId appId = convertFromProtoFormat(c.getAppId());
+        this.appCollectorsMap.put(appId, c.getAppCollectorAddr());
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.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/Context.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
index 0b378a1..8fce422 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/Context.java
@@ -64,7 +64,8 @@ public interface Context {
 
   /**
    * Get the registered collectors that located on this NM.
-   * @return registered
+   * @return registered collectors, or null if the timeline service v.2 is not
+   * enabled
    */
   Map<ApplicationId, String> getRegisteredCollectors();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.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/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 57e649e..596b5d3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -97,6 +97,7 @@ public class NodeManager extends CompositeService
   private Context context;
   private AsyncDispatcher dispatcher;
   private ContainerManagerImpl containerManager;
+  // the NM collector service is set only if the timeline service v.2 is enabled
   private NMCollectorService nmCollectorService;
   private NodeStatusUpdater nodeStatusUpdater;
   private NodeResourceMonitor nodeResourceMonitor;
@@ -357,8 +358,10 @@ public class NodeManager extends CompositeService
 
     DefaultMetricsSystem.initialize("NodeManager");
 
-    this.nmCollectorService = createNMCollectorService(context);
-    addService(nmCollectorService);
+    if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      this.nmCollectorService = createNMCollectorService(context);
+      addService(nmCollectorService);
+    }
 
     // StatusUpdater should be added last so that it get started last 
     // so that we make sure everything is up before registering with RM. 
@@ -454,8 +457,7 @@ public class NodeManager extends CompositeService
     protected final ConcurrentMap<ContainerId, Container> containers =
         new ConcurrentSkipListMap<ContainerId, Container>();
 
-    protected Map<ApplicationId, String> registeredCollectors =
-        new ConcurrentHashMap<ApplicationId, String>();
+    protected Map<ApplicationId, String> registeredCollectors;
 
     protected final ConcurrentMap<ContainerId,
         org.apache.hadoop.yarn.api.records.Container> increasedContainers =
@@ -481,6 +483,9 @@ public class NodeManager extends CompositeService
         NMTokenSecretManagerInNM nmTokenSecretManager,
         LocalDirsHandlerService dirsHandler, ApplicationACLsManager aclsManager,
         NMStateStoreService stateStore, Configuration conf) {
+      if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+        this.registeredCollectors = new ConcurrentHashMap<>();
+      }
       this.containerTokenSecretManager = containerTokenSecretManager;
       this.nmTokenSecretManager = nmTokenSecretManager;
       this.dirsHandler = dirsHandler;
@@ -706,7 +711,14 @@ public class NodeManager extends CompositeService
     return this.context;
   }
 
-  // For testing
+  /**
+   * Returns the NM collector service. It should be used only for testing
+   * purposes.
+   *
+   * @return the NM collector service, or null if the timeline service v.2 is
+   * not enabled
+   */
+  @VisibleForTesting
   NMCollectorService getNMCollectorService() {
     return this.nmCollectorService;
   }
@@ -714,6 +726,7 @@ public class NodeManager extends CompositeService
   public static void main(String[] args) throws IOException {
     Thread.setDefaultUncaughtExceptionHandler(new YarnUncaughtExceptionHandler());
     StringUtils.startupShutdownMessage(NodeManager.class, args, LOG);
+    @SuppressWarnings("resource")
     NodeManager nodeManager = new NodeManager();
     Configuration conf = new YarnConfiguration();
     new GenericOptionsParser(conf, args);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.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/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index 69279d1..544a6f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -823,7 +823,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
               dispatcher.getEventHandler().handle(
                   new CMgrSignalContainersEvent(containersToSignal));
             }
-            if (YarnConfiguration.systemMetricsPublisherEnabled(context.getConf())) {
+            if (YarnConfiguration.timelineServiceV2Enabled(context.getConf())) {
               updateTimelineClientsAddress(response);
             }
 
@@ -857,7 +857,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
       /**
        * Caller should take care of sending non null nodelabels for both
        * arguments
-       * 
+       *
        * @param nodeLabelsNew
        * @param nodeLabelsOld
        * @return if the New node labels are diff from the older one.
@@ -873,27 +873,37 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
 
       private void updateTimelineClientsAddress(
           NodeHeartbeatResponse response) {
-        Set<Map.Entry<ApplicationId, String>> rmKnownCollectors = 
-            response.getAppCollectorsMap().entrySet();
-        for (Map.Entry<ApplicationId, String> entry : rmKnownCollectors) {
-          ApplicationId appId = entry.getKey();
-          String collectorAddr = entry.getValue();
-
-          // Only handle applications running on local node.
-          // Not include apps with timeline collectors running in local
-          Application application = context.getApplications().get(appId);
-          if (application != null &&
-              !context.getRegisteredCollectors().containsKey(appId)) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Sync a new collector address: " + collectorAddr + 
-                  " for application: " + appId + " from RM.");
+        Map<ApplicationId, String> knownCollectorsMap =
+            response.getAppCollectorsMap();
+        if (knownCollectorsMap == null) {
+          LOG.warn("the collectors map is null");
+        } else {
+          Set<Map.Entry<ApplicationId, String>> rmKnownCollectors =
+              knownCollectorsMap.entrySet();
+          for (Map.Entry<ApplicationId, String> entry : rmKnownCollectors) {
+            ApplicationId appId = entry.getKey();
+            String collectorAddr = entry.getValue();
+
+            // Only handle applications running on local node.
+            // Not include apps with timeline collectors running in local
+            Application application = context.getApplications().get(appId);
+            // TODO this logic could be problematic if the collector address
+            // gets updated due to NM restart or collector service failure
+            if (application != null &&
+                !context.getRegisteredCollectors().containsKey(appId)) {
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Sync a new collector address: " + collectorAddr +
+                    " for application: " + appId + " from RM.");
+              }
+              TimelineClient client = application.getTimelineClient();
+              if (client != null) {
+                client.setTimelineServiceAddress(collectorAddr);
+              }
             }
-            TimelineClient client = application.getTimelineClient();
-            client.setTimelineServiceAddress(collectorAddr);
           }
         }
       }
-      
+
       private void updateMasterKeys(NodeHeartbeatResponse response) {
         // See if the master-key has rolled over
         MasterKey updatedMasterKey = response.getContainerTokenMasterKey();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/NMCollectorService.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/collectormanager/NMCollectorService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/NMCollectorService.java
index db79ee5..3ba81ce 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/NMCollectorService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/collectormanager/NMCollectorService.java
@@ -43,6 +43,10 @@ import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 
+/**
+ * Service that handles collector information. It is used only if the timeline
+ * service v.2 is enabled.
+ */
 public class NMCollectorService extends CompositeService implements
     CollectorNodemanagerProtocol {
 
@@ -113,9 +117,9 @@ public class NMCollectorService extends CompositeService implements
         String collectorAddr = collector.getCollectorAddr();
         newCollectorsMap.put(appId, collectorAddr);
         // set registered collector address to TimelineClient.
-        if (YarnConfiguration.systemMetricsPublisherEnabled(context.getConf())) {
-          TimelineClient client = 
-              context.getApplications().get(appId).getTimelineClient();
+        TimelineClient client =
+            context.getApplications().get(appId).getTimelineClient();
+        if (client != null) {
           client.setTimelineServiceAddress(collectorAddr);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 b010eee..e4668c1 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
@@ -115,6 +115,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationFinishEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl.FlowContext;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationInitEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
@@ -190,7 +191,8 @@ public class ContainerManagerImpl extends CompositeService implements
 
   private long waitForContainersOnShutdownMillis;
 
-  private final NMTimelinePublisher nmMetricsPublisher;
+  // NM metrics publisher is set only if the timeline service v.2 is enabled
+  private NMTimelinePublisher nmMetricsPublisher;
 
   public ContainerManagerImpl(Context context, ContainerExecutor exec,
       DeletionService deletionContext, NodeStatusUpdater nodeStatusUpdater,
@@ -218,8 +220,15 @@ public class ContainerManagerImpl extends CompositeService implements
     auxiliaryServices.registerServiceListener(this);
     addService(auxiliaryServices);
 
-    nmMetricsPublisher = createNMTimelinePublisher(context);
-    context.setNMTimelinePublisher(nmMetricsPublisher);
+    // initialize the metrics publisher if the timeline service v.2 is enabled
+    // and the system publisher is enabled
+    Configuration conf = context.getConf();
+    if (YarnConfiguration.timelineServiceV2Enabled(conf) &&
+        YarnConfiguration.systemMetricsPublisherEnabled(conf)) {
+      LOG.info("YARN system metrics publishing service is enabled");
+      nmMetricsPublisher = createNMTimelinePublisher(context);
+      context.setNMTimelinePublisher(nmMetricsPublisher);
+    }
     this.containersMonitor =
         new ContainersMonitorImpl(exec, dispatcher, this.context);
     addService(this.containersMonitor);
@@ -237,7 +246,6 @@ public class ContainerManagerImpl extends CompositeService implements
     
     addService(dispatcher);
 
-
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
     this.readLock = lock.readLock();
     this.writeLock = lock.writeLock();
@@ -335,7 +343,7 @@ public class ContainerManagerImpl extends CompositeService implements
     LOG.info("Recovering application " + appId);
     //TODO: Recover flow and flow run ID
     ApplicationImpl app = new ApplicationImpl(
-        dispatcher, p.getUser(), null, null, 0, appId, creds, context);
+        dispatcher, p.getUser(), appId, creds, context);
     context.getApplications().put(appId, app);
     app.handle(new ApplicationInitEvent(appId, acls, logAggregationContext));
   }
@@ -941,20 +949,27 @@ public class ContainerManagerImpl extends CompositeService implements
     try {
       if (!serviceStopped) {
         // Create the application
-        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);
+        // 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)) {
           Application application =
-              new ApplicationImpl(dispatcher, user, flowName, flowVersion,
-                  flowRunId, applicationID, credentials, context);
+              new ApplicationImpl(dispatcher, user, flowContext,
+                  applicationID, credentials, context);
           if (context.getApplications().putIfAbsent(applicationID,
               application) == null) {
             LOG.info("Creating a new application reference for app "
@@ -1310,7 +1325,9 @@ public class ContainerManagerImpl extends CompositeService implements
       Container c = containers.get(event.getContainerID());
       if (c != null) {
         c.handle(event);
-        nmMetricsPublisher.publishContainerEvent(event);
+        if (nmMetricsPublisher != null) {
+          nmMetricsPublisher.publishContainerEvent(event);
+        }
       } else {
         LOG.warn("Event " + event + " sent to absent container " +
             event.getContainerID());
@@ -1326,7 +1343,9 @@ public class ContainerManagerImpl extends CompositeService implements
               event.getApplicationID());
       if (app != null) {
         app.handle(event);
-        nmMetricsPublisher.publishApplicationEvent(event);
+        if (nmMetricsPublisher != null) {
+          nmMetricsPublisher.publishApplicationEvent(event);
+        }
       } else {
         LOG.warn("Event " + event + " sent to absent application "
             + event.getApplicationID());
@@ -1349,7 +1368,9 @@ public class ContainerManagerImpl extends CompositeService implements
     @Override
     public void handle(LocalizationEvent event) {
       origLocalizationEventHandler.handle(event);
-      timelinePublisher.publishLocalizationEvent(event);
+      if (timelinePublisher != null) {
+        timelinePublisher.publishLocalizationEvent(event);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/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 6e87cfd..93c6758 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
@@ -67,9 +67,8 @@ public class ApplicationImpl implements Application {
 
   final Dispatcher dispatcher;
   final String user;
-  final String flowName;
-  final String flowVersion;
-  final long flowRunId;
+  // flow context is set only if the timeline service v.2 is enabled
+  private FlowContext flowContext;
   final ApplicationId appId;
   final Credentials credentials;
   Map<ApplicationAccessType, String> applicationACLs;
@@ -86,14 +85,16 @@ public class ApplicationImpl implements Application {
   Map<ContainerId, Container> containers =
       new HashMap<ContainerId, Container>();
 
-  public ApplicationImpl(Dispatcher dispatcher, String user, String flowName,
-      String flowVersion, long flowRunId, ApplicationId appId,
-      Credentials credentials, Context context) {
+  public ApplicationImpl(Dispatcher dispatcher, String user,
+      ApplicationId appId, Credentials credentials, Context context) {
+    this(dispatcher, user, null, appId, credentials, context);
+  }
+
+  public ApplicationImpl(Dispatcher dispatcher, String user,
+      FlowContext flowContext, ApplicationId appId, Credentials credentials,
+      Context context) {
     this.dispatcher = dispatcher;
     this.user = user;
-    this.flowName = flowName;
-    this.flowVersion = flowVersion;
-    this.flowRunId = flowRunId;
     this.appId = appId;
     this.credentials = credentials;
     this.aclsManager = context.getApplicationACLsManager();
@@ -103,11 +104,44 @@ public class ApplicationImpl implements Application {
     writeLock = lock.writeLock();
     stateMachine = stateMachineFactory.make(this);
     Configuration conf = context.getConf();
-    if (YarnConfiguration.systemMetricsPublisherEnabled(conf)) {
-      createAndStartTimelineClient(conf);
+    if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      if (flowContext == null) {
+        throw new IllegalArgumentException("flow context cannot be null");
+      }
+      this.flowContext = flowContext;
+      if (YarnConfiguration.systemMetricsPublisherEnabled(conf)) {
+        createAndStartTimelineClient(conf);
+      }
     }
   }
-  
+
+  /**
+   * Data object that encapsulates the flow context for the application purpose.
+   */
+  public static class FlowContext {
+    private final String flowName;
+    private final String flowVersion;
+    private final long flowRunId;
+
+    public FlowContext(String flowName, String flowVersion, long flowRunId) {
+      this.flowName = flowName;
+      this.flowVersion = flowVersion;
+      this.flowRunId = flowRunId;
+    }
+
+    public String getFlowName() {
+      return flowName;
+    }
+
+    public String getFlowVersion() {
+      return flowVersion;
+    }
+
+    public long getFlowRunId() {
+      return flowRunId;
+    }
+  }
+
   private void createAndStartTimelineClient(Configuration conf) {
     // create and start timeline client
     this.timelineClient = TimelineClient.createTimelineClient(appId);
@@ -454,7 +488,11 @@ public class ApplicationImpl implements Application {
       // Remove collectors info for finished apps.
       // TODO check we remove related collectors info in failure cases
       // (YARN-3038)
-      app.context.getRegisteredCollectors().remove(app.getAppId());
+      Map<ApplicationId, String> registeredCollectors =
+          app.context.getRegisteredCollectors();
+      if (registeredCollectors != null) {
+        registeredCollectors.remove(app.getAppId());
+      }
       // stop timelineClient when application get finished.
       TimelineClient timelineClient = app.getTimelineClient();
       if (timelineClient != null) {
@@ -521,16 +559,16 @@ public class ApplicationImpl implements Application {
 
   @Override
   public String getFlowName() {
-    return flowName;
+    return flowContext == null ? null : flowContext.getFlowName();
   }
 
   @Override
   public String getFlowVersion() {
-    return flowVersion;
+    return flowContext == null ? null : flowContext.getFlowVersion();
   }
 
   @Override
   public long getFlowRunId() {
-    return flowRunId;
+    return flowContext == null ? 0L : flowContext.getFlowRunId();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5591eebd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.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/monitor/ContainersMonitorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
index 3b6b6c4..0553fe0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
+import org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher;
 import org.apache.hadoop.yarn.server.nodemanager.util.NodeManagerHardwareUtils;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
@@ -560,9 +561,13 @@ public class ContainersMonitorImpl extends AbstractService implements
 
             ContainerImpl container =
                 (ContainerImpl) context.getContainers().get(containerId);
-            container.getNMTimelinePublisher().reportContainerResourceUsage(
-                container, currentTime, pId, currentPmemUsage,
-                cpuUsageTotalCoresPercentage);
+            NMTimelinePublisher nmMetricsPublisher =
+                container.getNMTimelinePublisher();
+            if (nmMetricsPublisher != null) {
+              nmMetricsPublisher.reportContainerResourceUsage(
+                  container, currentTime, pId, currentPmemUsage,
+                  cpuUsageTotalCoresPercentage);
+            }
           } catch (Exception e) {
             // Log the exception and proceed to the next container.
             LOG.warn("Uncaught exception in ContainersMonitorImpl "


[32/50] [abbrv] hadoop git commit: YARN-4297. TestJobHistoryEventHandler and TestRMContainerAllocator failing on YARN-2928 branch (Varun Saxena via sjlee)

Posted by gt...@apache.org.
YARN-4297. TestJobHistoryEventHandler and TestRMContainerAllocator failing on YARN-2928 branch (Varun Saxena via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: 54a529df6bf4e87de63700053b5755b55ad2a002
Parents: e6f57ec
Author: Sangjin Lee <sj...@apache.org>
Authored: Wed Nov 25 11:28:00 2015 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:59:39 2016 -0800

----------------------------------------------------------------------
 .../jobhistory/TestJobHistoryEventHandler.java  | 28 +++++++++++++++-----
 .../v2/app/rm/TestRMContainerAllocator.java     |  3 ++-
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../collector/NodeTimelineCollectorManager.java |  2 --
 4 files changed, 26 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/54a529df/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 c173461..a84e6d2 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
@@ -54,6 +54,7 @@ import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.MRAppMaster.RunningAppContext;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
@@ -65,6 +66,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 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.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
@@ -486,7 +488,7 @@ public class TestJobHistoryEventHandler {
   // stored to the Timeline store
   @Test (timeout=50000)
   public void testTimelineEventHandling() throws Exception {
-    TestParams t = new TestParams(false);
+    TestParams t = new TestParams(RunningAppContext.class, false);
     Configuration conf = new YarnConfiguration();
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     MiniYARNCluster yarnCluster = null;
@@ -741,21 +743,30 @@ public class TestJobHistoryEventHandler {
     }
   }
 
-  private AppContext mockAppContext(ApplicationId appId, boolean isLastAMRetry) {
-    JobId jobId = TypeConverter.toYarn(TypeConverter.fromYarn(appId));
-    AppContext mockContext = mock(AppContext.class);
+  private Job mockJob() {
     Job mockJob = mock(Job.class);
     when(mockJob.getAllCounters()).thenReturn(new Counters());
     when(mockJob.getTotalMaps()).thenReturn(10);
     when(mockJob.getTotalReduces()).thenReturn(10);
     when(mockJob.getName()).thenReturn("mockjob");
+    return mockJob;
+  }
+
+  private AppContext mockAppContext(Class<? extends AppContext> contextClass,
+      ApplicationId appId, boolean isLastAMRetry) {
+    JobId jobId = TypeConverter.toYarn(TypeConverter.fromYarn(appId));
+    AppContext mockContext = mock(contextClass);
+    Job mockJob = mockJob();
     when(mockContext.getJob(jobId)).thenReturn(mockJob);
     when(mockContext.getApplicationID()).thenReturn(appId);
     when(mockContext.isLastAMRetry()).thenReturn(isLastAMRetry);
+    if (mockContext instanceof RunningAppContext) {
+      when(((RunningAppContext)mockContext).getTimelineClient()).
+          thenReturn(TimelineClient.createTimelineClient());
+    }
     return mockContext;
   }
 
-
   private class TestParams {
     boolean isLastAMRetry;
     String workDir = setupTestWorkDir();
@@ -770,11 +781,14 @@ public class TestJobHistoryEventHandler {
     AppContext mockAppContext;
 
     public TestParams() {
-      this(false);
+      this(AppContext.class, false);
     }
     public TestParams(boolean isLastAMRetry) {
+      this(AppContext.class, isLastAMRetry);
+    }
+    public TestParams(Class<? extends AppContext> contextClass, boolean isLastAMRetry) {
       this.isLastAMRetry = isLastAMRetry;
-      mockAppContext = mockAppContext(appId, this.isLastAMRetry);
+      mockAppContext = mockAppContext(contextClass, appId, this.isLastAMRetry);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54a529df/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
index a2aaa9d..6defe5f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.ClusterInfo;
 import org.apache.hadoop.mapreduce.v2.app.MRApp;
+import org.apache.hadoop.mapreduce.v2.app.MRAppMaster.RunningAppContext;
 import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
@@ -1894,7 +1895,7 @@ public class TestRMContainerAllocator {
     private AllocateResponse allocateResponse;
     private static AppContext createAppContext(
         ApplicationAttemptId appAttemptId, Job job) {
-      AppContext context = mock(AppContext.class);
+      AppContext context = mock(RunningAppContext.class);
       ApplicationId appId = appAttemptId.getApplicationId();
       when(context.getApplicationID()).thenReturn(appId);
       when(context.getApplicationAttemptId()).thenReturn(appAttemptId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54a529df/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 74f58f6..7636317 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -182,6 +182,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4178. [storage implementation] app id as string in row keys can cause
     incorrect ordering (Varun Saxena via sjlee)
 
+    YARN-4297. TestJobHistoryEventHandler and TestRMContainerAllocator failing
+    on YARN-2928 branch (Varun Saxena via sjlee)
+
 Trunk - Unreleased
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/54a529df/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 851ac30..0977ed9 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
@@ -104,8 +104,6 @@ public class NodeTimelineCollectorManager extends TimelineCollectorManager {
     String bindAddress = conf.get(YarnConfiguration.TIMELINE_SERVICE_BIND_HOST,
         YarnConfiguration.DEFAULT_TIMELINE_SERVICE_BIND_HOST) + ":0";
     try {
-      Configuration confForInfoServer = new Configuration(conf);
-      confForInfoServer.setInt(HttpServer2.HTTP_MAX_THREADS, 10);
       HttpServer2.Builder builder = new HttpServer2.Builder()
           .setName("timeline")
           .setConf(conf)


[30/50] [abbrv] hadoop git commit: YARN-4129. Refactor the SystemMetricPublisher in RM to better support newer events (Naganarasimha G R via sjlee)

Posted by gt...@apache.org.
YARN-4129. Refactor the SystemMetricPublisher in RM to better support newer events (Naganarasimha G R via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: 1c54f762c1a336ca525aed623e461609bd1759a7
Parents: d2ae82d
Author: Sangjin Lee <sj...@apache.org>
Authored: Thu Oct 22 17:56:32 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:59:38 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../dev-support/findbugs-exclude.xml            |   4 +-
 .../server/resourcemanager/ResourceManager.java |  33 +-
 .../metrics/AbstractSystemMetricsPublisher.java | 168 ++++++++
 .../AbstractTimelineServicePublisher.java       | 184 ---------
 .../metrics/AppAttemptFinishedEvent.java        |  90 -----
 .../metrics/AppAttemptRegisteredEvent.java      |  81 ----
 .../metrics/ApplicationACLsUpdatedEvent.java    |  45 ---
 .../metrics/ApplicationCreatedEvent.java        | 124 ------
 .../metrics/ApplicationFinishedEvent.java       |  91 -----
 .../metrics/ApplicationUpdatedEvent.java        |  54 ---
 .../metrics/ContainerCreatedEvent.java          |  73 ----
 .../metrics/ContainerFinishedEvent.java         |  65 ----
 .../metrics/NoOpSystemMetricPublisher.java      |  65 ++++
 .../metrics/SystemMetricsEvent.java             |  33 --
 .../metrics/SystemMetricsEventType.java         |  36 --
 .../metrics/SystemMetricsPublisher.java         | 242 +-----------
 .../metrics/TimelineServiceV1Publisher.java     | 296 +++++++-------
 .../metrics/TimelineServiceV2Publisher.java     | 389 +++++++++++--------
 .../metrics/TestSystemMetricsPublisher.java     |   5 +-
 .../TestSystemMetricsPublisherForV2.java        |  22 +-
 .../scheduler/fair/TestFSLeafQueue.java         |   1 -
 22 files changed, 676 insertions(+), 1428 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 938e9bd..5a91c8a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -135,6 +135,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3836. add equals and hashCode to TimelineEntity and other classes in
     the data model (Li Lu via sjlee)
 
+    YARN-4129. Refactor the SystemMetricPublisher in RM to better support
+    newer events (Naganarasimha G R via sjlee)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index 68ed723..bd6cca5 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -117,12 +117,12 @@
 
   <!-- Object cast is based on the event type -->
   <Match>
-    <Class name="org.apache.hadoop.yarn.server.resourcemanager.metrics.AbstractTimelineServicePublisher" />
+    <Class name="org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher$ApplicationEventHandler" />
      <Bug pattern="BC_UNCONFIRMED_CAST" />
   </Match>
 
   <Match>
-    <Class name="org.apache.hadoop.yarn.server.nodemanager.timelineservice.NMTimelinePublisher$ApplicationEventHandler" />
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV2Publisher$TimelineV2EventHandler" />
      <Bug pattern="BC_UNCONFIRMED_CAST" />
   </Match>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/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 8f21309..e939ba9 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
@@ -64,7 +64,10 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
+import org.apache.hadoop.yarn.server.resourcemanager.metrics.NoOpSystemMetricPublisher;
 import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
+import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV1Publisher;
+import org.apache.hadoop.yarn.server.resourcemanager.metrics.TimelineServiceV2Publisher;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy;
 import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
@@ -94,11 +97,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
 import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebApp;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.security.http.RMAuthenticationFilter;
 import org.apache.hadoop.yarn.server.security.http.RMAuthenticationFilterInitializer;
-import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager;
 import org.apache.hadoop.yarn.server.webproxy.AppReportFetcher;
 import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
 import org.apache.hadoop.yarn.server.webproxy.WebAppProxy;
@@ -274,8 +277,9 @@ public class ResourceManager extends CompositeService implements Recoverable {
     addService(rmApplicationHistoryWriter);
     rmContext.setRMApplicationHistoryWriter(rmApplicationHistoryWriter);
 
-    SystemMetricsPublisher systemMetricsPublisher = createSystemMetricsPublisher();
-    addService(systemMetricsPublisher);
+    SystemMetricsPublisher systemMetricsPublisher =
+        createSystemMetricsPublisher();
+    addIfService(systemMetricsPublisher);
     rmContext.setSystemMetricsPublisher(systemMetricsPublisher);
 
     super.serviceInit(this.conf);
@@ -379,7 +383,24 @@ public class ResourceManager extends CompositeService implements Recoverable {
   }
 
   protected SystemMetricsPublisher createSystemMetricsPublisher() {
-    return new SystemMetricsPublisher(rmContext);
+    boolean timelineServiceEnabled =
+        conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
+            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED);
+    SystemMetricsPublisher publisher = null;
+    if (timelineServiceEnabled) {
+      if (conf.getBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
+          YarnConfiguration.DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_ENABLED)) {
+        LOG.info("TimelineService V1 is configured");
+        publisher = new TimelineServiceV1Publisher();
+      } else {
+        LOG.info("TimelineService V2 is configured");
+        publisher = new TimelineServiceV2Publisher(rmContext);
+      }
+    } else {
+      LOG.info("TimelineServicePublisher is not configured");
+      publisher = new NoOpSystemMetricPublisher();
+    }
+    return publisher;
   }
 
   // sanity check for configurations
@@ -502,10 +523,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
       addService(rmApplicationHistoryWriter);
       rmContext.setRMApplicationHistoryWriter(rmApplicationHistoryWriter);
 
-      SystemMetricsPublisher systemMetricsPublisher = createSystemMetricsPublisher();
-      addService(systemMetricsPublisher);
-      rmContext.setSystemMetricsPublisher(systemMetricsPublisher);
-
       RMTimelineCollectorManager timelineCollectorManager =
           createRMTimelineCollectorManager();
       addService(timelineCollectorManager);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AbstractSystemMetricsPublisher.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/AbstractSystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AbstractSystemMetricsPublisher.java
new file mode 100644
index 0000000..a8c00a4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AbstractSystemMetricsPublisher.java
@@ -0,0 +1,168 @@
+/**
+ * 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.metrics;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+
+public abstract class AbstractSystemMetricsPublisher extends CompositeService
+    implements SystemMetricsPublisher {
+  private MultiThreadedDispatcher dispatcher;
+
+  protected Dispatcher getDispatcher() {
+    return dispatcher;
+  }
+
+  public AbstractSystemMetricsPublisher(String name) {
+    super(name);
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    dispatcher =
+    new MultiThreadedDispatcher(getConfig().getInt(
+        YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE,
+        YarnConfiguration.DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE));
+    dispatcher.setDrainEventsOnStop();
+    addIfService(dispatcher);
+    super.serviceInit(conf);
+  }
+
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  public static class MultiThreadedDispatcher extends CompositeService
+      implements Dispatcher {
+
+    private List<AsyncDispatcher> dispatchers =
+        new ArrayList<AsyncDispatcher>();
+
+    public MultiThreadedDispatcher(int num) {
+      super(MultiThreadedDispatcher.class.getName());
+      for (int i = 0; i < num; ++i) {
+        AsyncDispatcher dispatcher = createDispatcher();
+        dispatchers.add(dispatcher);
+        addIfService(dispatcher);
+      }
+    }
+
+    @Override
+    public EventHandler getEventHandler() {
+      return new CompositEventHandler();
+    }
+
+    @Override
+    public void register(Class<? extends Enum> eventType,
+        EventHandler handler) {
+      for (AsyncDispatcher dispatcher : dispatchers) {
+        dispatcher.register(eventType, handler);
+      }
+    }
+
+    public void setDrainEventsOnStop() {
+      for (AsyncDispatcher dispatcher : dispatchers) {
+        dispatcher.setDrainEventsOnStop();
+      }
+    }
+
+    private class CompositEventHandler implements EventHandler<Event> {
+
+      @Override
+      public void handle(Event event) {
+        // Use hashCode (of ApplicationId) to dispatch the event to the child
+        // dispatcher, such that all the writing events of one application will
+        // be handled by one thread, the scheduled order of the these events
+        // will be preserved
+        int index = (event.hashCode() & Integer.MAX_VALUE) % dispatchers.size();
+        dispatchers.get(index).getEventHandler().handle(event);
+      }
+    }
+
+    protected AsyncDispatcher createDispatcher() {
+      return new AsyncDispatcher();
+    }
+  }
+
+  /**
+   * EventType which is used while publishing the events
+   */
+  protected static enum SystemMetricsEventType {
+    PUBLISH_ENTITY, PUBLISH_APPLICATION_FINISHED_ENTITY
+  }
+
+  /**
+   * TimelinePublishEvent's hash code should be based on application's id this
+   * will ensure all the events related to a particular app goes to particular
+   * thread of MultiThreaded dispatcher.
+   */
+  protected static abstract class TimelinePublishEvent
+      extends AbstractEvent<SystemMetricsEventType> {
+
+    private ApplicationId appId;
+
+    public TimelinePublishEvent(SystemMetricsEventType type,
+        ApplicationId appId) {
+      super(type);
+      this.appId = appId;
+    }
+
+    public ApplicationId getApplicationId() {
+      return appId;
+    }
+
+    @Override
+    public int hashCode() {
+      return appId.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (!(obj instanceof TimelinePublishEvent)) {
+        return false;
+      }
+      TimelinePublishEvent other = (TimelinePublishEvent) obj;
+      if (appId == null) {
+        if (other.appId != null) {
+          return false;
+        }
+      } else if (getType() == null) {
+        if (other.getType() != null) {
+          return false;
+        }
+      } else
+        if (!appId.equals(other.appId) || !getType().equals(other.getType())) {
+        return false;
+      }
+      return true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AbstractTimelineServicePublisher.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/AbstractTimelineServicePublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AbstractTimelineServicePublisher.java
deleted file mode 100644
index 12145267..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AbstractTimelineServicePublisher.java
+++ /dev/null
@@ -1,184 +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.resourcemanager.metrics;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.CompositeService;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.event.Dispatcher;
-import org.apache.hadoop.yarn.event.Event;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher.TimelineServicePublisher;
-
-public abstract class AbstractTimelineServicePublisher extends CompositeService
-    implements TimelineServicePublisher, EventHandler<SystemMetricsEvent> {
-
-  private static final Log LOG = LogFactory
-      .getLog(TimelineServiceV2Publisher.class);
-
-  private Configuration conf;
-
-  public AbstractTimelineServicePublisher(String name) {
-    super(name);
-  }
-
-  @Override
-  protected void serviceInit(Configuration conf) throws Exception {
-    this.conf = conf;
-    super.serviceInit(conf);
-  }
-
-  @Override
-  protected void serviceStart() throws Exception {
-    super.serviceStart();
-  }
-
-  @Override
-  protected void serviceStop() throws Exception {
-    super.serviceStop();
-  }
-
-  @Override
-  public void handle(SystemMetricsEvent event) {
-    switch (event.getType()) {
-    case APP_CREATED:
-      publishApplicationCreatedEvent((ApplicationCreatedEvent) event);
-      break;
-    case APP_FINISHED:
-      publishApplicationFinishedEvent((ApplicationFinishedEvent) event);
-      break;
-    case APP_UPDATED:
-      publishApplicationUpdatedEvent((ApplicationUpdatedEvent) event);
-      break;
-    case APP_ACLS_UPDATED:
-      publishApplicationACLsUpdatedEvent((ApplicationACLsUpdatedEvent) event);
-      break;
-    case APP_ATTEMPT_REGISTERED:
-      publishAppAttemptRegisteredEvent((AppAttemptRegisteredEvent) event);
-      break;
-    case APP_ATTEMPT_FINISHED:
-      publishAppAttemptFinishedEvent((AppAttemptFinishedEvent) event);
-      break;
-    case CONTAINER_CREATED:
-      publishContainerCreatedEvent((ContainerCreatedEvent) event);
-      break;
-    case CONTAINER_FINISHED:
-      publishContainerFinishedEvent((ContainerFinishedEvent) event);
-      break;
-    default:
-      LOG.error("Unknown SystemMetricsEvent type: " + event.getType());
-    }
-  }
-
-  abstract void publishAppAttemptFinishedEvent(AppAttemptFinishedEvent event);
-
-  abstract void publishAppAttemptRegisteredEvent(AppAttemptRegisteredEvent event);
-
-  abstract void publishApplicationUpdatedEvent(ApplicationUpdatedEvent event);
-
-  abstract void publishApplicationACLsUpdatedEvent(
-      ApplicationACLsUpdatedEvent event);
-
-  abstract void publishApplicationFinishedEvent(ApplicationFinishedEvent event);
-
-  abstract void publishApplicationCreatedEvent(ApplicationCreatedEvent event);
-
-  abstract void publishContainerCreatedEvent(ContainerCreatedEvent event);
-
-  abstract void publishContainerFinishedEvent(ContainerFinishedEvent event);
-
-  @Override
-  public Dispatcher getDispatcher() {
-    MultiThreadedDispatcher dispatcher =
-        new MultiThreadedDispatcher(
-            conf.getInt(
-                YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE,
-                YarnConfiguration.DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE));
-    dispatcher.setDrainEventsOnStop();
-    return dispatcher;
-  }
-
-  @Override
-  public boolean publishRMContainerMetrics() {
-    return true;
-  }
-
-  @Override
-  public EventHandler<SystemMetricsEvent> getEventHandler() {
-    return this;
-  }
-
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  public static class MultiThreadedDispatcher extends CompositeService
-      implements Dispatcher {
-
-    private List<AsyncDispatcher> dispatchers =
-        new ArrayList<AsyncDispatcher>();
-
-    public MultiThreadedDispatcher(int num) {
-      super(MultiThreadedDispatcher.class.getName());
-      for (int i = 0; i < num; ++i) {
-        AsyncDispatcher dispatcher = createDispatcher();
-        dispatchers.add(dispatcher);
-        addIfService(dispatcher);
-      }
-    }
-
-    @Override
-    public EventHandler getEventHandler() {
-      return new CompositEventHandler();
-    }
-
-    @Override
-    public void register(Class<? extends Enum> eventType, EventHandler handler) {
-      for (AsyncDispatcher dispatcher : dispatchers) {
-        dispatcher.register(eventType, handler);
-      }
-    }
-
-    public void setDrainEventsOnStop() {
-      for (AsyncDispatcher dispatcher : dispatchers) {
-        dispatcher.setDrainEventsOnStop();
-      }
-    }
-
-    private class CompositEventHandler implements EventHandler<Event> {
-
-      @Override
-      public void handle(Event event) {
-        // Use hashCode (of ApplicationId) to dispatch the event to the child
-        // dispatcher, such that all the writing events of one application will
-        // be handled by one thread, the scheduled order of the these events
-        // will be preserved
-        int index = (event.hashCode() & Integer.MAX_VALUE) % dispatchers.size();
-        dispatchers.get(index).getEventHandler().handle(event);
-      }
-    }
-
-    protected AsyncDispatcher createDispatcher() {
-      return new AsyncDispatcher();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.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/AppAttemptFinishedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java
deleted file mode 100644
index fc1d10f..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptFinishedEvent.java
+++ /dev/null
@@ -1,90 +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.resourcemanager.metrics;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
-
-public class AppAttemptFinishedEvent extends
-    SystemMetricsEvent {
-
-  private ApplicationAttemptId appAttemptId;
-  private String trackingUrl;
-  private String originalTrackingUrl;
-  private String diagnosticsInfo;
-  private FinalApplicationStatus appStatus;
-  private YarnApplicationAttemptState state;
-  private ContainerId masterContainerId;
-
-  public AppAttemptFinishedEvent(
-      ApplicationAttemptId appAttemptId,
-      String trackingUrl,
-      String originalTrackingUrl,
-      String diagnosticsInfo,
-      FinalApplicationStatus appStatus,
-      YarnApplicationAttemptState state,
-      long finishedTime,
-      ContainerId masterContainerId) {
-    super(SystemMetricsEventType.APP_ATTEMPT_FINISHED, finishedTime);
-    this.appAttemptId = appAttemptId;
-    // This is the tracking URL after the application attempt is finished
-    this.trackingUrl = trackingUrl;
-    this.originalTrackingUrl = originalTrackingUrl;
-    this.diagnosticsInfo = diagnosticsInfo;
-    this.appStatus = appStatus;
-    this.state = state;
-    this.masterContainerId = masterContainerId;
-  }
-
-  @Override
-  public int hashCode() {
-    return appAttemptId.getApplicationId().hashCode();
-  }
-
-  public ApplicationAttemptId getApplicationAttemptId() {
-    return appAttemptId;
-  }
-
-  public String getTrackingUrl() {
-    return trackingUrl;
-  }
-
-  public String getOriginalTrackingURL() {
-    return originalTrackingUrl;
-  }
-
-  public String getDiagnosticsInfo() {
-    return diagnosticsInfo;
-  }
-
-  public FinalApplicationStatus getFinalApplicationStatus() {
-    return appStatus;
-  }
-
-  public YarnApplicationAttemptState getYarnApplicationAttemptState() {
-    return state;
-  }
-
-  public ContainerId getMasterContainerId() {
-    return masterContainerId;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptRegisteredEvent.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/AppAttemptRegisteredEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptRegisteredEvent.java
deleted file mode 100644
index 1d0f16d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/AppAttemptRegisteredEvent.java
+++ /dev/null
@@ -1,81 +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.resourcemanager.metrics;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-
-public class AppAttemptRegisteredEvent extends
-    SystemMetricsEvent {
-
-  private ApplicationAttemptId appAttemptId;
-  private String host;
-  private int rpcPort;
-  private String trackingUrl;
-  private String originalTrackingUrl;
-  private ContainerId masterContainerId;
-
-  public AppAttemptRegisteredEvent(
-      ApplicationAttemptId appAttemptId,
-      String host,
-      int rpcPort,
-      String trackingUrl,
-      String originalTrackingUrl,
-      ContainerId masterContainerId,
-      long registeredTime) {
-    super(SystemMetricsEventType.APP_ATTEMPT_REGISTERED, registeredTime);
-    this.appAttemptId = appAttemptId;
-    this.host = host;
-    this.rpcPort = rpcPort;
-    // This is the tracking URL after the application attempt is registered
-    this.trackingUrl = trackingUrl;
-    this.originalTrackingUrl = originalTrackingUrl;
-    this.masterContainerId = masterContainerId;
-  }
-
-  @Override
-  public int hashCode() {
-    return appAttemptId.getApplicationId().hashCode();
-  }
-
-  public ApplicationAttemptId getApplicationAttemptId() {
-    return appAttemptId;
-  }
-
-  public String getHost() {
-    return host;
-  }
-
-  public int getRpcPort() {
-    return rpcPort;
-  }
-
-  public String getTrackingUrl() {
-    return trackingUrl;
-  }
-
-  public String getOriginalTrackingURL() {
-    return originalTrackingUrl;
-  }
-
-  public ContainerId getMasterContainerId() {
-    return masterContainerId;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationACLsUpdatedEvent.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/ApplicationACLsUpdatedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationACLsUpdatedEvent.java
deleted file mode 100644
index c8b314c..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationACLsUpdatedEvent.java
+++ /dev/null
@@ -1,45 +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.resourcemanager.metrics;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-
-
-public class ApplicationACLsUpdatedEvent extends SystemMetricsEvent {
-
-  private ApplicationId appId;
-  private String viewAppACLs;
-
-  public ApplicationACLsUpdatedEvent(ApplicationId appId,
-      String viewAppACLs,
-      long updatedTime) {
-    super(SystemMetricsEventType.APP_ACLS_UPDATED, updatedTime);
-    this.appId = appId;
-    this.viewAppACLs = viewAppACLs;
-  }
-
-  public ApplicationId getApplicationId() {
-    return appId;
-  }
-
-  public String getViewAppACLs() {
-    return viewAppACLs;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationCreatedEvent.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/ApplicationCreatedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationCreatedEvent.java
deleted file mode 100644
index 968a8fd..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationCreatedEvent.java
+++ /dev/null
@@ -1,124 +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.resourcemanager.metrics;
-
-import java.util.Set;
-
-import org.apache.hadoop.ipc.CallerContext;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Priority;
-
-public class ApplicationCreatedEvent extends
-    SystemMetricsEvent {
-
-  private ApplicationId appId;
-  private String name;
-  private String type;
-  private String user;
-  private String queue;
-  private long submittedTime;
-  private Set<String> appTags;
-  private boolean unmanagedApplication;
-  private Priority applicationPriority;
-  private String appNodeLabelsExpression;
-  private String amNodeLabelsExpression;
-  private final CallerContext callerContext;
-
-
-  public ApplicationCreatedEvent(ApplicationId appId,
-      String name,
-      String type,
-      String user,
-      String queue,
-      long submittedTime,
-      long createdTime,
-      Set<String> appTags,
-      boolean unmanagedApplication,
-      Priority applicationPriority,
-      String appNodeLabelsExpression,
-      String amNodeLabelsExpression,
-      CallerContext callerContext) {
-    super(SystemMetricsEventType.APP_CREATED, createdTime);
-    this.appId = appId;
-    this.name = name;
-    this.type = type;
-    this.user = user;
-    this.queue = queue;
-    this.submittedTime = submittedTime;
-    this.appTags = appTags;
-    this.unmanagedApplication = unmanagedApplication;
-    this.applicationPriority = applicationPriority;
-    this.appNodeLabelsExpression = appNodeLabelsExpression;
-    this.amNodeLabelsExpression = amNodeLabelsExpression;
-    this.callerContext = callerContext;
-  }
-
-  @Override
-  public int hashCode() {
-    return appId.hashCode();
-  }
-
-  public ApplicationId getApplicationId() {
-    return appId;
-  }
-
-  public String getApplicationName() {
-    return name;
-  }
-
-  public String getApplicationType() {
-    return type;
-  }
-
-  public String getUser() {
-    return user;
-  }
-
-  public String getQueue() {
-    return queue;
-  }
-
-  public long getSubmittedTime() {
-    return submittedTime;
-  }
-
-  public Set<String> getAppTags() {
-    return appTags;
-  }
-
-  public boolean isUnmanagedApp() {
-    return unmanagedApplication;
-  }
-
-  public Priority getApplicationPriority() {
-    return applicationPriority;
-  }
-
-  public String getAppNodeLabelsExpression() {
-    return appNodeLabelsExpression;
-  }
-
-  public String getAmNodeLabelsExpression() {
-    return amNodeLabelsExpression;
-  }
-  
-  public CallerContext getCallerContext() {
-    return callerContext;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationFinishedEvent.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/ApplicationFinishedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationFinishedEvent.java
deleted file mode 100644
index d9241b2..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationFinishedEvent.java
+++ /dev/null
@@ -1,91 +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.resourcemanager.metrics;
-
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
-
-public class ApplicationFinishedEvent extends
-    SystemMetricsEvent {
-
-  private ApplicationId appId;;
-  private String diagnosticsInfo;
-  private FinalApplicationStatus appStatus;
-  private YarnApplicationState state;
-  private ApplicationAttemptId latestAppAttemptId;
-  private RMAppMetrics appMetrics;
-  private RMAppImpl app;
-
-  public ApplicationFinishedEvent(
-      ApplicationId appId,
-      String diagnosticsInfo,
-      FinalApplicationStatus appStatus,
-      YarnApplicationState state,
-      ApplicationAttemptId latestAppAttemptId,
-      long finishedTime,
-      RMAppMetrics appMetrics,
-      RMAppImpl app) {
-    super(SystemMetricsEventType.APP_FINISHED, finishedTime);
-    this.appId = appId;
-    this.diagnosticsInfo = diagnosticsInfo;
-    this.appStatus = appStatus;
-    this.latestAppAttemptId = latestAppAttemptId;
-    this.state = state;
-    this.appMetrics = appMetrics;
-    this.app = app;
-  }
-
-  @Override
-  public int hashCode() {
-    return appId.hashCode();
-  }
-
-  public RMAppImpl getApp() {
-    return app;
-  }
-
-  public ApplicationId getApplicationId() {
-    return appId;
-  }
-
-  public String getDiagnosticsInfo() {
-    return diagnosticsInfo;
-  }
-
-  public FinalApplicationStatus getFinalApplicationStatus() {
-    return appStatus;
-  }
-
-  public YarnApplicationState getYarnApplicationState() {
-    return state;
-  }
-
-  public ApplicationAttemptId getLatestApplicationAttemptId() {
-    return latestAppAttemptId;
-  }
-
-  public RMAppMetrics getAppMetrics() {
-    return appMetrics;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationUpdatedEvent.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/ApplicationUpdatedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationUpdatedEvent.java
deleted file mode 100644
index 9e5e1fd..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ApplicationUpdatedEvent.java
+++ /dev/null
@@ -1,54 +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.resourcemanager.metrics;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Priority;
-
-public class ApplicationUpdatedEvent extends SystemMetricsEvent {
-
-  private ApplicationId appId;
-  private String queue;
-  private Priority applicationPriority;
-
-  public ApplicationUpdatedEvent(ApplicationId appId, String queue,
-      long updatedTime, Priority applicationPriority) {
-    super(SystemMetricsEventType.APP_UPDATED, updatedTime);
-    this.appId = appId;
-    this.queue = queue;
-    this.applicationPriority = applicationPriority;
-  }
-
-  @Override
-  public int hashCode() {
-    return appId.hashCode();
-  }
-
-  public ApplicationId getApplicationId() {
-    return appId;
-  }
-
-  public String getQueue() {
-    return queue;
-  }
-
-  public Priority getApplicationPriority() {
-    return applicationPriority;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.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/ContainerCreatedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java
deleted file mode 100644
index 05b6781..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerCreatedEvent.java
+++ /dev/null
@@ -1,73 +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.resourcemanager.metrics;
-
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-
-public class ContainerCreatedEvent extends SystemMetricsEvent {
-
-  private ContainerId containerId;
-  private Resource allocatedResource;
-  private NodeId allocatedNode;
-  private Priority allocatedPriority;
-  private String nodeHttpAddress;
-
-  public ContainerCreatedEvent(
-      ContainerId containerId,
-      Resource allocatedResource,
-      NodeId allocatedNode,
-      Priority allocatedPriority,
-      long createdTime,
-      String nodeHttpAddress) {
-    super(SystemMetricsEventType.CONTAINER_CREATED, createdTime);
-    this.containerId = containerId;
-    this.allocatedResource = allocatedResource;
-    this.allocatedNode = allocatedNode;
-    this.allocatedPriority = allocatedPriority;
-    this.nodeHttpAddress = nodeHttpAddress;
-  }
-
-  @Override
-  public int hashCode() {
-    return containerId.getApplicationAttemptId().getApplicationId().hashCode();
-  }
-
-  public ContainerId getContainerId() {
-    return containerId;
-  }
-
-  public Resource getAllocatedResource() {
-    return allocatedResource;
-  }
-
-  public NodeId getAllocatedNode() {
-    return allocatedNode;
-  }
-
-  public Priority getAllocatedPriority() {
-    return allocatedPriority;
-  }
-
-  public String getNodeHttpAddress() {
-    return nodeHttpAddress;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerFinishedEvent.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/ContainerFinishedEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerFinishedEvent.java
deleted file mode 100644
index aafd760..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/ContainerFinishedEvent.java
+++ /dev/null
@@ -1,65 +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.resourcemanager.metrics;
-
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerState;
-
-public class ContainerFinishedEvent extends SystemMetricsEvent {
-
-  private ContainerId containerId;
-  private String diagnosticsInfo;
-  private int containerExitStatus;
-  private ContainerState state;
-
-  public ContainerFinishedEvent(
-      ContainerId containerId,
-      String diagnosticsInfo,
-      int containerExitStatus,
-      ContainerState state,
-      long finishedTime) {
-    super(SystemMetricsEventType.CONTAINER_FINISHED, finishedTime);
-    this.containerId = containerId;
-    this.diagnosticsInfo = diagnosticsInfo;
-    this.containerExitStatus = containerExitStatus;
-    this.state = state;
-  }
-
-  @Override
-  public int hashCode() {
-    return containerId.getApplicationAttemptId().getApplicationId().hashCode();
-  }
-
-  public ContainerId getContainerId() {
-    return containerId;
-  }
-
-  public String getDiagnosticsInfo() {
-    return diagnosticsInfo;
-  }
-
-  public int getContainerExitStatus() {
-    return containerExitStatus;
-  }
-
-  public ContainerState getContainerState() {
-    return state;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/NoOpSystemMetricPublisher.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/NoOpSystemMetricPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/NoOpSystemMetricPublisher.java
new file mode 100644
index 0000000..1810df1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/NoOpSystemMetricPublisher.java
@@ -0,0 +1,65 @@
+/**
+ * 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.metrics;
+
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
+import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
+
+/**
+ * This class does nothing when any of the methods are invoked on
+ * SystemMetricsPublisher
+ */
+public class NoOpSystemMetricPublisher implements SystemMetricsPublisher{
+
+  @Override
+  public void appCreated(RMApp app, long createdTime) {
+  }
+
+  @Override
+  public void appFinished(RMApp app, RMAppState state, long finishedTime) {
+  }
+
+  @Override
+  public void appACLsUpdated(RMApp app, String appViewACLs, long updatedTime) {
+  }
+
+  @Override
+  public void appAttemptRegistered(RMAppAttempt appAttempt,
+      long registeredTime) {
+  }
+
+  @Override
+  public void appAttemptFinished(RMAppAttempt appAttempt,
+      RMAppAttemptState appAttemtpState, RMApp app, long finishedTime) {
+  }
+
+  @Override
+  public void containerCreated(RMContainer container, long createdTime) {
+  }
+
+  @Override
+  public void containerFinished(RMContainer container, long finishedTime) {
+  }
+
+  @Override
+  public void appUpdated(RMApp app, long currentTimeMillis) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsEvent.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/SystemMetricsEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsEvent.java
deleted file mode 100644
index 1847396..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsEvent.java
+++ /dev/null
@@ -1,33 +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.resourcemanager.metrics;
-
-import org.apache.hadoop.yarn.event.AbstractEvent;
-
-public class SystemMetricsEvent extends AbstractEvent<SystemMetricsEventType> {
-
-  public SystemMetricsEvent(SystemMetricsEventType type) {
-    super(type);
-  }
-
-  public SystemMetricsEvent(SystemMetricsEventType type, long timestamp) {
-    super(type, timestamp);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsEventType.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/SystemMetricsEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsEventType.java
deleted file mode 100644
index c11034e..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsEventType.java
+++ /dev/null
@@ -1,36 +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.resourcemanager.metrics;
-
-
-public enum SystemMetricsEventType {
-  // app events
-  APP_CREATED,
-  APP_FINISHED,
-  APP_ACLS_UPDATED,
-  APP_UPDATED,
-
-  // app attempt events
-  APP_ATTEMPT_REGISTERED,
-  APP_ATTEMPT_FINISHED,
-
-  // container events
-  CONTAINER_CREATED,
-  CONTAINER_FINISHED
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c54f762/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.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/SystemMetricsPublisher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java
index 3eb0763..f895bba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/SystemMetricsPublisher.java
@@ -18,250 +18,28 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.metrics;
 
-
-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.service.CompositeService;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.service.Service;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.Dispatcher;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * The class that helps RM publish metrics to the timeline server. RM will
- * always invoke the methods of this class regardless the service is enabled or
- * not. If it is disabled, publishing requests will be ignored silently.
- */
-@Private
-@Unstable
-public class SystemMetricsPublisher extends CompositeService {
-
-  private static final Log LOG = LogFactory
-      .getLog(SystemMetricsPublisher.class);
-
-  private Dispatcher dispatcher;
-  private boolean publishSystemMetrics;
-  private boolean publishContainerMetrics;
-  protected RMContext rmContext;
-
-  public SystemMetricsPublisher(RMContext rmContext) {
-    super(SystemMetricsPublisher.class.getName());
-    this.rmContext = rmContext;
-  }
-
-  @Override
-  protected void serviceInit(Configuration conf) throws Exception {
-    publishSystemMetrics =
-        conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
-            YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED);
-    if (publishSystemMetrics) {
-      TimelineServicePublisher timelineServicePublisher =
-          getTimelineServicePublisher(conf);
-      if (timelineServicePublisher != null) {
-        addService(timelineServicePublisher);
-        // init required to be called so that other methods of
-        // TimelineServicePublisher can be utilized
-        timelineServicePublisher.init(conf);
-        dispatcher = createDispatcher(timelineServicePublisher);
-        publishContainerMetrics =
-            timelineServicePublisher.publishRMContainerMetrics();
-        dispatcher.register(SystemMetricsEventType.class,
-            timelineServicePublisher.getEventHandler());
-        addIfService(dispatcher);
-      } else {
-        LOG.info("TimelineServicePublisher is not configured");
-        publishSystemMetrics = false;
-      }
-      LOG.info("YARN system metrics publishing service is enabled");
-    } else {
-      LOG.info("YARN system metrics publishing service is not enabled");
-    }
-    super.serviceInit(conf);
-  }
-
-  @VisibleForTesting
-  Dispatcher createDispatcher(TimelineServicePublisher timelineServicePublisher) {
-    return timelineServicePublisher.getDispatcher();
-  }
-
-  TimelineServicePublisher getTimelineServicePublisher(Configuration conf) {
-    if (conf.getBoolean(YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_ENABLED,
-        YarnConfiguration.DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_ENABLED)) {
-      return new TimelineServiceV1Publisher();
-    } else if (conf.getBoolean(
-        YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED,
-        YarnConfiguration.DEFAULT_SYSTEM_METRICS_PUBLISHER_ENABLED)) {
-      return new TimelineServiceV2Publisher(rmContext);
-    }
-    return null;
-  }
-
-  @SuppressWarnings("unchecked")
-  public void appCreated(RMApp app, long createdTime) {
-    if (publishSystemMetrics) {
-      ApplicationSubmissionContext appSubmissionContext =
-          app.getApplicationSubmissionContext();
-      dispatcher.getEventHandler().handle(
-          new ApplicationCreatedEvent(
-              app.getApplicationId(),
-              app.getName(),
-              app.getApplicationType(),
-              app.getUser(),
-              app.getQueue(),
-              app.getSubmitTime(),
-              createdTime, app.getApplicationTags(),
-              appSubmissionContext.getUnmanagedAM(),
-              appSubmissionContext.getPriority(),
-              app.getAppNodeLabelExpression(),
-              app.getAmNodeLabelExpression(),
-              app.getCallerContext()));
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  public void appUpdated(RMApp app, long updatedTime) {
-    if (publishSystemMetrics) {
-      dispatcher.getEventHandler()
-          .handle(new ApplicationUpdatedEvent(app.getApplicationId(),
-              app.getQueue(), updatedTime,
-              app.getApplicationSubmissionContext().getPriority()));
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  public void appFinished(RMApp app, RMAppState state, long finishedTime) {
-    if (publishSystemMetrics) {
-      dispatcher.getEventHandler().handle(
-          new ApplicationFinishedEvent(
-              app.getApplicationId(),
-              app.getDiagnostics().toString(),
-              app.getFinalApplicationStatus(),
-              RMServerUtils.createApplicationState(state),
-              app.getCurrentAppAttempt() == null ?
-                  null : app.getCurrentAppAttempt().getAppAttemptId(),
-              finishedTime,
-              app.getRMAppMetrics(),
-              (RMAppImpl)app));
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  public void appACLsUpdated(RMApp app, String appViewACLs,
-      long updatedTime) {
-    if (publishSystemMetrics) {
-      dispatcher.getEventHandler().handle(
-          new ApplicationACLsUpdatedEvent(
-              app.getApplicationId(),
-              appViewACLs == null ? "" : appViewACLs,
-              updatedTime));
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  public void appAttemptRegistered(RMAppAttempt appAttempt,
-      long registeredTime) {
-    if (publishSystemMetrics) {
-      ContainerId container = (appAttempt.getMasterContainer() == null) ? null
-          : appAttempt.getMasterContainer().getId();
-      dispatcher.getEventHandler().handle(
-          new AppAttemptRegisteredEvent(
-              appAttempt.getAppAttemptId(),
-              appAttempt.getHost(),
-              appAttempt.getRpcPort(),
-              appAttempt.getTrackingUrl(),
-              appAttempt.getOriginalTrackingUrl(),
-              container,
-              registeredTime));
-    }
-  }
+public interface SystemMetricsPublisher {
 
-  @SuppressWarnings("unchecked")
-  public void appAttemptFinished(RMAppAttempt appAttempt,
-      RMAppAttemptState appAttemtpState, RMApp app, long finishedTime) {
-    if (publishSystemMetrics) {
-      ContainerId container = (appAttempt.getMasterContainer() == null) ? null
-          : appAttempt.getMasterContainer().getId();
-      dispatcher.getEventHandler().handle(
-          new AppAttemptFinishedEvent(
-              appAttempt.getAppAttemptId(),
-              appAttempt.getTrackingUrl(),
-              appAttempt.getOriginalTrackingUrl(),
-              appAttempt.getDiagnostics(),
-              // app will get the final status from app attempt, or create one
-              // based on app state if it doesn't exist
-              app.getFinalApplicationStatus(),
-              RMServerUtils.createApplicationAttemptState(appAttemtpState),
-              finishedTime,
-              container));
-    }
-  }
+  void appCreated(RMApp app, long createdTime);
 
-  @SuppressWarnings("unchecked")
-  public void containerCreated(RMContainer container, long createdTime) {
-    if (publishContainerMetrics) {
-      dispatcher.getEventHandler().handle(
-          new ContainerCreatedEvent(
-              container.getContainerId(),
-              container.getAllocatedResource(),
-              container.getAllocatedNode(),
-              container.getAllocatedPriority(),
-              createdTime, container.getNodeHttpAddress()));
-    }
-  }
+  void appACLsUpdated(RMApp app, String appViewACLs, long updatedTime);
 
-  @SuppressWarnings("unchecked")
-  public void containerFinished(RMContainer container, long finishedTime) {
-    if (publishContainerMetrics) {
-      dispatcher.getEventHandler().handle(
-          new ContainerFinishedEvent(
-              container.getContainerId(),
-              container.getDiagnosticsInfo(),
-              container.getContainerExitStatus(),
-              container.getContainerState(),
-              finishedTime));
-    }
-  }
+  void appUpdated(RMApp app, long updatedTime);
 
-  @VisibleForTesting
-  boolean isPublishContainerMetrics() {
-    return publishContainerMetrics;
-  }
+  void appFinished(RMApp app, RMAppState state, long finishedTime);
 
-  @VisibleForTesting
-  Dispatcher getDispatcher() {
-    return dispatcher;
-  }
+  void appAttemptRegistered(RMAppAttempt appAttempt, long registeredTime);
 
-  interface TimelineServicePublisher extends Service {
-    /**
-     * @return the Dispatcher which needs to be used to dispatch events
-     */
-    Dispatcher getDispatcher();
+  void appAttemptFinished(RMAppAttempt appAttempt,
+      RMAppAttemptState appAttemtpState, RMApp app, long finishedTime);
 
-    /**
-     * @return true if RMContainerMetricsNeeds to be sent
-     */
-    boolean publishRMContainerMetrics();
+  void containerCreated(RMContainer container, long createdTime);
 
-    /**
-     * @return EventHandler which needs to be registered to the dispatcher to
-     *         handle the SystemMetricsEvent
-     */
-    EventHandler<SystemMetricsEvent> getEventHandler();
-  }
+  void containerFinished(RMContainer container, long finishedTime);
 }


[08/50] [abbrv] hadoop git commit: YARN-3906. Split the application table from the entity table. Contributed by Sangjin Lee.

Posted by gt...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/30076229/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.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/TestHBaseTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
index ab02779..95f88d1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -47,6 +48,10 @@ 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.storage.application.ApplicationColumn;
+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.AppToFlowTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
@@ -60,7 +65,15 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
- * @throws Exception
+ * Various tests to test writing entities to HBase and reading them back from
+ * it.
+ *
+ * It uses a single HBase mini-cluster for all tests which is a little more
+ * realistic, and helps test correctness in the presence of other data.
+ *
+ * Each test uses a different cluster name to be able to handle its own data
+ * even if other records exist in the table. Use a different cluster name if
+ * you add a new test.
  */
 public class TestHBaseTimelineWriterImpl {
 
@@ -78,6 +91,199 @@ public class TestHBaseTimelineWriterImpl {
         .createTable(util.getHBaseAdmin(), util.getConfiguration());
     new AppToFlowTable()
         .createTable(util.getHBaseAdmin(), util.getConfiguration());
+    new ApplicationTable()
+        .createTable(util.getHBaseAdmin(), util.getConfiguration());
+  }
+
+  @Test
+  public void testWriteApplicationToHBase() throws Exception {
+    TimelineEntities te = new TimelineEntities();
+    ApplicationEntity entity = new ApplicationEntity();
+    String id = "hello";
+    entity.setId(id);
+    Long cTime = 1425016501000L;
+    Long mTime = 1425026901000L;
+    entity.setCreatedTime(cTime);
+    entity.setModifiedTime(mTime);
+
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<String, Object>();
+    infoMap.put("infoMapKey1", "infoMapValue1");
+    infoMap.put("infoMapKey2", 10);
+    entity.addInfo(infoMap);
+
+    // add the isRelatedToEntity info
+    String key = "task";
+    String value = "is_related_to_entity_id_here";
+    Set<String> isRelatedToSet = new HashSet<String>();
+    isRelatedToSet.add(value);
+    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
+    isRelatedTo.put(key, isRelatedToSet);
+    entity.setIsRelatedToEntities(isRelatedTo);
+
+    // add the relatesTo info
+    key = "container";
+    value = "relates_to_entity_id_here";
+    Set<String> relatesToSet = new HashSet<String>();
+    relatesToSet.add(value);
+    value = "relates_to_entity_id_here_Second";
+    relatesToSet.add(value);
+    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
+    relatesTo.put(key, relatesToSet);
+    entity.setRelatesToEntities(relatesTo);
+
+    // add some config entries
+    Map<String, String> conf = new HashMap<String, String>();
+    conf.put("config_param1", "value1");
+    conf.put("config_param2", "value2");
+    entity.addConfigs(conf);
+
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    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);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    entity.addMetrics(metrics);
+
+    te.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+      String cluster = "cluster_test_write_app";
+      String user = "user1";
+      String flow = "some_flow_name";
+      String flowVersion = "AB7822C10F1111";
+      long runid = 1002345678919L;
+      hbi.write(cluster, user, flow, flowVersion, runid, id, te);
+      hbi.stop();
+
+      // retrieve the row
+      byte[] rowKey =
+          ApplicationRowKey.getRowKey(cluster, user, flow, runid, id);
+      Get get = new Get(rowKey);
+      get.setMaxVersions(Integer.MAX_VALUE);
+      Connection conn = ConnectionFactory.createConnection(c1);
+      Result result = new ApplicationTable().getResult(c1, conn, get);
+
+      assertTrue(result != null);
+      assertEquals(16, result.size());
+
+      // check the row key
+      byte[] row1 = result.getRow();
+      assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
+          id));
+
+      // check info column family
+      String id1 = ApplicationColumn.ID.readResult(result).toString();
+      assertEquals(id, id1);
+
+      Number val =
+          (Number) ApplicationColumn.CREATED_TIME.readResult(result);
+      Long cTime1 = val.longValue();
+      assertEquals(cTime1, cTime);
+
+      val = (Number) ApplicationColumn.MODIFIED_TIME.readResult(result);
+      Long mTime1 = val.longValue();
+      assertEquals(mTime1, mTime);
+
+      Map<String, Object> infoColumns =
+          ApplicationColumnPrefix.INFO.readResults(result);
+      assertEquals(infoMap.size(), infoColumns.size());
+      for (String infoItem : infoMap.keySet()) {
+        assertEquals(infoMap.get(infoItem), infoColumns.get(infoItem));
+      }
+
+      // Remember isRelatedTo is of type Map<String, Set<String>>
+      for (String isRelatedToKey : isRelatedTo.keySet()) {
+        Object isRelatedToValue =
+            ApplicationColumnPrefix.IS_RELATED_TO.readResult(result,
+                isRelatedToKey);
+        String compoundValue = isRelatedToValue.toString();
+        // id7?id9?id6
+        Set<String> isRelatedToValues =
+            new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
+        assertEquals(isRelatedTo.get(isRelatedToKey).size(),
+            isRelatedToValues.size());
+        for (String v : isRelatedTo.get(isRelatedToKey)) {
+          assertTrue(isRelatedToValues.contains(v));
+        }
+      }
+
+      // RelatesTo
+      for (String relatesToKey : relatesTo.keySet()) {
+        String compoundValue =
+            ApplicationColumnPrefix.RELATES_TO.readResult(result,
+                relatesToKey).toString();
+        // id3?id4?id5
+        Set<String> relatesToValues =
+            new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
+        assertEquals(relatesTo.get(relatesToKey).size(),
+            relatesToValues.size());
+        for (String v : relatesTo.get(relatesToKey)) {
+          assertTrue(relatesToValues.contains(v));
+        }
+      }
+
+      // Configuration
+      Map<String, Object> configColumns =
+          ApplicationColumnPrefix.CONFIG.readResults(result);
+      assertEquals(conf.size(), configColumns.size());
+      for (String configItem : conf.keySet()) {
+        assertEquals(conf.get(configItem), configColumns.get(configItem));
+      }
+
+      NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
+          ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result);
+
+      NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
+      // We got metrics back
+      assertNotNull(metricMap);
+      // Same number of metrics as we wrote
+      assertEquals(metricValues.entrySet().size(), metricMap.entrySet().size());
+
+      // Iterate over original metrics and confirm that they are present
+      // here.
+      for (Entry<Long, Number> metricEntry : metricValues.entrySet()) {
+        assertEquals(metricEntry.getValue(),
+            metricMap.get(metricEntry.getKey()));
+      }
+
+      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, id,
+          entity.getType(), entity.getId(),
+          EnumSet.of(TimelineReader.Field.ALL));
+      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
+          id, entity.getType(), null, null, null, null, null, null, null,
+          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
+      assertNotNull(e1);
+      assertEquals(1, es1.size());
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+      if (hbr != null) {
+        hbr.stop();
+        hbr.close();
+      }
+    }
   }
 
   @Test
@@ -154,7 +360,7 @@ public class TestHBaseTimelineWriterImpl {
       hbr = new HBaseTimelineReaderImpl();
       hbr.init(c1);
       hbr.start();
-      String cluster = "cluster1";
+      String cluster = "cluster_test_write_entity";
       String user = "user1";
       String flow = "some_flow_name";
       String flowVersion = "AB7822C10F1111";
@@ -268,7 +474,8 @@ public class TestHBaseTimelineWriterImpl {
       assertEquals(17, colCount);
 
       TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
-          entity.getType(), entity.getId(), EnumSet.of(TimelineReader.Field.ALL));
+          entity.getType(), entity.getId(),
+          EnumSet.of(TimelineReader.Field.ALL));
       Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
           appName, entity.getType(), null, null, null, null, null, null, null,
           null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
@@ -284,10 +491,6 @@ public class TestHBaseTimelineWriterImpl {
         hbr.close();
       }
     }
-
-    // Somewhat of a hack, not a separate test in order not to have to deal with
-    // test case order exectution.
-    testAdditionalEntity();
   }
 
   private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user,
@@ -299,14 +502,31 @@ public class TestHBaseTimelineWriterImpl {
     assertEquals(user, Bytes.toString(rowKeyComponents[0]));
     assertEquals(cluster, Bytes.toString(rowKeyComponents[1]));
     assertEquals(flow, Bytes.toString(rowKeyComponents[2]));
-    assertEquals(TimelineWriterUtils.invert(runid), Bytes.toLong(rowKeyComponents[3]));
+    assertEquals(TimelineWriterUtils.invert(runid),
+        Bytes.toLong(rowKeyComponents[3]));
     assertEquals(appName, Bytes.toString(rowKeyComponents[4]));
     assertEquals(te.getType(), Bytes.toString(rowKeyComponents[5]));
     assertEquals(te.getId(), Bytes.toString(rowKeyComponents[6]));
     return true;
   }
 
-  private void testAdditionalEntity() throws IOException {
+  private boolean isApplicationRowKeyCorrect(byte[] rowKey, String cluster,
+      String user, String flow, Long runid, String appName) {
+
+    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey, -1);
+
+    assertTrue(rowKeyComponents.length == 5);
+    assertEquals(cluster, Bytes.toString(rowKeyComponents[0]));
+    assertEquals(user, Bytes.toString(rowKeyComponents[1]));
+    assertEquals(flow, Bytes.toString(rowKeyComponents[2]));
+    assertEquals(TimelineWriterUtils.invert(runid),
+        Bytes.toLong(rowKeyComponents[3]));
+    assertEquals(appName, Bytes.toString(rowKeyComponents[4]));
+    return true;
+  }
+
+  @Test
+  public void testEvents() throws IOException {
     TimelineEvent event = new TimelineEvent();
     String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE;
     event.setId(eventId);
@@ -333,7 +553,7 @@ public class TestHBaseTimelineWriterImpl {
       hbr = new HBaseTimelineReaderImpl();
       hbr.init(c1);
       hbr.start();
-      String cluster = "cluster2";
+      String cluster = "cluster_test_events";
       String user = "user2";
       String flow = "other_flow_name";
       String flowVersion = "1111F01C2287BA";
@@ -341,50 +561,46 @@ public class TestHBaseTimelineWriterImpl {
       String appName = "some app name";
       hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
       hbi.stop();
-      // scan the table and see that entity exists
-      Scan s = new Scan();
-      byte[] startRow =
-          EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
-      s.setStartRow(startRow);
-      Connection conn = ConnectionFactory.createConnection(c1);
-      ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
-
-      int rowCount = 0;
-      for (Result result : scanner) {
-        if (result != null && !result.isEmpty()) {
-          rowCount++;
-
-          // check the row key
-          byte[] row1 = result.getRow();
-          assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
-              entity));
 
-          Map<String, Object> eventsResult =
-              EntityColumnPrefix.EVENT.readResults(result);
-          // there should be only one event
-          assertEquals(1, eventsResult.size());
-          // key name for the event
-          byte[] compoundColumnQualifierBytes =
-              Separator.VALUES.join(Bytes.toBytes(eventId),
-                  Bytes.toBytes(TimelineWriterUtils.invert(expTs)),
-                  Bytes.toBytes(expKey));
-          String valueKey = Bytes.toString(compoundColumnQualifierBytes);
-          for (Map.Entry<String, Object> e :
-              eventsResult.entrySet()) {
-            // the value key must match
-            assertEquals(valueKey, e.getKey());
-            Object value = e.getValue();
-            // there should be only one timestamp and value
-            assertEquals(expVal, value.toString());
-          }
-        }
+      // retrieve the row
+      byte[] rowKey =
+          ApplicationRowKey.getRowKey(cluster, user, flow, runid, appName);
+      Get get = new Get(rowKey);
+      get.setMaxVersions(Integer.MAX_VALUE);
+      Connection conn = ConnectionFactory.createConnection(c1);
+      Result result = new ApplicationTable().getResult(c1, conn, get);
+
+      assertTrue(result != null);
+
+      // check the row key
+      byte[] row1 = result.getRow();
+      assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
+          appName));
+
+      Map<String, Object> eventsResult =
+          ApplicationColumnPrefix.EVENT.readResults(result);
+      // there should be only one event
+      assertEquals(1, eventsResult.size());
+      // key name for the event
+      byte[] compoundColumnQualifierBytes =
+          Separator.VALUES.join(Bytes.toBytes(eventId),
+              Bytes.toBytes(TimelineWriterUtils.invert(expTs)),
+              Bytes.toBytes(expKey));
+      String valueKey = Bytes.toString(compoundColumnQualifierBytes);
+      for (Map.Entry<String, Object> e : eventsResult.entrySet()) {
+        // the value key must match
+        assertEquals(valueKey, e.getKey());
+        Object value = e.getValue();
+        // there should be only one timestamp and value
+        assertEquals(expVal, value.toString());
       }
-      assertEquals(1, rowCount);
 
       TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
-          entity.getType(), entity.getId(), EnumSet.of(TimelineReader.Field.ALL));
+          entity.getType(), entity.getId(),
+          EnumSet.of(TimelineReader.Field.ALL));
       TimelineEntity e2 = hbr.getEntity(user, cluster, null, null, appName,
-          entity.getType(), entity.getId(), EnumSet.of(TimelineReader.Field.ALL));
+          entity.getType(), entity.getId(),
+          EnumSet.of(TimelineReader.Field.ALL));
       Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
           appName, entity.getType(), null, null, null, null, null, null, null,
           null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
@@ -410,7 +626,7 @@ public class TestHBaseTimelineWriterImpl {
   }
 
   @Test
-  public void testAdditionalEntityEmptyEventInfo() throws IOException {
+  public void testEventsWithEmptyInfo() throws IOException {
     TimelineEvent event = new TimelineEvent();
     String eventId = "foo_event_id";
     event.setId(eventId);
@@ -435,7 +651,7 @@ public class TestHBaseTimelineWriterImpl {
       hbr = new HBaseTimelineReaderImpl();
       hbr.init(c1);
       hbr.start();
-      String cluster = "cluster_emptyeventkey";
+      String cluster = "cluster_test_empty_eventkey";
       String user = "user_emptyeventkey";
       String flow = "other_flow_name";
       String flowVersion = "1111F01C2287BA";
@@ -487,7 +703,8 @@ public class TestHBaseTimelineWriterImpl {
       assertEquals(1, rowCount);
 
       TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
-          entity.getType(), entity.getId(), EnumSet.of(TimelineReader.Field.ALL));
+          entity.getType(), entity.getId(),
+          EnumSet.of(TimelineReader.Field.ALL));
       Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
           appName, entity.getType(), null, null, null, null, null, null, null,
           null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));


[36/50] [abbrv] hadoop git commit: YARN-3623. Add a config to indicate the Timeline Service version. Contributed by Xuan Gong.

Posted by gt...@apache.org.
YARN-3623. Add a config to indicate the Timeline Service version. Contributed by Xuan Gong.

(cherry picked from commit f910e4f639dc311fcb257bfcb869b1aa8b2c0643)


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

Branch: refs/heads/feature-YARN-2928
Commit: cb46808d6a7779cf331adaf09530e5d4f4e39bee
Parents: 76bc71c
Author: Junping Du <ju...@apache.org>
Authored: Thu Dec 10 02:44:30 2015 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 18:01:06 2016 -0800

----------------------------------------------------------------------
 .../hadoop-yarn-common/src/main/resources/yarn-default.xml         | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb46808d/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 eb66475..a748135 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
@@ -1811,7 +1811,7 @@
     <description>Indicate what is the current version of the running
     timeline service. For example, if "yarn.timeline-service.version" is 1.5,
     and "yarn.timeline-service.enabled" is true, it means the cluster will and
-    should bring up the timeline service v.1.5.
+    should bring up the timeline service v.1.5 (and nothing else).
     On the client side, if the client uses the same version of timeline service,
     it should succeed. If the client chooses to use a smaller version in spite of this,
     then depending on how robust the compatibility story is between versions,


[49/50] [abbrv] hadoop git commit: YARN-4200. Refactor reader classes in storage to nest under hbase specific package name. Contributed by Li Lu.

Posted by gt...@apache.org.
YARN-4200. Refactor reader classes in storage to nest under hbase
specific package name. Contributed by Li Lu.


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

Branch: refs/heads/feature-YARN-2928
Commit: 63d909927b4dd123f43523c1449033b419f51834
Parents: 5157c30
Author: Li Lu <gt...@apache.org>
Authored: Mon Jan 11 18:05:36 2016 -0800
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 18:03:31 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../storage/ApplicationEntityReader.java        | 382 --------------
 .../storage/FlowActivityEntityReader.java       | 163 ------
 .../storage/FlowRunEntityReader.java            | 225 ---------
 .../storage/GenericEntityReader.java            | 496 ------------------
 .../storage/HBaseTimelineReaderImpl.java        |   2 +
 .../storage/TimelineEntityReader.java           | 274 ----------
 .../storage/TimelineEntityReaderFactory.java    | 100 ----
 .../storage/reader/ApplicationEntityReader.java | 383 ++++++++++++++
 .../reader/FlowActivityEntityReader.java        | 164 ++++++
 .../storage/reader/FlowRunEntityReader.java     | 226 +++++++++
 .../storage/reader/GenericEntityReader.java     | 497 +++++++++++++++++++
 .../storage/reader/TimelineEntityReader.java    | 274 ++++++++++
 .../reader/TimelineEntityReaderFactory.java     | 100 ++++
 .../storage/reader/package-info.java            |  23 +
 15 files changed, 1672 insertions(+), 1640 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7827d77..1a0ed56 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -129,6 +129,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
 
   IMPROVEMENTS
 
+    YARN-4200. Refactor reader classes in storage to nest under hbase specific 
+    package name. Contributed by Li Lu. 
+
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via
     zjshen)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/ApplicationEntityReader.java
deleted file mode 100644
index d812a6c..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/ApplicationEntityReader.java
+++ /dev/null
@@ -1,382 +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;
-
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.Map;
-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.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.filter.BinaryComparator;
-import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.filter.FamilyFilter;
-import org.apache.hadoop.hbase.filter.FilterList;
-import org.apache.hadoop.hbase.filter.FilterList.Operator;
-import org.apache.hadoop.hbase.filter.PageFilter;
-import org.apache.hadoop.hbase.filter.QualifierFilter;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
-import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumn;
-import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnFamily;
-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.common.BaseTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Timeline entity reader for application entities that are stored in the
- * application table.
- */
-class ApplicationEntityReader extends GenericEntityReader {
-  private static final ApplicationTable APPLICATION_TABLE =
-      new ApplicationTable();
-
-  public ApplicationEntityReader(String userId, String clusterId,
-      String flowName, Long flowRunId, String appId, String entityType,
-      Long limit, Long createdTimeBegin, Long createdTimeEnd,
-      Long modifiedTimeBegin, Long modifiedTimeEnd,
-      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
-      Map<String, Object> infoFilters, Map<String, String> configFilters,
-      Set<String> metricFilters, Set<String> eventFilters,
-      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
-      EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
-        createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
-        relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, confsToRetrieve, metricsToRetrieve, fieldsToRetrieve,
-        true);
-  }
-
-  public ApplicationEntityReader(String userId, String clusterId,
-      String flowName, Long flowRunId, String appId, String entityType,
-      String entityId, TimelineFilterList confsToRetrieve,
-      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
-        confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
-  }
-
-  /**
-   * Uses the {@link ApplicationTable}.
-   */
-  protected BaseTable<?> getTable() {
-    return APPLICATION_TABLE;
-  }
-
-  @Override
-  protected FilterList constructFilterListBasedOnFields() {
-    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
-    // Fetch all the columns.
-    if (fieldsToRetrieve.contains(Field.ALL) &&
-        (confsToRetrieve == null ||
-        confsToRetrieve.getFilterList().isEmpty()) &&
-        (metricsToRetrieve == null ||
-        metricsToRetrieve.getFilterList().isEmpty())) {
-      return list;
-    }
-    FilterList infoColFamilyList = new FilterList();
-    // By default fetch everything in INFO column family.
-    FamilyFilter infoColumnFamily =
-        new FamilyFilter(CompareOp.EQUAL,
-           new BinaryComparator(ApplicationColumnFamily.INFO.getBytes()));
-    infoColFamilyList.addFilter(infoColumnFamily);
-    // Events not required.
-    if (!fieldsToRetrieve.contains(Field.EVENTS) &&
-        !fieldsToRetrieve.contains(Field.ALL) && eventFilters == null) {
-      infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-          ApplicationColumnPrefix.EVENT.getColumnPrefixBytes(""))));
-    }
-    // info not required.
-    if (!fieldsToRetrieve.contains(Field.INFO) &&
-        !fieldsToRetrieve.contains(Field.ALL) && infoFilters == null) {
-      infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-          ApplicationColumnPrefix.INFO.getColumnPrefixBytes(""))));
-    }
-    // is releated to not required.
-    if (!fieldsToRetrieve.contains(Field.IS_RELATED_TO) &&
-        !fieldsToRetrieve.contains(Field.ALL) && isRelatedTo == null) {
-      infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-          ApplicationColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes(""))));
-    }
-    // relates to not required.
-    if (!fieldsToRetrieve.contains(Field.RELATES_TO) &&
-        !fieldsToRetrieve.contains(Field.ALL) && relatesTo == null) {
-      infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-          ApplicationColumnPrefix.RELATES_TO.getColumnPrefixBytes(""))));
-    }
-    list.addFilter(infoColFamilyList);
-    if ((fieldsToRetrieve.contains(Field.CONFIGS) || configFilters != null) ||
-        (confsToRetrieve != null &&
-        !confsToRetrieve.getFilterList().isEmpty())) {
-      FilterList filterCfg =
-          new FilterList(new FamilyFilter(CompareOp.EQUAL,
-          new BinaryComparator(ApplicationColumnFamily.CONFIGS.getBytes())));
-      if (confsToRetrieve != null &&
-          !confsToRetrieve.getFilterList().isEmpty()) {
-        filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList(
-            ApplicationColumnPrefix.CONFIG, confsToRetrieve));
-      }
-      list.addFilter(filterCfg);
-    }
-    if ((fieldsToRetrieve.contains(Field.METRICS) || metricFilters != null) ||
-        (metricsToRetrieve != null &&
-        !metricsToRetrieve.getFilterList().isEmpty())) {
-      FilterList filterMetrics =
-          new FilterList(new FamilyFilter(CompareOp.EQUAL,
-          new BinaryComparator(ApplicationColumnFamily.METRICS.getBytes())));
-      if (metricsToRetrieve != null &&
-          !metricsToRetrieve.getFilterList().isEmpty()) {
-        filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList(
-            ApplicationColumnPrefix.METRIC, metricsToRetrieve));
-      }
-      list.addFilter(filterMetrics);
-    }
-    return list;
-  }
-
-  @Override
-  protected Result getResult(Configuration hbaseConf, Connection conn,
-      FilterList filterList) throws IOException {
-    byte[] rowKey =
-        ApplicationRowKey.getRowKey(clusterId, userId, flowName, flowRunId,
-            appId);
-    Get get = new Get(rowKey);
-    get.setMaxVersions(Integer.MAX_VALUE);
-    if (filterList != null && !filterList.getFilters().isEmpty()) {
-      get.setFilter(filterList);
-    }
-    return table.getResult(hbaseConf, conn, get);
-  }
-
-  @Override
-  protected void validateParams() {
-    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
-    Preconditions.checkNotNull(entityType, "entityType shouldn't be null");
-    if (singleEntityRead) {
-      Preconditions.checkNotNull(appId, "appId shouldn't be null");
-    } else {
-      Preconditions.checkNotNull(userId, "userId shouldn't be null");
-      Preconditions.checkNotNull(flowName, "flowName shouldn't be null");
-    }
-  }
-
-  @Override
-  protected void augmentParams(Configuration hbaseConf, Connection conn)
-      throws IOException {
-    if (singleEntityRead) {
-      if (flowName == null || flowRunId == null || userId == null) {
-        FlowContext context =
-            lookupFlowContext(clusterId, appId, hbaseConf, conn);
-        flowName = context.flowName;
-        flowRunId = context.flowRunId;
-        userId = context.userId;
-      }
-    }
-    if (fieldsToRetrieve == null) {
-      fieldsToRetrieve = EnumSet.noneOf(Field.class);
-    }
-    if (!fieldsToRetrieve.contains(Field.CONFIGS) &&
-        confsToRetrieve != null && !confsToRetrieve.getFilterList().isEmpty()) {
-      fieldsToRetrieve.add(Field.CONFIGS);
-    }
-    if (!fieldsToRetrieve.contains(Field.METRICS) &&
-        metricsToRetrieve != null &&
-        !metricsToRetrieve.getFilterList().isEmpty()) {
-      fieldsToRetrieve.add(Field.METRICS);
-    }
-    if (!singleEntityRead) {
-      if (limit == null || limit < 0) {
-        limit = TimelineReader.DEFAULT_LIMIT;
-      }
-      if (createdTimeBegin == null) {
-        createdTimeBegin = DEFAULT_BEGIN_TIME;
-      }
-      if (createdTimeEnd == null) {
-        createdTimeEnd = DEFAULT_END_TIME;
-      }
-      if (modifiedTimeBegin == null) {
-        modifiedTimeBegin = DEFAULT_BEGIN_TIME;
-      }
-      if (modifiedTimeEnd == null) {
-        modifiedTimeEnd = DEFAULT_END_TIME;
-      }
-    }
-  }
-
-  @Override
-  protected ResultScanner getResults(Configuration hbaseConf,
-      Connection conn, FilterList filterList) throws IOException {
-    Scan scan = new Scan();
-    if (flowRunId != null) {
-      scan.setRowPrefixFilter(ApplicationRowKey.
-          getRowKeyPrefix(clusterId, userId, flowName, flowRunId));
-    } else {
-      scan.setRowPrefixFilter(ApplicationRowKey.
-          getRowKeyPrefix(clusterId, userId, flowName));
-    }
-    FilterList newList = new FilterList();
-    newList.addFilter(new PageFilter(limit));
-    if (filterList != null && !filterList.getFilters().isEmpty()) {
-      newList.addFilter(filterList);
-    }
-    scan.setFilter(newList);
-    return table.getResultScanner(hbaseConf, conn, scan);
-  }
-
-  @Override
-  protected TimelineEntity parseEntity(Result result) throws IOException {
-    if (result == null || result.isEmpty()) {
-      return null;
-    }
-    TimelineEntity entity = new TimelineEntity();
-    entity.setType(TimelineEntityType.YARN_APPLICATION.toString());
-    String entityId = ApplicationColumn.ID.readResult(result).toString();
-    entity.setId(entityId);
-
-    // fetch created time
-    Number createdTime =
-        (Number)ApplicationColumn.CREATED_TIME.readResult(result);
-    entity.setCreatedTime(createdTime.longValue());
-    if (!singleEntityRead && (entity.getCreatedTime() < createdTimeBegin ||
-        entity.getCreatedTime() > createdTimeEnd)) {
-      return null;
-    }
-
-    // fetch modified time
-    Number modifiedTime =
-        (Number)ApplicationColumn.MODIFIED_TIME.readResult(result);
-    entity.setModifiedTime(modifiedTime.longValue());
-    if (!singleEntityRead && (entity.getModifiedTime() < modifiedTimeBegin ||
-        entity.getModifiedTime() > modifiedTimeEnd)) {
-      return null;
-    }
-
-    // fetch is related to entities
-    boolean checkIsRelatedTo = isRelatedTo != null && isRelatedTo.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
-      readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO,
-          true);
-      if (checkIsRelatedTo && !TimelineStorageUtils.matchRelations(
-          entity.getIsRelatedToEntities(), isRelatedTo)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.IS_RELATED_TO)) {
-        entity.getIsRelatedToEntities().clear();
-      }
-    }
-
-    // fetch relates to entities
-    boolean checkRelatesTo = relatesTo != null && relatesTo.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
-      readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO,
-          false);
-      if (checkRelatesTo && !TimelineStorageUtils.matchRelations(
-          entity.getRelatesToEntities(), relatesTo)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.RELATES_TO)) {
-        entity.getRelatesToEntities().clear();
-      }
-    }
-
-    // fetch info
-    boolean checkInfo = infoFilters != null && infoFilters.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
-      readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false);
-      if (checkInfo &&
-          !TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.INFO)) {
-        entity.getInfo().clear();
-      }
-    }
-
-    // fetch configs
-    boolean checkConfigs = configFilters != null && configFilters.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
-      readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true);
-      if (checkConfigs && !TimelineStorageUtils.matchFilters(
-          entity.getConfigs(), configFilters)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.CONFIGS)) {
-        entity.getConfigs().clear();
-      }
-    }
-
-    // fetch events
-    boolean checkEvents = eventFilters != null && eventFilters.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
-      readEvents(entity, result, true);
-      if (checkEvents && !TimelineStorageUtils.matchEventFilters(
-          entity.getEvents(), eventFilters)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.EVENTS)) {
-        entity.getEvents().clear();
-      }
-    }
-
-    // fetch metrics
-    boolean checkMetrics = metricFilters != null && metricFilters.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
-      readMetrics(entity, result, ApplicationColumnPrefix.METRIC);
-      if (checkMetrics && !TimelineStorageUtils.matchMetricFilters(
-          entity.getMetrics(), metricFilters)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.METRICS)) {
-        entity.getMetrics().clear();
-      }
-    }
-    return entity;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.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/FlowActivityEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
deleted file mode 100644
index 7e8d4ba..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowActivityEntityReader.java
+++ /dev/null
@@ -1,163 +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;
-
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.filter.FilterList;
-import org.apache.hadoop.hbase.filter.PageFilter;
-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.TimelineEntity;
-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.flow.FlowActivityColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Timeline entity reader for flow activity entities that are stored in the
- * flow activity table.
- */
-class FlowActivityEntityReader extends TimelineEntityReader {
-  private static final FlowActivityTable FLOW_ACTIVITY_TABLE =
-      new FlowActivityTable();
-
-  public FlowActivityEntityReader(String userId, String clusterId,
-      String flowName, Long flowRunId, String appId, String entityType,
-      Long limit, Long createdTimeBegin, Long createdTimeEnd,
-      Long modifiedTimeBegin, Long modifiedTimeEnd,
-      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
-      Map<String, Object> infoFilters, Map<String, String> configFilters,
-      Set<String> metricFilters, Set<String> eventFilters,
-      EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
-        createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
-        relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, null, null, fieldsToRetrieve, true);
-  }
-
-  public FlowActivityEntityReader(String userId, String clusterId,
-      String flowName, Long flowRunId, String appId, String entityType,
-      String entityId, EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
-        null, null, fieldsToRetrieve);
-  }
-
-  /**
-   * Uses the {@link FlowActivityTable}.
-   */
-  @Override
-  protected BaseTable<?> getTable() {
-    return FLOW_ACTIVITY_TABLE;
-  }
-
-  @Override
-  protected void validateParams() {
-    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
-  }
-
-  @Override
-  protected void augmentParams(Configuration hbaseConf, Connection conn)
-      throws IOException {
-    if (limit == null || limit < 0) {
-      limit = TimelineReader.DEFAULT_LIMIT;
-    }
-    if (createdTimeBegin == null) {
-      createdTimeBegin = DEFAULT_BEGIN_TIME;
-    }
-    if (createdTimeEnd == null) {
-      createdTimeEnd = DEFAULT_END_TIME;
-    }
-  }
-
-  @Override
-  protected FilterList constructFilterListBasedOnFields() {
-    return null;
-  }
-
-  @Override
-  protected Result getResult(Configuration hbaseConf, Connection conn,
-      FilterList filterList) throws IOException {
-    throw new UnsupportedOperationException(
-        "we don't support a single entity query");
-  }
-
-  @Override
-  protected ResultScanner getResults(Configuration hbaseConf,
-      Connection conn, FilterList filterList) throws IOException {
-    Scan scan = new Scan();
-    if (createdTimeBegin == DEFAULT_BEGIN_TIME &&
-        createdTimeEnd == DEFAULT_END_TIME) {
-      scan.setRowPrefixFilter(FlowActivityRowKey.getRowKeyPrefix(clusterId));
-    } else {
-      scan.setStartRow(
-          FlowActivityRowKey.getRowKeyPrefix(clusterId, createdTimeEnd));
-      scan.setStopRow(
-          FlowActivityRowKey.getRowKeyPrefix(clusterId,
-              (createdTimeBegin <= 0 ? 0: (createdTimeBegin - 1))));
-    }
-    // use the page filter to limit the result to the page size
-    // the scanner may still return more than the limit; therefore we need to
-    // read the right number as we iterate
-    scan.setFilter(new PageFilter(limit));
-    return table.getResultScanner(hbaseConf, conn, scan);
-  }
-
-  @Override
-  protected TimelineEntity parseEntity(Result result) throws IOException {
-    FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(result.getRow());
-
-    long time = rowKey.getDayTimestamp();
-    String user = rowKey.getUserId();
-    String flowName = rowKey.getFlowName();
-
-    FlowActivityEntity flowActivity =
-        new FlowActivityEntity(clusterId, time, user, flowName);
-    // set the id
-    flowActivity.setId(flowActivity.getId());
-    // get the list of run ids along with the version that are associated with
-    // this flow on this day
-    Map<String, Object> runIdsMap =
-        FlowActivityColumnPrefix.RUN_ID.readResults(result);
-    for (Map.Entry<String, Object> e : runIdsMap.entrySet()) {
-      Long runId = Long.valueOf(e.getKey());
-      String version = (String)e.getValue();
-      FlowRunEntity flowRun = new FlowRunEntity();
-      flowRun.setUser(user);
-      flowRun.setName(flowName);
-      flowRun.setRunId(runId);
-      flowRun.setVersion(version);
-      // set the id
-      flowRun.setId(flowRun.getId());
-      flowActivity.addFlowRun(flowRun);
-    }
-
-    return flowActivity;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/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/FlowRunEntityReader.java
deleted file mode 100644
index c9076ee..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FlowRunEntityReader.java
+++ /dev/null
@@ -1,225 +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;
-
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.Map;
-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.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.filter.BinaryComparator;
-import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
-import org.apache.hadoop.hbase.filter.FamilyFilter;
-import org.apache.hadoop.hbase.filter.FilterList;
-import org.apache.hadoop.hbase.filter.PageFilter;
-import org.apache.hadoop.hbase.filter.QualifierFilter;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.filter.FilterList.Operator;
-import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnFamily;
-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 com.google.common.base.Preconditions;
-
-/**
- * Timeline entity reader for flow run entities that are stored in the flow run
- * table.
- */
-class FlowRunEntityReader extends TimelineEntityReader {
-  private static final FlowRunTable FLOW_RUN_TABLE = new FlowRunTable();
-
-  public FlowRunEntityReader(String userId, String clusterId,
-      String flowName, Long flowRunId, String appId, String entityType,
-      Long limit, Long createdTimeBegin, Long createdTimeEnd,
-      Long modifiedTimeBegin, Long modifiedTimeEnd,
-      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
-      Map<String, Object> infoFilters, Map<String, String> configFilters,
-      Set<String> metricFilters, Set<String> eventFilters,
-      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
-      EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
-        createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
-        relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, null, metricsToRetrieve, fieldsToRetrieve, true);
-  }
-
-  public FlowRunEntityReader(String userId, String clusterId,
-      String flowName, Long flowRunId, String appId, String entityType,
-      String entityId, TimelineFilterList confsToRetrieve,
-      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
-        null, metricsToRetrieve, fieldsToRetrieve);
-  }
-
-  /**
-   * Uses the {@link FlowRunTable}.
-   */
-  @Override
-  protected BaseTable<?> getTable() {
-    return FLOW_RUN_TABLE;
-  }
-
-  @Override
-  protected void validateParams() {
-    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
-    Preconditions.checkNotNull(userId, "userId shouldn't be null");
-    Preconditions.checkNotNull(flowName, "flowName shouldn't be null");
-    if (singleEntityRead) {
-      Preconditions.checkNotNull(flowRunId, "flowRunId shouldn't be null");
-    }
-  }
-
-  @Override
-  protected void augmentParams(Configuration hbaseConf, Connection conn) {
-    if (!singleEntityRead) {
-      if (fieldsToRetrieve == null) {
-        fieldsToRetrieve = EnumSet.noneOf(Field.class);
-      }
-      if (limit == null || limit < 0) {
-        limit = TimelineReader.DEFAULT_LIMIT;
-      }
-      if (createdTimeBegin == null) {
-        createdTimeBegin = DEFAULT_BEGIN_TIME;
-      }
-      if (createdTimeEnd == null) {
-        createdTimeEnd = DEFAULT_END_TIME;
-      }
-      if (!fieldsToRetrieve.contains(Field.METRICS) &&
-          metricsToRetrieve != null &&
-          !metricsToRetrieve.getFilterList().isEmpty()) {
-        fieldsToRetrieve.add(Field.METRICS);
-      }
-    }
-  }
-
-  @Override
-  protected FilterList constructFilterListBasedOnFields() {
-    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
-
-    // By default fetch everything in INFO column family.
-    FamilyFilter infoColumnFamily =
-        new FamilyFilter(CompareOp.EQUAL,
-           new BinaryComparator(FlowRunColumnFamily.INFO.getBytes()));
-    // Metrics not required.
-    if (!singleEntityRead && !fieldsToRetrieve.contains(Field.METRICS) &&
-        !fieldsToRetrieve.contains(Field.ALL)) {
-      FilterList infoColFamilyList = new FilterList(Operator.MUST_PASS_ONE);
-      infoColFamilyList.addFilter(infoColumnFamily);
-      infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(""))));
-      list.addFilter(infoColFamilyList);
-    }
-    if (metricsToRetrieve != null &&
-        !metricsToRetrieve.getFilterList().isEmpty()) {
-      FilterList infoColFamilyList = new FilterList();
-      infoColFamilyList.addFilter(infoColumnFamily);
-      infoColFamilyList.addFilter(TimelineFilterUtils.createHBaseFilterList(
-          FlowRunColumnPrefix.METRIC, metricsToRetrieve));
-      list.addFilter(infoColFamilyList);
-    }
-    return list;
-  }
-
-  @Override
-  protected Result getResult(Configuration hbaseConf, Connection conn,
-      FilterList filterList) throws IOException {
-    byte[] rowKey =
-        FlowRunRowKey.getRowKey(clusterId, userId, flowName, flowRunId);
-    Get get = new Get(rowKey);
-    get.setMaxVersions(Integer.MAX_VALUE);
-    if (filterList != null && !filterList.getFilters().isEmpty()) {
-      get.setFilter(filterList);
-    }
-    return table.getResult(hbaseConf, conn, get);
-  }
-
-  @Override
-  protected ResultScanner getResults(Configuration hbaseConf,
-      Connection conn, FilterList filterList) throws IOException {
-    Scan scan = new Scan();
-    scan.setRowPrefixFilter(
-        FlowRunRowKey.getRowKeyPrefix(clusterId, userId, flowName));
-    FilterList newList = new FilterList();
-    newList.addFilter(new PageFilter(limit));
-    if (filterList != null && !filterList.getFilters().isEmpty()) {
-      newList.addFilter(filterList);
-    }
-    scan.setFilter(newList);
-    return table.getResultScanner(hbaseConf, conn, scan);
-  }
-
-  @Override
-  protected TimelineEntity parseEntity(Result result) throws IOException {
-    FlowRunEntity flowRun = new FlowRunEntity();
-    flowRun.setUser(userId);
-    flowRun.setName(flowName);
-    if (singleEntityRead) {
-      flowRun.setRunId(flowRunId);
-    } else {
-      FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(result.getRow());
-      flowRun.setRunId(rowKey.getFlowRunId());
-    }
-
-    // read the start time
-    Long startTime = (Long)FlowRunColumn.MIN_START_TIME.readResult(result);
-    if (startTime != null) {
-      flowRun.setStartTime(startTime.longValue());
-    }
-    if (!singleEntityRead && (flowRun.getStartTime() < createdTimeBegin ||
-        flowRun.getStartTime() > createdTimeEnd)) {
-      return null;
-    }
-
-    // read the end time if available
-    Long endTime = (Long)FlowRunColumn.MAX_END_TIME.readResult(result);
-    if (endTime != null) {
-      flowRun.setMaxEndTime(endTime.longValue());
-    }
-
-    // read the flow version
-    String version = (String)FlowRunColumn.FLOW_VERSION.readResult(result);
-    if (version != null) {
-      flowRun.setVersion(version);
-    }
-
-    // read metrics
-    if (singleEntityRead || fieldsToRetrieve.contains(Field.METRICS)) {
-      readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC);
-    }
-
-    // set the id
-    flowRun.setId(flowRun.getId());
-    return flowRun;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.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/GenericEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
deleted file mode 100644
index 784dfd5..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/GenericEntityReader.java
+++ /dev/null
@@ -1,496 +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;
-
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-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.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.filter.BinaryComparator;
-import org.apache.hadoop.hbase.filter.BinaryPrefixComparator;
-import org.apache.hadoop.hbase.filter.FamilyFilter;
-import org.apache.hadoop.hbase.filter.FilterList;
-import org.apache.hadoop.hbase.filter.QualifierFilter;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.filter.FilterList.Operator;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
-import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
-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.AppToFlowRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Timeline entity reader for generic entities that are stored in the entity
- * table.
- */
-class GenericEntityReader extends TimelineEntityReader {
-  private static final EntityTable ENTITY_TABLE = new EntityTable();
-  private static final Log LOG = LogFactory.getLog(GenericEntityReader.class);
-
-  /**
-   * Used to look up the flow context.
-   */
-  private final AppToFlowTable appToFlowTable = new AppToFlowTable();
-
-  public GenericEntityReader(String userId, String clusterId,
-      String flowName, Long flowRunId, String appId, String entityType,
-      Long limit, Long createdTimeBegin, Long createdTimeEnd,
-      Long modifiedTimeBegin, Long modifiedTimeEnd,
-      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
-      Map<String, Object> infoFilters, Map<String, String> configFilters,
-      Set<String> metricFilters, Set<String> eventFilters,
-      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
-      EnumSet<Field> fieldsToRetrieve, boolean sortedKeys) {
-    super(userId, clusterId, flowName, flowRunId, appId, entityType, limit,
-        createdTimeBegin, createdTimeEnd, modifiedTimeBegin, modifiedTimeEnd,
-        relatesTo, isRelatedTo, infoFilters, configFilters, metricFilters,
-        eventFilters, confsToRetrieve, metricsToRetrieve, fieldsToRetrieve,
-        sortedKeys);
-  }
-
-  public GenericEntityReader(String userId, String clusterId,
-      String flowName, Long flowRunId, String appId, String entityType,
-      String entityId, TimelineFilterList confsToRetrieve,
-      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
-    super(userId, clusterId, flowName, flowRunId, appId, entityType, entityId,
-        confsToRetrieve, metricsToRetrieve, fieldsToRetrieve);
-  }
-
-  /**
-   * Uses the {@link EntityTable}.
-   */
-  protected BaseTable<?> getTable() {
-    return ENTITY_TABLE;
-  }
-
-  @Override
-  protected FilterList constructFilterListBasedOnFields() {
-    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
-    // Fetch all the columns.
-    if (fieldsToRetrieve.contains(Field.ALL) &&
-        (confsToRetrieve == null ||
-        confsToRetrieve.getFilterList().isEmpty()) &&
-        (metricsToRetrieve == null ||
-        metricsToRetrieve.getFilterList().isEmpty())) {
-      return list;
-    }
-    FilterList infoColFamilyList = new FilterList();
-    // By default fetch everything in INFO column family.
-    FamilyFilter infoColumnFamily =
-        new FamilyFilter(CompareOp.EQUAL,
-           new BinaryComparator(EntityColumnFamily.INFO.getBytes()));
-    infoColFamilyList.addFilter(infoColumnFamily);
-    // Events not required.
-    if (!fieldsToRetrieve.contains(Field.EVENTS) &&
-        !fieldsToRetrieve.contains(Field.ALL) && eventFilters == null) {
-      infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-          EntityColumnPrefix.EVENT.getColumnPrefixBytes(""))));
-    }
-    // info not required.
-    if (!fieldsToRetrieve.contains(Field.INFO) &&
-        !fieldsToRetrieve.contains(Field.ALL) && infoFilters == null) {
-      infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-              EntityColumnPrefix.INFO.getColumnPrefixBytes(""))));
-    }
-    // is related to not required.
-    if (!fieldsToRetrieve.contains(Field.IS_RELATED_TO) &&
-        !fieldsToRetrieve.contains(Field.ALL) && isRelatedTo == null) {
-      infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-              EntityColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes(""))));
-    }
-    // relates to not required.
-    if (!fieldsToRetrieve.contains(Field.RELATES_TO) &&
-        !fieldsToRetrieve.contains(Field.ALL) && relatesTo == null) {
-      infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-              EntityColumnPrefix.RELATES_TO.getColumnPrefixBytes(""))));
-    }
-    list.addFilter(infoColFamilyList);
-    if ((fieldsToRetrieve.contains(Field.CONFIGS) || configFilters != null) ||
-        (confsToRetrieve != null &&
-        !confsToRetrieve.getFilterList().isEmpty())) {
-      FilterList filterCfg =
-          new FilterList(new FamilyFilter(CompareOp.EQUAL,
-              new BinaryComparator(EntityColumnFamily.CONFIGS.getBytes())));
-      if (confsToRetrieve != null &&
-          !confsToRetrieve.getFilterList().isEmpty()) {
-        filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList(
-            EntityColumnPrefix.CONFIG, confsToRetrieve));
-      }
-      list.addFilter(filterCfg);
-    }
-    if ((fieldsToRetrieve.contains(Field.METRICS) || metricFilters != null) ||
-        (metricsToRetrieve != null &&
-        !metricsToRetrieve.getFilterList().isEmpty())) {
-      FilterList filterMetrics =
-          new FilterList(new FamilyFilter(CompareOp.EQUAL,
-              new BinaryComparator(EntityColumnFamily.METRICS.getBytes())));
-      if (metricsToRetrieve != null &&
-          !metricsToRetrieve.getFilterList().isEmpty()) {
-        filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList(
-            EntityColumnPrefix.METRIC, metricsToRetrieve));
-      }
-      list.addFilter(filterMetrics);
-    }
-    return list;
-  }
-
-  protected FlowContext lookupFlowContext(String clusterId, String appId,
-      Configuration hbaseConf, Connection conn) throws IOException {
-    byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
-    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());
-    } else {
-       throw new IOException(
-           "Unable to find the context flow ID and flow run ID for clusterId=" +
-           clusterId + ", appId=" + appId);
-    }
-  }
-
-  protected static class FlowContext {
-    protected final String userId;
-    protected final String flowName;
-    protected final Long flowRunId;
-    public FlowContext(String user, String flowName, Long flowRunId) {
-      this.userId = user;
-      this.flowName = flowName;
-      this.flowRunId = flowRunId;
-    }
-  }
-
-  @Override
-  protected void validateParams() {
-    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
-    Preconditions.checkNotNull(appId, "appId shouldn't be null");
-    Preconditions.checkNotNull(entityType, "entityType shouldn't be null");
-    if (singleEntityRead) {
-      Preconditions.checkNotNull(entityId, "entityId shouldn't be null");
-    }
-  }
-
-  @Override
-  protected void augmentParams(Configuration hbaseConf, Connection conn)
-      throws IOException {
-    // In reality all three should be null or neither should be null
-    if (flowName == null || flowRunId == null || userId == null) {
-      FlowContext context =
-          lookupFlowContext(clusterId, appId, hbaseConf, conn);
-      flowName = context.flowName;
-      flowRunId = context.flowRunId;
-      userId = context.userId;
-    }
-    if (fieldsToRetrieve == null) {
-      fieldsToRetrieve = EnumSet.noneOf(Field.class);
-    }
-    if (!fieldsToRetrieve.contains(Field.CONFIGS) &&
-        confsToRetrieve != null && !confsToRetrieve.getFilterList().isEmpty()) {
-      fieldsToRetrieve.add(Field.CONFIGS);
-    }
-    if (!fieldsToRetrieve.contains(Field.METRICS) &&
-        metricsToRetrieve != null &&
-        !metricsToRetrieve.getFilterList().isEmpty()) {
-      fieldsToRetrieve.add(Field.METRICS);
-    }
-    if (!singleEntityRead) {
-      if (limit == null || limit < 0) {
-        limit = TimelineReader.DEFAULT_LIMIT;
-      }
-      if (createdTimeBegin == null) {
-        createdTimeBegin = DEFAULT_BEGIN_TIME;
-      }
-      if (createdTimeEnd == null) {
-        createdTimeEnd = DEFAULT_END_TIME;
-      }
-      if (modifiedTimeBegin == null) {
-        modifiedTimeBegin = DEFAULT_BEGIN_TIME;
-      }
-      if (modifiedTimeEnd == null) {
-        modifiedTimeEnd = DEFAULT_END_TIME;
-      }
-    }
-  }
-
-  @Override
-  protected Result getResult(Configuration hbaseConf, Connection conn,
-      FilterList filterList) throws IOException {
-    byte[] rowKey =
-        EntityRowKey.getRowKey(clusterId, userId, flowName, flowRunId, appId,
-            entityType, entityId);
-    Get get = new Get(rowKey);
-    get.setMaxVersions(Integer.MAX_VALUE);
-    if (filterList != null && !filterList.getFilters().isEmpty()) {
-      get.setFilter(filterList);
-    }
-    return table.getResult(hbaseConf, conn, get);
-  }
-
-  @Override
-  protected ResultScanner getResults(Configuration hbaseConf,
-      Connection conn, FilterList filterList) throws IOException {
-    // Scan through part of the table to find the entities belong to one app
-    // and one type
-    Scan scan = new Scan();
-    scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
-        clusterId, userId, flowName, flowRunId, appId, entityType));
-    scan.setMaxVersions(Integer.MAX_VALUE);
-    if (filterList != null && !filterList.getFilters().isEmpty()) {
-      scan.setFilter(filterList);
-    }
-    return table.getResultScanner(hbaseConf, conn, scan);
-  }
-
-  @Override
-  protected TimelineEntity parseEntity(Result result) throws IOException {
-    if (result == null || result.isEmpty()) {
-      return null;
-    }
-    TimelineEntity entity = new TimelineEntity();
-    String entityType = EntityColumn.TYPE.readResult(result).toString();
-    entity.setType(entityType);
-    String entityId = EntityColumn.ID.readResult(result).toString();
-    entity.setId(entityId);
-
-    // fetch created time
-    Number createdTime = (Number)EntityColumn.CREATED_TIME.readResult(result);
-    entity.setCreatedTime(createdTime.longValue());
-    if (!singleEntityRead && (entity.getCreatedTime() < createdTimeBegin ||
-        entity.getCreatedTime() > createdTimeEnd)) {
-      return null;
-    }
-
-    // fetch modified time
-    Number modifiedTime = (Number)EntityColumn.MODIFIED_TIME.readResult(result);
-    entity.setModifiedTime(modifiedTime.longValue());
-    if (!singleEntityRead && (entity.getModifiedTime() < modifiedTimeBegin ||
-        entity.getModifiedTime() > modifiedTimeEnd)) {
-      return null;
-    }
-
-    // fetch is related to entities
-    boolean checkIsRelatedTo = isRelatedTo != null && isRelatedTo.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
-      readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO, true);
-      if (checkIsRelatedTo && !TimelineStorageUtils.matchRelations(
-          entity.getIsRelatedToEntities(), isRelatedTo)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.IS_RELATED_TO)) {
-        entity.getIsRelatedToEntities().clear();
-      }
-    }
-
-    // fetch relates to entities
-    boolean checkRelatesTo = relatesTo != null && relatesTo.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
-      readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false);
-      if (checkRelatesTo && !TimelineStorageUtils.matchRelations(
-          entity.getRelatesToEntities(), relatesTo)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.RELATES_TO)) {
-        entity.getRelatesToEntities().clear();
-      }
-    }
-
-    // fetch info
-    boolean checkInfo = infoFilters != null && infoFilters.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
-      readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false);
-      if (checkInfo &&
-          !TimelineStorageUtils.matchFilters(entity.getInfo(), infoFilters)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.INFO)) {
-        entity.getInfo().clear();
-      }
-    }
-
-    // fetch configs
-    boolean checkConfigs = configFilters != null && configFilters.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
-      readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true);
-      if (checkConfigs && !TimelineStorageUtils.matchFilters(
-          entity.getConfigs(), configFilters)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.CONFIGS)) {
-        entity.getConfigs().clear();
-      }
-    }
-
-    // fetch events
-    boolean checkEvents = eventFilters != null && eventFilters.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
-      readEvents(entity, result, false);
-      if (checkEvents && !TimelineStorageUtils.matchEventFilters(
-          entity.getEvents(), eventFilters)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.EVENTS)) {
-        entity.getEvents().clear();
-      }
-    }
-
-    // fetch metrics
-    boolean checkMetrics = metricFilters != null && metricFilters.size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
-      readMetrics(entity, result, EntityColumnPrefix.METRIC);
-      if (checkMetrics && !TimelineStorageUtils.matchMetricFilters(
-          entity.getMetrics(), metricFilters)) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.METRICS)) {
-        entity.getMetrics().clear();
-      }
-    }
-    return entity;
-  }
-
-  /**
-   * Helper method for reading relationship.
-   */
-  protected <T> void readRelationship(
-      TimelineEntity entity, Result result, ColumnPrefix<T> prefix,
-      boolean isRelatedTo) throws IOException {
-    // isRelatedTo and relatesTo are of type Map<String, Set<String>>
-    Map<String, Object> columns = prefix.readResults(result);
-    for (Map.Entry<String, Object> column : columns.entrySet()) {
-      for (String id : Separator.VALUES.splitEncoded(
-          column.getValue().toString())) {
-        if (isRelatedTo) {
-          entity.addIsRelatedToEntity(column.getKey(), id);
-        } else {
-          entity.addRelatesToEntity(column.getKey(), id);
-        }
-      }
-    }
-  }
-
-  /**
-   * Helper method for reading key-value pairs for either info or config.
-   */
-  protected <T> void readKeyValuePairs(
-      TimelineEntity entity, Result result, ColumnPrefix<T> prefix,
-      boolean isConfig) throws IOException {
-    // info and configuration are of type Map<String, Object or String>
-    Map<String, Object> columns = prefix.readResults(result);
-    if (isConfig) {
-      for (Map.Entry<String, Object> column : columns.entrySet()) {
-        entity.addConfig(column.getKey(), column.getValue().toString());
-      }
-    } else {
-      entity.addInfo(columns);
-    }
-  }
-
-  /**
-   * Read events from the entity table or the application table. The column name
-   * is of the form "eventId=timestamp=infoKey" where "infoKey" may be omitted
-   * if there is no info associated with the event.
-   *
-   * See {@link EntityTable} and {@link ApplicationTable} for a more detailed
-   * schema description.
-   */
-  protected void readEvents(TimelineEntity entity, Result result,
-      boolean isApplication) throws IOException {
-    Map<String, TimelineEvent> eventsMap = new HashMap<>();
-    Map<?, Object> eventsResult = isApplication ?
-        ApplicationColumnPrefix.EVENT.
-            readResultsHavingCompoundColumnQualifiers(result) :
-        EntityColumnPrefix.EVENT.
-            readResultsHavingCompoundColumnQualifiers(result);
-    for (Map.Entry<?, Object> eventResult : eventsResult.entrySet()) {
-      byte[][] karr = (byte[][])eventResult.getKey();
-      // the column name is of the form "eventId=timestamp=infoKey"
-      if (karr.length == 3) {
-        String id = Bytes.toString(karr[0]);
-        long ts = TimelineStorageUtils.invertLong(Bytes.toLong(karr[1]));
-        String key = Separator.VALUES.joinEncoded(id, Long.toString(ts));
-        TimelineEvent event = eventsMap.get(key);
-        if (event == null) {
-          event = new TimelineEvent();
-          event.setId(id);
-          event.setTimestamp(ts);
-          eventsMap.put(key, event);
-        }
-        // handle empty info
-        String infoKey = karr[2].length == 0 ? null : Bytes.toString(karr[2]);
-        if (infoKey != null) {
-          event.addInfo(infoKey, eventResult.getValue());
-        }
-      } else {
-        LOG.warn("incorrectly formatted column name: it will be discarded");
-        continue;
-      }
-    }
-    Set<TimelineEvent> eventsSet = new HashSet<>(eventsMap.values());
-    entity.addEvents(eventsSet);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.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/HBaseTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
index 96c5a19..bc48cbe 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
@@ -32,6 +32,8 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReader;
+import org.apache.hadoop.yarn.server.timelineservice.storage.reader.TimelineEntityReaderFactory;
 
 public class HBaseTimelineReaderImpl
     extends AbstractService implements TimelineReader {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.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/TimelineEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
deleted file mode 100644
index a26c0c2..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReader.java
+++ /dev/null
@@ -1,274 +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;
-
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.filter.FilterList;
-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.reader.filter.TimelineFilterList;
-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.ColumnPrefix;
-
-/**
- * The base class for reading and deserializing timeline entities from the
- * HBase storage. Different types can be defined for different types of the
- * entities that are being requested.
- */
-abstract class TimelineEntityReader {
-  private static final Log LOG = LogFactory.getLog(TimelineEntityReader.class);
-  protected static final long DEFAULT_BEGIN_TIME = 0L;
-  protected static final long DEFAULT_END_TIME = Long.MAX_VALUE;
-
-  protected final boolean singleEntityRead;
-
-  protected String userId;
-  protected String clusterId;
-  protected String flowName;
-  protected Long flowRunId;
-  protected String appId;
-  protected String entityType;
-  protected EnumSet<Field> fieldsToRetrieve;
-  // used only for a single entity read mode
-  protected String entityId;
-  // used only for multiple entity read mode
-  protected Long limit;
-  protected Long createdTimeBegin;
-  protected Long createdTimeEnd;
-  protected Long modifiedTimeBegin;
-  protected Long modifiedTimeEnd;
-  protected Map<String, Set<String>> relatesTo;
-  protected Map<String, Set<String>> isRelatedTo;
-  protected Map<String, Object> infoFilters;
-  protected Map<String, String> configFilters;
-  protected Set<String> metricFilters;
-  protected Set<String> eventFilters;
-  protected TimelineFilterList confsToRetrieve;
-  protected TimelineFilterList metricsToRetrieve;
-
-  /**
-   * Main table the entity reader uses.
-   */
-  protected BaseTable<?> table;
-
-  /**
-   * Specifies whether keys for this table are sorted in a manner where entities
-   * can be retrieved by created time. If true, it will be sufficient to collect
-   * the first results as specified by the limit. Otherwise all matched entities
-   * will be fetched and then limit applied.
-   */
-  private boolean sortedKeys = false;
-
-  /**
-   * Instantiates a reader for multiple-entity reads.
-   */
-  protected TimelineEntityReader(String userId, String clusterId,
-      String flowName, Long flowRunId, String appId, String entityType,
-      Long limit, Long createdTimeBegin, Long createdTimeEnd,
-      Long modifiedTimeBegin, Long modifiedTimeEnd,
-      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
-      Map<String, Object> infoFilters, Map<String, String> configFilters,
-      Set<String> metricFilters, Set<String> eventFilters,
-      TimelineFilterList confsToRetrieve, TimelineFilterList metricsToRetrieve,
-      EnumSet<Field> fieldsToRetrieve, boolean sortedKeys) {
-    this.singleEntityRead = false;
-    this.sortedKeys = sortedKeys;
-    this.userId = userId;
-    this.clusterId = clusterId;
-    this.flowName = flowName;
-    this.flowRunId = flowRunId;
-    this.appId = appId;
-    this.entityType = entityType;
-    this.fieldsToRetrieve = fieldsToRetrieve;
-    this.limit = limit;
-    this.createdTimeBegin = createdTimeBegin;
-    this.createdTimeEnd = createdTimeEnd;
-    this.modifiedTimeBegin = modifiedTimeBegin;
-    this.modifiedTimeEnd = modifiedTimeEnd;
-    this.relatesTo = relatesTo;
-    this.isRelatedTo = isRelatedTo;
-    this.infoFilters = infoFilters;
-    this.configFilters = configFilters;
-    this.metricFilters = metricFilters;
-    this.eventFilters = eventFilters;
-    this.confsToRetrieve = confsToRetrieve;
-    this.metricsToRetrieve = metricsToRetrieve;
-
-    this.table = getTable();
-  }
-
-  /**
-   * Instantiates a reader for single-entity reads.
-   */
-  protected TimelineEntityReader(String userId, String clusterId,
-      String flowName, Long flowRunId, String appId, String entityType,
-      String entityId, TimelineFilterList confsToRetrieve,
-      TimelineFilterList metricsToRetrieve, EnumSet<Field> fieldsToRetrieve) {
-    this.singleEntityRead = true;
-    this.userId = userId;
-    this.clusterId = clusterId;
-    this.flowName = flowName;
-    this.flowRunId = flowRunId;
-    this.appId = appId;
-    this.entityType = entityType;
-    this.fieldsToRetrieve = fieldsToRetrieve;
-    this.entityId = entityId;
-    this.confsToRetrieve = confsToRetrieve;
-    this.metricsToRetrieve = metricsToRetrieve;
-
-    this.table = getTable();
-  }
-
-  /**
-   * Creates a {@link FilterList} based on fields, confs and metrics to
-   * retrieve. This filter list will be set in Scan/Get objects to trim down
-   * results fetched from HBase back-end storage.
-   * @return a {@link FilterList} object.
-   */
-  protected abstract FilterList constructFilterListBasedOnFields();
-
-  /**
-   * Reads and deserializes a single timeline entity from the HBase storage.
-   */
-  public TimelineEntity readEntity(Configuration hbaseConf, Connection conn)
-      throws IOException {
-    validateParams();
-    augmentParams(hbaseConf, conn);
-
-    FilterList filterList = constructFilterListBasedOnFields();
-    Result result = getResult(hbaseConf, conn, filterList);
-    if (result == null || result.isEmpty()) {
-      // Could not find a matching row.
-      LOG.info("Cannot find matching entity of type " + entityType);
-      return null;
-    }
-    return parseEntity(result);
-  }
-
-  /**
-   * Reads and deserializes a set of timeline entities from the HBase storage.
-   * It goes through all the results available, and returns the number of
-   * entries as specified in the limit in the entity's natural sort order.
-   */
-  public Set<TimelineEntity> readEntities(Configuration hbaseConf,
-      Connection conn) throws IOException {
-    validateParams();
-    augmentParams(hbaseConf, conn);
-
-    NavigableSet<TimelineEntity> entities = new TreeSet<>();
-    FilterList filterList = constructFilterListBasedOnFields();
-    ResultScanner results = getResults(hbaseConf, conn, filterList);
-    try {
-      for (Result result : results) {
-        TimelineEntity entity = parseEntity(result);
-        if (entity == null) {
-          continue;
-        }
-        entities.add(entity);
-        if (!sortedKeys) {
-          if (entities.size() > limit) {
-            entities.pollLast();
-          }
-        } else {
-          if (entities.size() == limit) {
-            break;
-          }
-        }
-      }
-      return entities;
-    } finally {
-      results.close();
-    }
-  }
-
-  /**
-   * Returns the main table to be used by the entity reader.
-   */
-  protected abstract BaseTable<?> getTable();
-
-  /**
-   * Validates the required parameters to read the entities.
-   */
-  protected abstract void validateParams();
-
-  /**
-   * Sets certain parameters to defaults if the values are not provided.
-   */
-  protected abstract void augmentParams(Configuration hbaseConf,
-      Connection conn) throws IOException;
-
-  /**
-   * Fetches a {@link Result} instance for a single-entity read.
-   *
-   * @return the {@link Result} instance or null if no such record is found.
-   */
-  protected abstract Result getResult(Configuration hbaseConf, Connection conn,
-      FilterList filterList) throws IOException;
-
-  /**
-   * Fetches a {@link ResultScanner} for a multi-entity read.
-   */
-  protected abstract ResultScanner getResults(Configuration hbaseConf,
-      Connection conn, FilterList filterList) throws IOException;
-
-  /**
-   * Given a {@link Result} instance, deserializes and creates a
-   * {@link TimelineEntity}.
-   *
-   * @return the {@link TimelineEntity} instance, or null if the {@link Result}
-   * is null or empty.
-   */
-  protected abstract TimelineEntity parseEntity(Result result)
-      throws IOException;
-
-  /**
-   * Helper method for reading and deserializing {@link TimelineMetric} objects
-   * using the specified column prefix. The timeline metrics then are added to
-   * the given timeline entity.
-   */
-  protected void readMetrics(TimelineEntity entity, Result result,
-      ColumnPrefix<?> columnPrefix) throws IOException {
-    NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
-        columnPrefix.readResultsWithTimestamps(result);
-    for (Map.Entry<String, NavigableMap<Long, Number>> metricResult:
-        metricsResult.entrySet()) {
-      TimelineMetric metric = new TimelineMetric();
-      metric.setId(metricResult.getKey());
-      // Simply assume that if the value set contains more than 1 elements, the
-      // metric is a TIME_SERIES metric, otherwise, it's a SINGLE_VALUE metric
-      metric.setType(metricResult.getValue().size() > 1 ?
-          TimelineMetric.Type.TIME_SERIES : TimelineMetric.Type.SINGLE_VALUE);
-      metric.addValues(metricResult.getValue());
-      entity.addMetric(metric);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/63d90992/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.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/TimelineEntityReaderFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.java
deleted file mode 100644
index 36ed4ca..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineEntityReaderFactory.java
+++ /dev/null
@@ -1,100 +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;
-
-import java.util.EnumSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
-import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
-
-/**
- * Factory methods for instantiating a timeline entity reader.
- */
-class TimelineEntityReaderFactory {
-  /**
-   * Creates a timeline entity reader instance for reading a single entity with
-   * the specified input.
-   */
-  public static TimelineEntityReader createSingleEntityReader(String userId,
-      String clusterId, String flowName, Long flowRunId, String appId,
-      String entityType, String entityId, TimelineFilterList confs,
-      TimelineFilterList metrics, EnumSet<Field> fieldsToRetrieve) {
-    // currently the types that are handled separate from the generic entity
-    // table are application, flow run, and flow activity entities
-    if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) {
-      return new ApplicationEntityReader(userId, clusterId, flowName, flowRunId,
-          appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
-    } else if (TimelineEntityType.YARN_FLOW_RUN.matches(entityType)) {
-      return new FlowRunEntityReader(userId, clusterId, flowName, flowRunId,
-          appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
-    } else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) {
-      return new FlowActivityEntityReader(userId, clusterId, flowName, flowRunId,
-          appId, entityType, entityId, fieldsToRetrieve);
-    } else {
-      // assume we're dealing with a generic entity read
-      return new GenericEntityReader(userId, clusterId, flowName, flowRunId,
-        appId, entityType, entityId, confs, metrics, fieldsToRetrieve);
-    }
-  }
-
-  /**
-   * Creates a timeline entity reader instance for reading set of entities with
-   * the specified input and predicates.
-   */
-  public static TimelineEntityReader createMultipleEntitiesReader(String userId,
-      String clusterId, String flowName, Long flowRunId, String appId,
-      String entityType, Long limit, Long createdTimeBegin, Long createdTimeEnd,
-      Long modifiedTimeBegin, Long modifiedTimeEnd,
-      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
-      Map<String, Object> infoFilters, Map<String, String> configFilters,
-      Set<String> metricFilters, Set<String> eventFilters,
-      TimelineFilterList confs, TimelineFilterList metrics,
-      EnumSet<Field> fieldsToRetrieve) {
-    // currently the types that are handled separate from the generic entity
-    // table are application, flow run, and flow activity entities
-    if (TimelineEntityType.YARN_APPLICATION.matches(entityType)) {
-      return new ApplicationEntityReader(userId, clusterId, flowName, flowRunId,
-          appId, entityType, limit, createdTimeBegin, createdTimeEnd,
-          modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
-          infoFilters, configFilters, metricFilters, eventFilters, confs,
-          metrics, fieldsToRetrieve);
-    } else if (TimelineEntityType.YARN_FLOW_ACTIVITY.matches(entityType)) {
-      return new FlowActivityEntityReader(userId, clusterId, flowName, flowRunId,
-          appId, entityType, limit, createdTimeBegin, createdTimeEnd,
-          modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
-          infoFilters, configFilters, metricFilters, eventFilters,
-          fieldsToRetrieve);
-    } else if (TimelineEntityType.YARN_FLOW_RUN.matches(entityType)) {
-      return new FlowRunEntityReader(userId, clusterId, flowName, flowRunId,
-          appId, entityType, limit, createdTimeBegin, createdTimeEnd,
-          modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
-          infoFilters, configFilters, metricFilters, eventFilters, confs,
-          metrics, fieldsToRetrieve);
-    } else {
-      // assume we're dealing with a generic entity read
-      return new GenericEntityReader(userId, clusterId, flowName, flowRunId,
-          appId, entityType, limit, createdTimeBegin, createdTimeEnd,
-          modifiedTimeBegin, modifiedTimeEnd, relatesTo, isRelatedTo,
-          infoFilters, configFilters, metricFilters, eventFilters, confs,
-          metrics, fieldsToRetrieve, false);
-    }
-  }
-}


[34/50] [abbrv] hadoop git commit: YARN-3862. Support for fetching specific configs and metrics based on prefixes (Varun Saxena via sjlee)

Posted by gt...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java
index 38c0f3f..21ddcc2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityColumnPrefix.java
@@ -83,6 +83,18 @@ public enum FlowActivityColumnPrefix implements ColumnPrefix<FlowActivityTable>
     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);
+  }
+
   public byte[] getColumnPrefixBytes() {
     return columnPrefixBytes.clone();
   }
@@ -112,8 +124,7 @@ public enum FlowActivityColumnPrefix implements ColumnPrefix<FlowActivityTable>
           + tableMutator.getName().getNameAsString());
     }
 
-    byte[] columnQualifier = ColumnHelper.getColumnQualifier(
-        this.columnPrefixBytes, qualifier);
+    byte[] columnQualifier = getColumnPrefixBytes(qualifier);
     Attribute[] combinedAttributes = TimelineStorageUtils.combineAttributes(
         attributes, this.aggOp);
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
@@ -233,8 +244,7 @@ public enum FlowActivityColumnPrefix implements ColumnPrefix<FlowActivityTable>
           + tableMutator.getName().getNameAsString());
     }
 
-    byte[] columnQualifier = ColumnHelper.getColumnQualifier(
-        this.columnPrefixBytes, qualifier);
+    byte[] columnQualifier = getColumnPrefixBytes(qualifier);
     Attribute[] combinedAttributes = TimelineStorageUtils.combineAttributes(
         attributes, this.aggOp);
     column.store(rowKey, tableMutator, columnQualifier, null, inputValue,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java
index eb055fe..e3bb52d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunColumnPrefix.java
@@ -89,8 +89,16 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
     return columnPrefixBytes.clone();
   }
 
-  public byte[] getColumnPrefixBytes(String qualifier) {
-    return ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
+  @Override
+  public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
+    return ColumnHelper.getColumnQualifier(
+        this.columnPrefixBytes, qualifierPrefix);
+  }
+
+  @Override
+  public byte[] getColumnPrefixBytes(String qualifierPrefix) {
+    return ColumnHelper.getColumnQualifier(
+        this.columnPrefixBytes, qualifierPrefix);
   }
 
   public byte[] getColumnFamilyBytes() {
@@ -121,8 +129,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
           + tableMutator.getName().getNameAsString());
     }
 
-    byte[] columnQualifier = ColumnHelper.getColumnQualifier(
-        this.columnPrefixBytes, qualifier);
+    byte[] columnQualifier = getColumnPrefixBytes(qualifier);
     Attribute[] combinedAttributes = TimelineStorageUtils.combineAttributes(
         attributes, this.aggOp);
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
@@ -149,8 +156,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
           + tableMutator.getName().getNameAsString());
     }
 
-    byte[] columnQualifier = ColumnHelper.getColumnQualifier(
-        this.columnPrefixBytes, qualifier);
+    byte[] columnQualifier = getColumnPrefixBytes(qualifier);
     Attribute[] combinedAttributes = TimelineStorageUtils.combineAttributes(
         attributes, this.aggOp);
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/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 4e23e49..e864d61 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
@@ -266,7 +266,7 @@ public class TestFileSystemTimelineReaderImpl {
     // only the id, created and modified time
     TimelineEntity result =
         reader.getEntity("user1", "cluster1", "flow1", 1L, "app1",
-            "app", "id_1", null);
+            "app", "id_1", null, null, null);
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
@@ -281,7 +281,7 @@ public class TestFileSystemTimelineReaderImpl {
     // Cluster and AppId should be enough to get an entity.
     TimelineEntity result =
         reader.getEntity(null, "cluster1", null, null, "app1",
-            "app", "id_1", null);
+            "app", "id_1", null, null, null);
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
@@ -298,7 +298,7 @@ public class TestFileSystemTimelineReaderImpl {
     // in app flow mapping csv has commas.
     TimelineEntity result =
         reader.getEntity(null, "cluster1", null, null, "app2",
-            "app", "id_5", null);
+            "app", "id_5", null, null, null);
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_5")).toString(),
         result.getIdentifier().toString());
@@ -311,7 +311,7 @@ public class TestFileSystemTimelineReaderImpl {
     // Specified fields in addition to default view will be returned.
     TimelineEntity result =
         reader.getEntity("user1", "cluster1", "flow1", 1L,
-            "app1", "app", "id_1",
+            "app1", "app", "id_1", null, null,
             EnumSet.of(Field.INFO, Field.CONFIGS, Field.METRICS));
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
@@ -329,8 +329,8 @@ public class TestFileSystemTimelineReaderImpl {
   public void testGetEntityAllFields() throws Exception {
     // All fields of TimelineEntity will be returned.
     TimelineEntity result =
-        reader.getEntity("user1", "cluster1", "flow1", 1L,
-            "app1", "app", "id_1", EnumSet.of(Field.ALL));
+        reader.getEntity("user1", "cluster1", "flow1", 1L, "app1", "app",
+        "id_1", null, null, EnumSet.of(Field.ALL));
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
@@ -347,7 +347,7 @@ public class TestFileSystemTimelineReaderImpl {
     Set<TimelineEntity> result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, null, null, null, null, null, null, null, null, null,
-            null, null);
+            null, null, null, null);
     // All 3 entities will be returned
     Assert.assertEquals(4, result.size());
   }
@@ -357,7 +357,7 @@ public class TestFileSystemTimelineReaderImpl {
     Set<TimelineEntity> result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             2L, null, null, null, null, null, null, null, null, null,
-            null, null);
+            null, null, null, null);
     Assert.assertEquals(2, result.size());
     // Needs to be rewritten once hashcode and equals for
     // TimelineEntity is implemented
@@ -371,7 +371,7 @@ public class TestFileSystemTimelineReaderImpl {
     result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             3L, null, null, null, null, null, null, null, null, null,
-                null, null);
+                null, null, null, null);
      // Even though 2 entities out of 4 have same created time, one entity
      // is left out due to limit
      Assert.assertEquals(3, result.size());
@@ -383,7 +383,7 @@ public class TestFileSystemTimelineReaderImpl {
     Set<TimelineEntity> result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, 1425016502030L, 1425016502060L, null, null, null, null, null,
-            null, null, null, null);
+            null, null, null, null, null, null);
     Assert.assertEquals(1, result.size());
     // Only one entity with ID id_4 should be returned.
     for (TimelineEntity entity : result) {
@@ -396,7 +396,7 @@ public class TestFileSystemTimelineReaderImpl {
     result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, null, 1425016502010L, null, null, null, null, null, null,
-            null, null, null);
+            null, null, null, null, null);
     Assert.assertEquals(3, result.size());
     for (TimelineEntity entity : result) {
       if (entity.getId().equals("id_4")) {
@@ -408,7 +408,7 @@ public class TestFileSystemTimelineReaderImpl {
     result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, 1425016502010L, null, null, null, null, null, null, null,
-            null, null, null);
+            null, null, null, null, null);
     Assert.assertEquals(1, result.size());
     for (TimelineEntity entity : result) {
       if (!entity.getId().equals("id_4")) {
@@ -420,7 +420,7 @@ public class TestFileSystemTimelineReaderImpl {
     result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, null, null, 1425016502090L, 1425016503020L, null, null, null,
-            null, null, null, null);
+            null, null, null, null, null, null);
     Assert.assertEquals(2, result.size());
     // Two entities with IDs' id_1 and id_4 should be returned.
     for (TimelineEntity entity : result) {
@@ -433,7 +433,7 @@ public class TestFileSystemTimelineReaderImpl {
     result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, null, null, null, 1425016502090L, null, null, null, null,
-            null, null, null);
+            null, null, null, null, null);
     Assert.assertEquals(2, result.size());
     for (TimelineEntity entity : result) {
       if (entity.getId().equals("id_1") || entity.getId().equals("id_4")) {
@@ -445,7 +445,7 @@ public class TestFileSystemTimelineReaderImpl {
     result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, null, null, 1425016503005L, null, null, null, null, null,
-            null, null, null);
+            null, null, null, null, null);
     Assert.assertEquals(1, result.size());
     for (TimelineEntity entity : result) {
       if (!entity.getId().equals("id_4")) {
@@ -462,7 +462,7 @@ public class TestFileSystemTimelineReaderImpl {
     Set<TimelineEntity> result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, null, null, null, null, null, null, infoFilters, null, null,
-            null, null);
+            null, null, null, null);
     Assert.assertEquals(1, result.size());
     // Only one entity with ID id_3 should be returned.
     for (TimelineEntity entity : result) {
@@ -478,7 +478,7 @@ public class TestFileSystemTimelineReaderImpl {
     result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, null, null, null, null, null, null, null, configFilters, null,
-            null, null);
+            null, null, null, null);
     Assert.assertEquals(2, result.size());
     for (TimelineEntity entity : result) {
       if (!entity.getId().equals("id_1") && !entity.getId().equals("id_3")) {
@@ -493,7 +493,7 @@ public class TestFileSystemTimelineReaderImpl {
     result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, null, null, null, null, null, null, null, null, null,
-            eventFilters, null);
+            eventFilters, null, null, null);
     Assert.assertEquals(1, result.size());
     for (TimelineEntity entity : result) {
       if (!entity.getId().equals("id_3")) {
@@ -507,7 +507,7 @@ public class TestFileSystemTimelineReaderImpl {
     result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, null, null, null, null, null, null, null, null, metricFilters,
-            null, null);
+            null, null, null, null);
     Assert.assertEquals(2, result.size());
     // Two entities with IDs' id_1 and id_2 should be returned.
     for (TimelineEntity entity : result) {
@@ -527,7 +527,7 @@ public class TestFileSystemTimelineReaderImpl {
     Set<TimelineEntity> result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, null, null, null, null, relatesTo, null, null, null, null,
-            null, null);
+            null, null, null, null);
     Assert.assertEquals(1, result.size());
     // Only one entity with ID id_1 should be returned.
     for (TimelineEntity entity : result) {
@@ -544,7 +544,7 @@ public class TestFileSystemTimelineReaderImpl {
     result =
         reader.getEntities("user1", "cluster1", "flow1", 1L, "app1", "app",
             null, null, null, null, null, null, isRelatedTo, null, null, null,
-            null, null);
+            null, null, null, null);
     Assert.assertEquals(2, result.size());
     // Two entities with IDs' id_1 and id_3 should be returned.
     for (TimelineEntity entity : result) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.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/TestHBaseTimelineStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index 30ead40..bc7b3a4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -49,6 +49,11 @@ 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.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 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;
@@ -60,11 +65,17 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
+
 /**
  * Various tests to test writing entities to HBase and reading them back from
  * it.
@@ -79,18 +90,344 @@ import org.junit.Test;
 public class TestHBaseTimelineStorage {
 
   private static HBaseTestingUtility util;
+  private HBaseTimelineReaderImpl reader;
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
     util = new HBaseTestingUtility();
     util.startMiniCluster();
     createSchema();
+    loadEntities();
+    loadApps();
   }
 
   private static void createSchema() throws IOException {
     TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
   }
 
+  private static void loadApps() throws IOException {
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entity = new TimelineEntity();
+    String id = "application_1111111111_2222";
+    entity.setId(id);
+    entity.setType(TimelineEntityType.YARN_APPLICATION.toString());
+    Long cTime = 1425016501000L;
+    Long mTime = 1425026901000L;
+    entity.setCreatedTime(cTime);
+    entity.setModifiedTime(mTime);
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<String, Object>();
+    infoMap.put("infoMapKey1", "infoMapValue1");
+    infoMap.put("infoMapKey2", 10);
+    entity.addInfo(infoMap);
+    // add the isRelatedToEntity info
+    String key = "task";
+    String value = "is_related_to_entity_id_here";
+    Set<String> isRelatedToSet = new HashSet<String>();
+    isRelatedToSet.add(value);
+    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
+    isRelatedTo.put(key, isRelatedToSet);
+    entity.setIsRelatedToEntities(isRelatedTo);
+    // add the relatesTo info
+    key = "container";
+    value = "relates_to_entity_id_here";
+    Set<String> relatesToSet = new HashSet<String>();
+    relatesToSet.add(value);
+    value = "relates_to_entity_id_here_Second";
+    relatesToSet.add(value);
+    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
+    relatesTo.put(key, relatesToSet);
+    entity.setRelatesToEntities(relatesTo);
+    // add some config entries
+    Map<String, String> conf = new HashMap<String, String>();
+    conf.put("config_param1", "value1");
+    conf.put("config_param2", "value2");
+    conf.put("cfg_param1", "value3");
+    entity.addConfigs(conf);
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    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);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+
+    TimelineMetric m12 = new TimelineMetric();
+    m12.setId("MAP1_BYTES");
+    m12.addValue(ts, 50);
+    metrics.add(m12);
+    entity.addMetrics(metrics);
+    TimelineEvent event = new TimelineEvent();
+    event.setId("event1");
+    event.setTimestamp(ts - 2000);
+    entity.addEvent(event);
+    te.addEntity(entity);
+
+    TimelineEntities te1 = new TimelineEntities();
+    TimelineEntity entity1 = new TimelineEntity();
+    String id1 = "application_1111111111_3333";
+    entity1.setId(id1);
+    entity1.setType(TimelineEntityType.YARN_APPLICATION.toString());
+    entity1.setCreatedTime(cTime);
+    entity1.setModifiedTime(mTime);
+
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap1 = new HashMap<String, Object>();
+    infoMap1.put("infoMapKey1", "infoMapValue1");
+    infoMap1.put("infoMapKey2", 10);
+    entity1.addInfo(infoMap1);
+
+    // add the isRelatedToEntity info
+    String key1 = "task";
+    String value1 = "is_related_to_entity_id_here";
+    Set<String> isRelatedToSet1 = new HashSet<String>();
+    isRelatedToSet1.add(value1);
+    Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
+    isRelatedTo1.put(key, isRelatedToSet1);
+    entity1.setIsRelatedToEntities(isRelatedTo1);
+
+    // add the relatesTo info
+    key1 = "container";
+    value1 = "relates_to_entity_id_here";
+    Set<String> relatesToSet1 = new HashSet<String>();
+    relatesToSet1.add(value1);
+    value1 = "relates_to_entity_id_here_Second";
+    relatesToSet1.add(value1);
+    Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
+    relatesTo1.put(key1, relatesToSet1);
+    entity1.setRelatesToEntities(relatesTo1);
+
+    // add some config entries
+    Map<String, String> conf1 = new HashMap<String, String>();
+    conf1.put("cfg_param1", "value1");
+    conf1.put("cfg_param2", "value2");
+    entity1.addConfigs(conf1);
+
+    // add metrics
+    Set<TimelineMetric> metrics1 = new HashSet<>();
+    TimelineMetric m2 = new TimelineMetric();
+    m2.setId("MAP1_SLOT_MILLIS");
+    Map<Long, Number> metricValues1 = new HashMap<Long, Number>();
+    long ts1 = System.currentTimeMillis();
+    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);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues1);
+    metrics1.add(m2);
+    entity1.addMetrics(metrics1);
+    te1.addEntity(entity1);
+
+    TimelineEntities te2 = new TimelineEntities();
+    TimelineEntity entity2 = new TimelineEntity();
+    String id2 = "application_1111111111_4444";
+    entity2.setId(id2);
+    entity2.setType(TimelineEntityType.YARN_APPLICATION.toString());
+    entity2.setCreatedTime(cTime);
+    entity2.setModifiedTime(mTime);
+    te2.addEntity(entity2);
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      hbi = new HBaseTimelineWriterImpl(util.getConfiguration());
+      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);
+      hbi.stop();
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
+  private static void loadEntities() throws IOException {
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entity = new TimelineEntity();
+    String id = "hello";
+    String type = "world";
+    entity.setId(id);
+    entity.setType(type);
+    Long cTime = 1425016501000L;
+    Long mTime = 1425026901000L;
+    entity.setCreatedTime(cTime);
+    entity.setModifiedTime(mTime);
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<String, Object>();
+    infoMap.put("infoMapKey1", "infoMapValue1");
+    infoMap.put("infoMapKey2", 10);
+    entity.addInfo(infoMap);
+    // add the isRelatedToEntity info
+    String key = "task";
+    String value = "is_related_to_entity_id_here";
+    Set<String> isRelatedToSet = new HashSet<String>();
+    isRelatedToSet.add(value);
+    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
+    isRelatedTo.put(key, isRelatedToSet);
+    entity.setIsRelatedToEntities(isRelatedTo);
+
+    // add the relatesTo info
+    key = "container";
+    value = "relates_to_entity_id_here";
+    Set<String> relatesToSet = new HashSet<String>();
+    relatesToSet.add(value);
+    value = "relates_to_entity_id_here_Second";
+    relatesToSet.add(value);
+    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
+    relatesTo.put(key, relatesToSet);
+    entity.setRelatesToEntities(relatesTo);
+
+    // add some config entries
+    Map<String, String> conf = new HashMap<String, String>();
+    conf.put("config_param1", "value1");
+    conf.put("config_param2", "value2");
+    conf.put("cfg_param1", "value3");
+    entity.addConfigs(conf);
+
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    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);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+
+    TimelineMetric m12 = new TimelineMetric();
+    m12.setId("MAP1_BYTES");
+    m12.addValue(ts, 50);
+    metrics.add(m12);
+    entity.addMetrics(metrics);
+    te.addEntity(entity);
+
+    TimelineEntity entity1 = new TimelineEntity();
+    String id1 = "hello1";
+    entity1.setId(id1);
+    entity1.setType(type);
+    entity1.setCreatedTime(cTime);
+    entity1.setModifiedTime(mTime);
+
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap1 = new HashMap<String, Object>();
+    infoMap1.put("infoMapKey1", "infoMapValue1");
+    infoMap1.put("infoMapKey2", 10);
+    entity1.addInfo(infoMap1);
+
+    // add the isRelatedToEntity info
+    String key1 = "task";
+    String value1 = "is_related_to_entity_id_here";
+    Set<String> isRelatedToSet1 = new HashSet<String>();
+    isRelatedToSet1.add(value1);
+    Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
+    isRelatedTo1.put(key, isRelatedToSet1);
+    entity1.setIsRelatedToEntities(isRelatedTo1);
+
+    // add the relatesTo info
+    key1 = "container";
+    value1 = "relates_to_entity_id_here";
+    Set<String> relatesToSet1 = new HashSet<String>();
+    relatesToSet1.add(value1);
+    value1 = "relates_to_entity_id_here_Second";
+    relatesToSet1.add(value1);
+    Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
+    relatesTo1.put(key1, relatesToSet1);
+    entity1.setRelatesToEntities(relatesTo1);
+
+    // add some config entries
+    Map<String, String> conf1 = new HashMap<String, String>();
+    conf1.put("cfg_param1", "value1");
+    conf1.put("cfg_param2", "value2");
+    entity1.addConfigs(conf1);
+
+    // add metrics
+    Set<TimelineMetric> metrics1 = new HashSet<>();
+    TimelineMetric m2 = new TimelineMetric();
+    m2.setId("MAP1_SLOT_MILLIS");
+    Map<Long, Number> metricValues1 = new HashMap<Long, Number>();
+    long ts1 = System.currentTimeMillis();
+    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);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues1);
+    metrics1.add(m2);
+    entity1.addMetrics(metrics1);
+    te.addEntity(entity1);
+
+    TimelineEntity entity2 = new TimelineEntity();
+    String id2 = "hello2";
+    entity2.setId(id2);
+    entity2.setType(type);
+    entity2.setCreatedTime(cTime);
+    entity2.setModifiedTime(mTime);
+    te.addEntity(entity2);
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+        hbi = new HBaseTimelineWriterImpl(util.getConfiguration());
+        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_1231111111_1111";
+        hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+        hbi.stop();
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
+  @Before
+  public void init() throws Exception {
+    reader = new HBaseTimelineReaderImpl();
+    reader.init(util.getConfiguration());
+    reader.start();
+  }
+
+  @After
+  public void stop() throws Exception {
+    if (reader != null) {
+      reader.stop();
+      reader.close();
+    }
+  }
+
   private static void matchMetrics(Map<Long, Number> m1, Map<Long, Number> m2) {
     assertEquals(m1.size(), m2.size());
     for (Map.Entry<Long, Number> entry : m2.entrySet()) {
@@ -163,15 +500,11 @@ public class TestHBaseTimelineStorage {
     te.addEntity(entity);
 
     HBaseTimelineWriterImpl hbi = null;
-    HBaseTimelineReaderImpl hbr = null;
     try {
       Configuration c1 = util.getConfiguration();
       hbi = new HBaseTimelineWriterImpl(c1);
       hbi.init(c1);
       hbi.start();
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
       String cluster = "cluster_test_write_app";
       String user = "user1";
       String flow = "some_flow_name";
@@ -256,8 +589,8 @@ public class TestHBaseTimelineStorage {
       matchMetrics(metricValues, metricMap);
 
       // read the timeline entity using the reader this time
-      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appId,
-          entity.getType(), entity.getId(),
+      TimelineEntity e1 = reader.getEntity(user, cluster, flow, runid, appId,
+          entity.getType(), entity.getId(), null, null,
           EnumSet.of(TimelineReader.Field.ALL));
       assertNotNull(e1);
 
@@ -290,10 +623,6 @@ public class TestHBaseTimelineStorage {
         hbi.stop();
         hbi.close();
       }
-      if (hbr != null) {
-        hbr.stop();
-        hbr.close();
-      }
     }
   }
 
@@ -362,15 +691,11 @@ public class TestHBaseTimelineStorage {
     te.addEntity(entity);
 
     HBaseTimelineWriterImpl hbi = null;
-    HBaseTimelineReaderImpl hbr = null;
     try {
       Configuration c1 = util.getConfiguration();
       hbi = new HBaseTimelineWriterImpl(c1);
       hbi.init(c1);
       hbi.start();
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
       String cluster = "cluster_test_write_entity";
       String user = "user1";
       String flow = "some_flow_name";
@@ -468,12 +793,13 @@ public class TestHBaseTimelineStorage {
       assertEquals(17, colCount);
 
       // read the timeline entity using the reader this time
-      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
-          entity.getType(), entity.getId(),
+      TimelineEntity e1 = reader.getEntity(user, cluster, flow, runid, appName,
+          entity.getType(), entity.getId(), null, null,
           EnumSet.of(TimelineReader.Field.ALL));
-      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
+      Set<TimelineEntity> es1 = reader.getEntities(user, cluster, flow, runid,
           appName, entity.getType(), null, null, null, null, null, null, null,
-          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
+          null, null, null, null, null, null,
+          EnumSet.of(TimelineReader.Field.ALL));
       assertNotNull(e1);
       assertEquals(1, es1.size());
 
@@ -505,10 +831,6 @@ public class TestHBaseTimelineStorage {
         hbi.stop();
         hbi.close();
       }
-      if (hbr != null) {
-        hbr.stop();
-        hbr.close();
-      }
     }
   }
 
@@ -559,15 +881,11 @@ public class TestHBaseTimelineStorage {
     entities.addEntity(entity);
 
     HBaseTimelineWriterImpl hbi = null;
-    HBaseTimelineReaderImpl hbr = null;
     try {
       Configuration c1 = util.getConfiguration();
       hbi = new HBaseTimelineWriterImpl(c1);
       hbi.init(c1);
       hbi.start();
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
       String cluster = "cluster_test_events";
       String user = "user2";
       String flow = "other_flow_name";
@@ -612,11 +930,11 @@ public class TestHBaseTimelineStorage {
       }
 
       // read the timeline entity using the reader this time
-      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
-          entity.getType(), entity.getId(),
+      TimelineEntity e1 = reader.getEntity(user, cluster, flow, runid, appName,
+          entity.getType(), entity.getId(), null, null,
           EnumSet.of(TimelineReader.Field.ALL));
-      TimelineEntity e2 = hbr.getEntity(user, cluster, null, null, appName,
-          entity.getType(), entity.getId(),
+      TimelineEntity e2 = reader.getEntity(user, cluster, null, null, appName,
+          entity.getType(), entity.getId(), null, null,
           EnumSet.of(TimelineReader.Field.ALL));
       assertNotNull(e1);
       assertNotNull(e2);
@@ -641,10 +959,6 @@ public class TestHBaseTimelineStorage {
         hbi.stop();
         hbi.close();
       }
-      if (hbr != null) {
-        hbr.stop();
-        hbr.close();
-      }
     }
   }
 
@@ -665,15 +979,11 @@ public class TestHBaseTimelineStorage {
     entities.addEntity(entity);
 
     HBaseTimelineWriterImpl hbi = null;
-    HBaseTimelineReaderImpl hbr = null;
     try {
       Configuration c1 = util.getConfiguration();
       hbi = new HBaseTimelineWriterImpl(c1);
       hbi.init(c1);
       hbi.start();
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
       String cluster = "cluster_test_empty_eventkey";
       String user = "user_emptyeventkey";
       String flow = "other_flow_name";
@@ -726,12 +1036,13 @@ public class TestHBaseTimelineStorage {
       assertEquals(1, rowCount);
 
       // read the timeline entity using the reader this time
-      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
-          entity.getType(), entity.getId(),
+      TimelineEntity e1 = reader.getEntity(user, cluster, flow, runid, appName,
+          entity.getType(), entity.getId(), null, null,
           EnumSet.of(TimelineReader.Field.ALL));
-      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
+      Set<TimelineEntity> es1 = reader.getEntities(user, cluster, flow, runid,
           appName, entity.getType(), null, null, null, null, null, null, null,
-          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
+          null, null, null, null, null, null,
+          EnumSet.of(TimelineReader.Field.ALL));
       assertNotNull(e1);
       assertEquals(1, es1.size());
 
@@ -748,8 +1059,6 @@ public class TestHBaseTimelineStorage {
     } finally {
       hbi.stop();
       hbi.close();
-      hbr.stop();;
-      hbr.close();
     }
   }
 
@@ -816,6 +1125,291 @@ public class TestHBaseTimelineStorage {
     }
   }
 
+  @Test
+  public void testReadEntities() throws Exception {
+    TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", "hello", null,
+        null, EnumSet.of(Field.ALL));
+    assertNotNull(e1);
+    assertEquals(3, e1.getConfigs().size());
+    assertEquals(1, e1.getIsRelatedToEntities().size());
+    Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, "application_1231111111_1111","world",
+        null, null, null, null, null, null, null, null, null, null, null, null,
+        null, EnumSet.of(Field.ALL));
+    assertEquals(3, es1.size());
+  }
+
+  @Test
+  public void testReadEntitiesDefaultView() throws Exception {
+    TimelineEntity e1 =
+        reader.getEntity("user1", "cluster1", "some_flow_name", 1002345678919L,
+        "application_1231111111_1111","world", "hello", null, null, null);
+    assertNotNull(e1);
+    assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() &&
+        e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() &&
+        e1.getRelatesToEntities().isEmpty());
+    Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, "application_1231111111_1111","world",
+        null, null, null, null, null, null, null, null, null, null, null, null,
+        null, null);
+    assertEquals(3, es1.size());
+    for (TimelineEntity e : es1) {
+      assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() &&
+          e.getMetrics().isEmpty() && e.getIsRelatedToEntities().isEmpty() &&
+          e.getRelatesToEntities().isEmpty());
+    }
+  }
+
+  @Test
+  public void testReadEntitiesByFields() throws Exception {
+    TimelineEntity e1 =
+        reader.getEntity("user1", "cluster1", "some_flow_name", 1002345678919L,
+        "application_1231111111_1111","world", "hello", null, null,
+        EnumSet.of(Field.INFO, Field.CONFIGS));
+    assertNotNull(e1);
+    assertEquals(3, e1.getConfigs().size());
+    assertEquals(0, e1.getIsRelatedToEntities().size());
+    Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, "application_1231111111_1111","world",
+        null, null, null, null, null, null, null, null, null, null, null, null,
+        null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS));
+    assertEquals(3, es1.size());
+    int metricsCnt = 0;
+    int isRelatedToCnt = 0;
+    int infoCnt = 0;
+    for (TimelineEntity entity : es1) {
+      metricsCnt += entity.getMetrics().size();
+      isRelatedToCnt += entity.getIsRelatedToEntities().size();
+      infoCnt += entity.getInfo().size();
+    }
+    assertEquals(0, infoCnt);
+    assertEquals(2, isRelatedToCnt);
+    assertEquals(3, metricsCnt);
+  }
+
+  @Test
+  public void testReadEntitiesConfigPrefix() throws Exception {
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
+    TimelineEntity e1 =
+        reader.getEntity("user1", "cluster1", "some_flow_name", 1002345678919L,
+        "application_1231111111_1111","world", "hello", list, null, null);
+    assertNotNull(e1);
+    assertEquals(1, e1.getConfigs().size());
+    Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, "application_1231111111_1111","world",
+        null, null, null, null, null, null, null, null, null, null, null,
+        list, null, null);
+    int cfgCnt = 0;
+    for (TimelineEntity entity : es1) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(3, cfgCnt);
+  }
+
+  @Test
+  public void testReadEntitiesConfigFilterPrefix() throws Exception {
+    Map<String, String> confFilters = ImmutableMap.of("cfg_param1","value1");
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
+    Set<TimelineEntity> entities = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, "application_1231111111_1111","world",
+        null, null, null, null, null, null, null, null, confFilters, null, null,
+        list, null, null);
+    assertEquals(1, entities.size());
+    int cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(2, cfgCnt);
+  }
+
+  @Test
+  public void testReadEntitiesMetricPrefix() throws Exception {
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    TimelineEntity e1 =
+        reader.getEntity("user1", "cluster1", "some_flow_name", 1002345678919L,
+        "application_1231111111_1111","world", "hello", null, list, null);
+    assertNotNull(e1);
+    assertEquals(1, e1.getMetrics().size());
+    Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, "application_1231111111_1111","world",
+        null, null, null, null, null, null, null, null, null, null, null, null,
+        list, null);
+    int metricCnt = 0;
+    for (TimelineEntity entity : es1) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(2, metricCnt);
+  }
+
+  @Test
+  public void testReadEntitiesMetricFilterPrefix() throws Exception {
+    Set<String> metricFilters = ImmutableSet.of("MAP1_SLOT_MILLIS");
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    Set<TimelineEntity> entities = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, "application_1231111111_1111","world",
+        null, null, null, null, null, null, null, null, null, metricFilters,
+        null, null, list, null);
+    assertEquals(1, entities.size());
+    int metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(1, metricCnt);
+  }
+
+  @Test
+  public void testReadApps() throws Exception {
+    TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name",
+        1002345678919L, "application_1111111111_2222",
+        TimelineEntityType.YARN_APPLICATION.toString(), null, null, null,
+        EnumSet.of(Field.ALL));
+    assertNotNull(e1);
+    assertEquals(3, e1.getConfigs().size());
+    assertEquals(1, e1.getIsRelatedToEntities().size());
+    Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null,
+        null, null, null, null, null, null, null, null, null,
+        EnumSet.of(Field.ALL));
+    assertEquals(3, es1.size());
+  }
+
+  @Test
+  public void testReadAppsDefaultView() throws Exception {
+    TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name",
+        1002345678919L, "application_1111111111_2222",
+        TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null);
+    assertNotNull(e1);
+    assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() &&
+        e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() &&
+        e1.getRelatesToEntities().isEmpty());
+    Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null,
+        null, null, null, null, null, null, null, null, null, null);
+    assertEquals(3, es1.size());
+    for (TimelineEntity e : es1) {
+      assertTrue(e.getInfo().isEmpty() && e.getConfigs().isEmpty() &&
+          e.getMetrics().isEmpty() && e.getIsRelatedToEntities().isEmpty() &&
+          e.getRelatesToEntities().isEmpty());
+    }
+  }
+
+  @Test
+  public void testReadAppsByFields() throws Exception {
+    TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name",
+        1002345678919L, "application_1111111111_2222",
+        TimelineEntityType.YARN_APPLICATION.toString(), null, null, null,
+        EnumSet.of(Field.INFO, Field.CONFIGS));
+    assertNotNull(e1);
+    assertEquals(3, e1.getConfigs().size());
+    assertEquals(0, e1.getIsRelatedToEntities().size());
+    Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null,
+        null, null, null, null, null, null, null, null, null,
+        EnumSet.of(Field.IS_RELATED_TO, Field.METRICS));
+    assertEquals(3, es1.size());
+    int metricsCnt = 0;
+    int isRelatedToCnt = 0;
+    int infoCnt = 0;
+    for (TimelineEntity entity : es1) {
+      metricsCnt += entity.getMetrics().size();
+      isRelatedToCnt += entity.getIsRelatedToEntities().size();
+      infoCnt += entity.getInfo().size();
+    }
+    assertEquals(0, infoCnt);
+    assertEquals(2, isRelatedToCnt);
+    assertEquals(3, metricsCnt);
+  }
+
+  @Test
+  public void testReadAppsConfigPrefix() throws Exception {
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
+    TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name",
+        1002345678919L, "application_1111111111_2222",
+        TimelineEntityType.YARN_APPLICATION.toString(), null, list, null, null);
+    assertNotNull(e1);
+    assertEquals(1, e1.getConfigs().size());
+    Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null,
+        null, null, null, null, null, null, null, list, null, null);
+    int cfgCnt = 0;
+    for (TimelineEntity entity : es1) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(3, cfgCnt);
+  }
+
+  @Test
+  public void testReadAppsConfigFilterPrefix() throws Exception {
+    Map<String, String> confFilters = ImmutableMap.of("cfg_param1","value1");
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
+    Set<TimelineEntity> entities = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null,
+        null, null, null, null, confFilters, null, null, list, null, null);
+    assertEquals(1, entities.size());
+    int cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(2, cfgCnt);
+  }
+
+  @Test
+  public void testReadAppsMetricPrefix() throws Exception {
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    TimelineEntity e1 = reader.getEntity("user1", "cluster1", "some_flow_name",
+        1002345678919L, "application_1111111111_2222",
+        TimelineEntityType.YARN_APPLICATION.toString(), null, null, list, null);
+    assertNotNull(e1);
+    assertEquals(1, e1.getMetrics().size());
+    Set<TimelineEntity> es1 = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null,
+        null, null, null, null, null, null, null, null, list, null);
+    int metricCnt = 0;
+    for (TimelineEntity entity : es1) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(2, metricCnt);
+  }
+
+  @Test
+  public void testReadAppsMetricFilterPrefix() throws Exception {
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    Set<String> metricFilters = ImmutableSet.of("MAP1_SLOT_MILLIS");
+    Set<TimelineEntity> entities = reader.getEntities("user1", "cluster1",
+        "some_flow_name", 1002345678919L, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), null, null, null, null,
+        null, null, null, null, null, metricFilters, null, null, list, null);
+    int metricCnt = 0;
+    assertEquals(1, entities.size());
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(1, metricCnt);
+  }
+
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     util.shutdownMiniCluster();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
index c957dad..434adac 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
@@ -182,7 +182,7 @@ public class TestHBaseStorageFlowActivity {
       Set<TimelineEntity> entities =
           hbr.getEntities(null, cluster, null, null, null,
               TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), 10L, null, null,
-              null, null, null, null, null, null, null, null, null);
+              null, null, null, null, null, null, null, null, null, null, null);
       assertEquals(1, entities.size());
       for (TimelineEntity e : entities) {
         FlowActivityEntity flowActivity = (FlowActivityEntity)e;
@@ -238,7 +238,7 @@ public class TestHBaseStorageFlowActivity {
       Set<TimelineEntity> entities =
           hbr.getEntities(user, cluster, flow, null, null,
               TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), 10L, null, null,
-              null, null, null, null, null, null, null, null, null);
+              null, null, null, null, null, null, null, null, null, null, null);
       assertEquals(1, entities.size());
       for (TimelineEntity e : entities) {
         FlowActivityEntity entity = (FlowActivityEntity)e;
@@ -353,7 +353,7 @@ public class TestHBaseStorageFlowActivity {
       Set<TimelineEntity> entities =
           hbr.getEntities(null, cluster, null, null, null,
               TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), 10L, null, null,
-              null, null, null, null, null, null, null, null, null);
+              null, null, null, null, null, null, null, null, null, null, null);
       assertEquals(1, entities.size());
       for (TimelineEntity e : entities) {
         FlowActivityEntity flowActivity = (FlowActivityEntity)e;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76bc71cc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
index 4fb8f0e..5da0192 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.Map;
 import java.util.Set;
 
@@ -44,9 +45,13 @@ 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.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl;
 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.TimelineReader.Field;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -178,9 +183,8 @@ public class TestHBaseStorageFlowRun {
       hbr.init(c1);
       hbr.start();
       // get the flow run entity
-      TimelineEntity entity =
-          hbr.getEntity(user, cluster, flow, runid, null,
-              TimelineEntityType.YARN_FLOW_RUN.toString(), null, null);
+      TimelineEntity entity = hbr.getEntity(user, cluster, flow, runid, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null, null);
       assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
       FlowRunEntity flowRun = (FlowRunEntity)entity;
       assertEquals(minStartTs, flowRun.getStartTime());
@@ -238,9 +242,8 @@ public class TestHBaseStorageFlowRun {
       hbr = new HBaseTimelineReaderImpl();
       hbr.init(c1);
       hbr.start();
-      TimelineEntity entity =
-          hbr.getEntity(user, cluster, flow, runid, null,
-            TimelineEntityType.YARN_FLOW_RUN.toString(), null, null);
+      TimelineEntity entity = hbr.getEntity(user, cluster, flow, runid, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null, null);
       assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
       Set<TimelineMetric> metrics = entity.getMetrics();
       assertEquals(2, metrics.size());
@@ -305,6 +308,181 @@ public class TestHBaseStorageFlowRun {
     assertEquals(1, rowCount);
   }
 
+  @Test
+  public void testWriteFlowRunMetricsPrefix() throws Exception {
+    String cluster = "testWriteFlowRunMetricsOneFlow_cluster1";
+    String user = "testWriteFlowRunMetricsOneFlow_user1";
+    String flow = "testing_flowRun_metrics_flow_name";
+    String flowVersion = "CF7022C10F1354";
+    long runid = 1002345678919L;
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1();
+    te.addEntity(entityApp1);
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      String appName = "application_11111111111111_1111";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      // write another application with same metric to this flow
+      te = new TimelineEntities();
+      TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2();
+      te.addEntity(entityApp2);
+      appName = "application_11111111111111_2222";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.flush();
+    } finally {
+      hbi.close();
+    }
+
+    // check flow run
+    checkFlowRunTable(cluster, user, flow, runid, c1);
+
+    // use the timeline reader to verify data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+      TimelineFilterList metricsToRetrieve =
+          new TimelineFilterList(new TimelinePrefixFilter(TimelineCompareOp.EQUAL,
+          metric1.substring(0, metric1.indexOf("_") + 1)));
+      TimelineEntity entity = hbr.getEntity(user, cluster, flow, runid, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null, null,
+          metricsToRetrieve, null);
+      assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
+      Set<TimelineMetric> metrics = entity.getMetrics();
+      assertEquals(1, metrics.size());
+      for (TimelineMetric metric : metrics) {
+        String id = metric.getId();
+        Map<Long, Number> values = metric.getValues();
+        assertEquals(1, values.size());
+        Number value = null;
+        for (Number n : values.values()) {
+          value = n;
+        }
+        switch (id) {
+        case metric1:
+          assertEquals(141L, value);
+          break;
+        default:
+          fail("unrecognized metric: " + id);
+        }
+      }
+
+      Set<TimelineEntity> entities = hbr.getEntities(user, cluster, flow, runid,
+          null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null,
+          null, null, null, null, null, null, null, null, null,
+          metricsToRetrieve, null);
+      assertEquals(1, entities.size());
+      for (TimelineEntity timelineEntity : entities) {
+        Set<TimelineMetric> timelineMetrics = timelineEntity.getMetrics();
+        assertEquals(1, timelineMetrics.size());
+        for (TimelineMetric metric : timelineMetrics) {
+          String id = metric.getId();
+          Map<Long, Number> values = metric.getValues();
+          assertEquals(1, values.size());
+          Number value = null;
+          for (Number n : values.values()) {
+            value = n;
+          }
+          switch (id) {
+          case metric1:
+            assertEquals(141L, value);
+            break;
+          default:
+            fail("unrecognized metric: " + id);
+          }
+        }
+      }
+    } finally {
+      hbr.close();
+    }
+  }
+
+  @Test
+  public void testWriteFlowRunsMetricFields() throws Exception {
+    String cluster = "testWriteFlowRunMetricsOneFlow_cluster1";
+    String user = "testWriteFlowRunMetricsOneFlow_user1";
+    String flow = "testing_flowRun_metrics_flow_name";
+    String flowVersion = "CF7022C10F1354";
+    long runid = 1002345678919L;
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1();
+    te.addEntity(entityApp1);
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      String appName = "application_11111111111111_1111";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      // write another application with same metric to this flow
+      te = new TimelineEntities();
+      TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2();
+      te.addEntity(entityApp2);
+      appName = "application_11111111111111_2222";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.flush();
+    } finally {
+      hbi.close();
+    }
+
+    // check flow run
+    checkFlowRunTable(cluster, user, flow, runid, c1);
+
+    // use the timeline reader to verify data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+      Set<TimelineEntity> entities = hbr.getEntities(user, cluster, flow, runid,
+          null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null,
+          null, null, null, null, null, null, null, null, null, null, null);
+      assertEquals(1, entities.size());
+      for (TimelineEntity timelineEntity : entities) {
+        assertEquals(0, timelineEntity.getMetrics().size());
+      }
+
+      entities = hbr.getEntities(user, cluster, flow, runid,
+          null, TimelineEntityType.YARN_FLOW_RUN.toString(), null, null, null,
+          null, null, null, null, null, null, null, null, null,
+          null, EnumSet.of(Field.METRICS));
+      assertEquals(1, entities.size());
+      for (TimelineEntity timelineEntity : entities) {
+        Set<TimelineMetric> timelineMetrics = timelineEntity.getMetrics();
+        assertEquals(2, timelineMetrics.size());
+        for (TimelineMetric metric : timelineMetrics) {
+          String id = metric.getId();
+          Map<Long, Number> values = metric.getValues();
+          assertEquals(1, values.size());
+          Number value = null;
+          for (Number n : values.values()) {
+            value = n;
+          }
+          switch (id) {
+          case metric1:
+            assertEquals(141L, value);
+            break;
+          case metric2:
+            assertEquals(57L, value);
+            break;
+          default:
+            fail("unrecognized metric: " + id);
+          }
+        }
+      }
+    } finally {
+      hbr.close();
+    }
+  }
+
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     util.shutdownMiniCluster();


[05/50] [abbrv] hadoop git commit: YARN-3904. Refactor timelineservice.storage to add support to online and offline aggregation writers (Li Lu via sjlee)

Posted by gt...@apache.org.
YARN-3904. Refactor timelineservice.storage to add support to online and offline aggregation writers (Li Lu via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: a87a00ee68ee929b8a297c97dad0999e800a9f59
Parents: 557fd5e
Author: Sangjin Lee <sj...@apache.org>
Authored: Mon Aug 17 16:48:58 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:37:47 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../dev-support/findbugs-exclude.xml            |   7 +-
 .../hadoop/yarn/conf/YarnConfiguration.java     |  10 +
 .../storage/OfflineAggregationWriter.java       |  66 +++
 .../PhoenixOfflineAggregationWriterImpl.java    | 356 +++++++++++++
 .../storage/PhoenixTimelineWriterImpl.java      | 530 -------------------
 .../storage/TimelineSchemaCreator.java          |  45 +-
 .../storage/common/OfflineAggregationInfo.java  | 110 ++++
 ...TestPhoenixOfflineAggregationWriterImpl.java | 162 ++++++
 .../storage/TestPhoenixTimelineWriterImpl.java  | 152 ------
 .../storage/TestTimelineWriterImpl.java         |  74 ---
 11 files changed, 754 insertions(+), 761 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a87a00ee/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index c3a8172..42da97b 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -88,6 +88,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3906. Split the application table from the entity table. (Sangjin Lee 
     via junping_du)
 
+    YARN-3904. Refactor timelineservice.storage to add support to online and
+    offline aggregation writers (Li Lu via sjlee)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a87a00ee/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index 0dcdd15..d36f245 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -499,13 +499,12 @@
   <!-- Ignore SQL_PREPARED_STATEMENT_GENERATED_FROM_NONCONSTANT_STRING warnings for Timeline Phoenix storage. -->
   <!-- Since we're using dynamic columns, we have to generate SQL statements dynamically -->
   <Match>
-    <Class name="org.apache.hadoop.yarn.server.timelineservice.storage.PhoenixTimelineWriterImpl" />
+    <Class name="org.apache.hadoop.yarn.server.timelineservice.storage.PhoenixOfflineAggregationWriterImpl" />
     <Or>
       <Method name="storeEntityVariableLengthFields" />
-      <Method name="storeEvents" />
-      <Method name="storeMetrics" />
-      <Method name="write" />
+      <Method name="writeAggregatedEntity" />
     </Or>
+    <Bug pattern="SQL_PREPARED_STATEMENT_GENERATED_FROM_NONCONSTANT_STRING" />
   </Match>
   
   <!-- Following fields are used in ErrorsAndWarningsBlock, which is not a part of analysis of findbugs -->

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a87a00ee/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 b8281b0..2946240 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
@@ -2005,6 +2005,16 @@ public class YarnConfiguration extends Configuration {
   public static final long    DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME =
       7*24*60*60*1000; // 7 days
 
+  // Timeline service v2 offlien aggregation related keys
+  public static final String TIMELINE_OFFLINE_AGGREGATION_PREFIX =
+      YarnConfiguration.TIMELINE_SERVICE_PREFIX + "aggregation.offline.";
+  public static final String PHOENIX_OFFLINE_STORAGE_CONN_STR
+      = TIMELINE_OFFLINE_AGGREGATION_PREFIX
+          + "phoenix.connectionString";
+
+  public static final String PHOENIX_OFFLINE_STORAGE_CONN_STR_DEFAULT
+      = "jdbc:phoenix:localhost:2181:/hbase";
+
   // ///////////////////////////////
   // Shared Cache Configs
   // ///////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a87a00ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/OfflineAggregationWriter.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/OfflineAggregationWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/OfflineAggregationWriter.java
new file mode 100644
index 0000000..e1219e0
--- /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/OfflineAggregationWriter.java
@@ -0,0 +1,66 @@
+/**
+ * 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;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.OfflineAggregationInfo;
+
+import java.io.IOException;
+
+/**
+ * YARN timeline service v2 offline aggregation storage interface
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public abstract class OfflineAggregationWriter extends AbstractService {
+
+  /**
+   * Construct the offline writer.
+   *
+   * @param name service name
+   */
+  public OfflineAggregationWriter(String name) {
+    super(name);
+  }
+
+  /**
+   * Persist aggregated timeline entities to the offline store based on which
+   * track this entity is to be rolled up to. The tracks along which aggregations
+   * are to be done are given by {@link OfflineAggregationInfo}.
+   *
+   * @param context a {@link TimelineCollectorContext} object that describes the
+   *                context information of the aggregated data. Depends on the
+   *                type of the aggregation, some fields of this context maybe
+   *                empty or null.
+   * @param entities {@link TimelineEntities} to be persisted.
+   * @param info an {@link OfflineAggregationInfo} object that describes the
+   *             detail of the aggregation. Current supported option is
+   *             {@link OfflineAggregationInfo#FLOW_AGGREGATION}.
+   * @return a {@link TimelineWriteResponse} object.
+   * @throws IOException
+   */
+  abstract TimelineWriteResponse writeAggregatedEntity(
+      TimelineCollectorContext context,
+      TimelineEntities entities, OfflineAggregationInfo info) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a87a00ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixOfflineAggregationWriterImpl.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/PhoenixOfflineAggregationWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixOfflineAggregationWriterImpl.java
new file mode 100644
index 0000000..4c1352c
--- /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/PhoenixOfflineAggregationWriterImpl.java
@@ -0,0 +1,356 @@
+/**
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang.StringUtils;
+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.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.TimelineWriteResponse;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.OfflineAggregationInfo;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
+import org.apache.phoenix.util.PropertiesUtil;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Offline aggregation Phoenix storage. This storage currently consists of two
+ * aggregation tables, one for flow level aggregation and one for user level
+ * aggregation.
+ *
+ * Example table record:
+ *
+ * <pre>
+ * |---------------------------|
+ * |  Primary   | Column Family|
+ * |  key       | metrics      |
+ * |---------------------------|
+ * | row_key    | metricId1:   |
+ * |            | metricValue1 |
+ * |            | @timestamp1  |
+ * |            |              |
+ * |            | metriciD1:   |
+ * |            | metricValue2 |
+ * |            | @timestamp2  |
+ * |            |              |
+ * |            | metricId2:   |
+ * |            | metricValue1 |
+ * |            | @timestamp2  |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |            |              |
+ * |---------------------------|
+ * </pre>
+ *
+ * For the flow aggregation table, the primary key contains user, cluster, and
+ * flow id. For user aggregation table,the primary key is user.
+ *
+ * Metrics column family stores all aggregated metrics for each record.
+ */
+@Private
+@Unstable
+public class PhoenixOfflineAggregationWriterImpl
+    extends OfflineAggregationWriter {
+
+  private static final Log LOG
+      = LogFactory.getLog(PhoenixOfflineAggregationWriterImpl.class);
+  private static final String PHOENIX_COL_FAMILY_PLACE_HOLDER
+      = "timeline_cf_placeholder";
+
+  /** Default Phoenix JDBC driver name */
+  private static final String DRIVER_CLASS_NAME
+      = "org.apache.phoenix.jdbc.PhoenixDriver";
+
+  /** Default Phoenix timeline config column family */
+  private static final String METRIC_COLUMN_FAMILY = "m.";
+  /** Default Phoenix timeline info column family */
+  private static final String INFO_COLUMN_FAMILY = "i.";
+  /** Default separator for Phoenix storage */
+  private static final String AGGREGATION_STORAGE_SEPARATOR = ";";
+
+  /** Connection string to the deployed Phoenix cluster */
+  private String connString = null;
+  private Properties connProperties = new Properties();
+
+  public PhoenixOfflineAggregationWriterImpl(Properties prop) {
+    super(PhoenixOfflineAggregationWriterImpl.class.getName());
+    connProperties = PropertiesUtil.deepCopy(prop);
+  }
+
+  public PhoenixOfflineAggregationWriterImpl() {
+    super(PhoenixOfflineAggregationWriterImpl.class.getName());
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    Class.forName(DRIVER_CLASS_NAME);
+    // so check it here and only read in the config if it's not overridden.
+    connString =
+        conf.get(YarnConfiguration.PHOENIX_OFFLINE_STORAGE_CONN_STR,
+            YarnConfiguration.PHOENIX_OFFLINE_STORAGE_CONN_STR_DEFAULT);
+    super.init(conf);
+  }
+
+  @Override
+  public TimelineWriteResponse writeAggregatedEntity(
+      TimelineCollectorContext context, TimelineEntities entities,
+      OfflineAggregationInfo info) throws IOException {
+    TimelineWriteResponse response = new TimelineWriteResponse();
+    String sql = "UPSERT INTO " + info.getTableName()
+        + " (" + StringUtils.join(info.getPrimaryKeyList(), ",")
+        + ", created_time, modified_time, metric_names) "
+        + "VALUES ("
+        + StringUtils.repeat("?,", info.getPrimaryKeyList().length)
+        + "?, ?, ?)";
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("TimelineEntity write SQL: " + sql);
+    }
+
+    try (Connection conn = getConnection();
+        PreparedStatement ps = conn.prepareStatement(sql)) {
+      for (TimelineEntity entity : entities.getEntities()) {
+        HashMap<String, TimelineMetric> formattedMetrics = new HashMap<>();
+        if (entity.getMetrics() != null) {
+          for (TimelineMetric m : entity.getMetrics()) {
+            formattedMetrics.put(m.getId(), m);
+          }
+        }
+        int idx = info.setStringsForPrimaryKey(ps, context, null, 1);
+        ps.setLong(idx++, entity.getCreatedTime());
+        ps.setLong(idx++, entity.getModifiedTime());
+        ps.setString(idx++, StringUtils.join(formattedMetrics.keySet().toArray(),
+            AGGREGATION_STORAGE_SEPARATOR));
+        ps.execute();
+
+        storeEntityVariableLengthFields(entity, formattedMetrics, context, conn,
+            info);
+
+        conn.commit();
+      }
+    } catch (SQLException se) {
+      LOG.error("Failed to add entity to Phoenix " + se.getMessage());
+      throw new IOException(se);
+    } catch (Exception e) {
+      LOG.error("Exception on getting connection: " + e.getMessage());
+      throw new IOException(e);
+    }
+    return response;
+  }
+
+  /**
+   * Create Phoenix tables for offline aggregation storage if the tables do not
+   * exist.
+   *
+   * @throws IOException
+   */
+  public void createPhoenixTables() throws IOException {
+    // Create tables if necessary
+    try (Connection conn = getConnection();
+        Statement stmt = conn.createStatement()) {
+      // Table schema defined as in YARN-3817.
+      String sql = "CREATE TABLE IF NOT EXISTS "
+          + OfflineAggregationInfo.FLOW_AGGREGATION_TABLE_NAME
+          + "(user VARCHAR NOT NULL, cluster VARCHAR NOT NULL, "
+          + "flow_name VARCHAR NOT NULL, "
+          + "created_time UNSIGNED_LONG, modified_time UNSIGNED_LONG, "
+          + METRIC_COLUMN_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER + " VARBINARY, "
+          + "metric_names VARCHAR, info_keys VARCHAR "
+          + "CONSTRAINT pk PRIMARY KEY("
+          + "user, cluster, flow_name))";
+      stmt.executeUpdate(sql);
+      sql = "CREATE TABLE IF NOT EXISTS "
+          + OfflineAggregationInfo.USER_AGGREGATION_TABLE_NAME
+          + "(user VARCHAR NOT NULL, cluster VARCHAR NOT NULL, "
+          + "created_time UNSIGNED_LONG, modified_time UNSIGNED_LONG, "
+          + METRIC_COLUMN_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER + " VARBINARY, "
+          + "metric_names VARCHAR, info_keys VARCHAR "
+          + "CONSTRAINT pk PRIMARY KEY(user, cluster))";
+      stmt.executeUpdate(sql);
+      conn.commit();
+    } catch (SQLException se) {
+      LOG.error("Failed in init data " + se.getLocalizedMessage());
+      throw new IOException(se);
+    }
+    return;
+  }
+
+  // Utility functions
+  @Private
+  @VisibleForTesting
+  Connection getConnection() throws IOException {
+    Connection conn;
+    try {
+      conn = DriverManager.getConnection(connString, connProperties);
+      conn.setAutoCommit(false);
+    } catch (SQLException se) {
+      LOG.error("Failed to connect to phoenix server! "
+          + se.getLocalizedMessage());
+      throw new IOException(se);
+    }
+    return conn;
+  }
+
+  // WARNING: This method will permanently drop a table!
+  @Private
+  @VisibleForTesting
+  void dropTable(String tableName) throws Exception {
+    try (Connection conn = getConnection();
+         Statement stmt = conn.createStatement()) {
+      String sql = "DROP TABLE " + tableName;
+      stmt.executeUpdate(sql);
+    } catch (SQLException se) {
+      LOG.error("Failed in dropping entity table " + se.getLocalizedMessage());
+      throw se;
+    }
+  }
+
+  private static class DynamicColumns<K> {
+    static final String COLUMN_FAMILY_TYPE_BYTES = " VARBINARY";
+    static final String COLUMN_FAMILY_TYPE_STRING = " VARCHAR";
+    String columnFamilyPrefix;
+    String type;
+    Set<K> columns;
+
+    public DynamicColumns(String columnFamilyPrefix, String type,
+        Set<K> keyValues) {
+      this.columnFamilyPrefix = columnFamilyPrefix;
+      this.columns = keyValues;
+      this.type = type;
+    }
+  }
+
+  private static <K> StringBuilder appendColumnsSQL(
+      StringBuilder colNames, DynamicColumns<K> cfInfo) {
+    // Prepare the sql template by iterating through all keys
+    for (K key : cfInfo.columns) {
+      colNames.append(",").append(cfInfo.columnFamilyPrefix)
+          .append(key.toString()).append(cfInfo.type);
+    }
+    return colNames;
+  }
+
+  private static <K, V> int setValuesForColumnFamily(
+      PreparedStatement ps, Map<K, V> keyValues, int startPos,
+      boolean converToBytes) throws SQLException {
+    int idx = startPos;
+    for (Map.Entry<K, V> entry : keyValues.entrySet()) {
+      V value = entry.getValue();
+      if (value instanceof Collection) {
+        ps.setString(idx++, StringUtils.join(
+            (Collection) value, AGGREGATION_STORAGE_SEPARATOR));
+      } else {
+        if (converToBytes) {
+          try {
+            ps.setBytes(idx++, GenericObjectMapper.write(entry.getValue()));
+          } catch (IOException ie) {
+            LOG.error("Exception in converting values into bytes "
+                + ie.getMessage());
+            throw new SQLException(ie);
+          }
+        } else {
+          ps.setString(idx++, value.toString());
+        }
+      }
+    }
+    return idx;
+  }
+
+  private static <K, V> int setBytesForColumnFamily(
+      PreparedStatement ps, Map<K, V> keyValues, int startPos)
+      throws SQLException {
+    return setValuesForColumnFamily(ps, keyValues, startPos, true);
+  }
+
+  private static <K, V> int setStringsForColumnFamily(
+      PreparedStatement ps, Map<K, V> keyValues, int startPos)
+      throws SQLException {
+    return setValuesForColumnFamily(ps, keyValues, startPos, false);
+  }
+
+  private static void storeEntityVariableLengthFields(TimelineEntity entity,
+      Map<String, TimelineMetric> formattedMetrics,
+      TimelineCollectorContext context, Connection conn,
+      OfflineAggregationInfo aggregationInfo) throws SQLException {
+    int numPlaceholders = 0;
+    StringBuilder columnDefs = new StringBuilder(
+        StringUtils.join(aggregationInfo.getPrimaryKeyList(), ","));
+    if (formattedMetrics != null && formattedMetrics.size() > 0) {
+      appendColumnsSQL(columnDefs, new DynamicColumns<>(
+          METRIC_COLUMN_FAMILY, DynamicColumns.COLUMN_FAMILY_TYPE_BYTES,
+          formattedMetrics.keySet()));
+      numPlaceholders += formattedMetrics.keySet().size();
+    }
+    if (numPlaceholders == 0) {
+      return;
+    }
+    StringBuilder placeholders = new StringBuilder();
+    placeholders.append(
+        StringUtils.repeat("?,", aggregationInfo.getPrimaryKeyList().length));
+    // numPlaceholders >= 1 now
+    placeholders.append("?")
+        .append(StringUtils.repeat(",?", numPlaceholders - 1));
+    String sqlVariableLengthFields = new StringBuilder("UPSERT INTO ")
+        .append(aggregationInfo.getTableName()).append(" (").append(columnDefs)
+        .append(") VALUES(").append(placeholders).append(")").toString();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("SQL statement for variable length fields: "
+          + sqlVariableLengthFields);
+    }
+    // Use try with resource statement for the prepared statement
+    try (PreparedStatement psVariableLengthFields =
+        conn.prepareStatement(sqlVariableLengthFields)) {
+      int idx = aggregationInfo.setStringsForPrimaryKey(
+          psVariableLengthFields, context, null, 1);
+      if (formattedMetrics != null && formattedMetrics.size() > 0) {
+        idx = setBytesForColumnFamily(
+            psVariableLengthFields, formattedMetrics, idx);
+      }
+      psVariableLengthFields.execute();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a87a00ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixTimelineWriterImpl.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/PhoenixTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixTimelineWriterImpl.java
deleted file mode 100644
index 381ff17..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/PhoenixTimelineWriterImpl.java
+++ /dev/null
@@ -1,530 +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;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.lang.StringUtils;
-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.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.TimelineEvent;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
-import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
-
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-
-@Private
-@Unstable
-public class PhoenixTimelineWriterImpl extends AbstractService
-    implements TimelineWriter {
-
-  public static final String TIMELINE_SERVICE_PHOENIX_STORAGE_CONN_STR
-      = YarnConfiguration.TIMELINE_SERVICE_PREFIX
-          + "writer.phoenix.connectionString";
-
-  public static final String TIMELINE_SERVICE_PHEONIX_STORAGE_CONN_STR_DEFAULT
-      = "jdbc:phoenix:localhost:2181:/hbase";
-
-  private static final Log LOG
-      = LogFactory.getLog(PhoenixTimelineWriterImpl.class);
-  private static final String PHOENIX_COL_FAMILY_PLACE_HOLDER
-      = "timeline_cf_placeholder";
-  // These lists are not taking effects in table creations.
-  private static final String[] PHOENIX_STORAGE_PK_LIST
-      = {"cluster", "user", "flow_name", "flow_version", "flow_run", "app_id",
-         "type", "entity_id"};
-  private static final String[] TIMELINE_EVENT_EXTRA_PK_LIST =
-      {"timestamp", "event_id"};
-  private static final String[] TIMELINE_METRIC_EXTRA_PK_LIST =
-      {"metric_id"};
-  /** Default Phoenix JDBC driver name */
-  private static final String DRIVER_CLASS_NAME
-      = "org.apache.phoenix.jdbc.PhoenixDriver";
-
-  /** Default Phoenix timeline entity table name */
-  @VisibleForTesting
-  static final String ENTITY_TABLE_NAME = "timeline_entity";
-  /** Default Phoenix event table name */
-  @VisibleForTesting
-  static final String EVENT_TABLE_NAME = "timeline_event";
-  /** Default Phoenix metric table name */
-  @VisibleForTesting
-  static final String METRIC_TABLE_NAME = "metric_singledata";
-
-  /** Default Phoenix timeline config column family */
-  private static final String CONFIG_COLUMN_FAMILY = "c.";
-  /** Default Phoenix timeline info column family */
-  private static final String INFO_COLUMN_FAMILY = "i.";
-  /** Default Phoenix event info column family */
-  private static final String EVENT_INFO_COLUMN_FAMILY = "ei.";
-  /** Default Phoenix isRelatedTo column family */
-  private static final String IS_RELATED_TO_FAMILY = "ir.";
-  /** Default Phoenix relatesTo column family */
-  private static final String RELATES_TO_FAMILY = "rt.";
-  /** Default separator for Phoenix storage */
-  private static final String PHOENIX_STORAGE_SEPARATOR = ";";
-
-  /** Connection string to the deployed Phoenix cluster */
-  @VisibleForTesting
-  String connString = null;
-  @VisibleForTesting
-  Properties connProperties = new Properties();
-
-  PhoenixTimelineWriterImpl() {
-    super((PhoenixTimelineWriterImpl.class.getName()));
-  }
-
-  @Override
-  protected void serviceInit(Configuration conf) throws Exception {
-    // so check it here and only read in the config if it's not overridden.
-    connString =
-        conf.get(TIMELINE_SERVICE_PHOENIX_STORAGE_CONN_STR,
-        TIMELINE_SERVICE_PHEONIX_STORAGE_CONN_STR_DEFAULT);
-    createTables();
-    super.init(conf);
-  }
-
-  @Override
-  protected void serviceStop() throws Exception {
-    super.serviceStop();
-  }
-
-  @Override
-  public TimelineWriteResponse write(String clusterId, String userId,
-      String flowName, String flowVersion, long flowRunId, String appId,
-      TimelineEntities entities) throws IOException {
-    TimelineWriteResponse response = new TimelineWriteResponse();
-    TimelineCollectorContext currContext = new TimelineCollectorContext(
-        clusterId, userId, flowName, flowVersion, flowRunId, appId);
-    String sql = "UPSERT INTO " + ENTITY_TABLE_NAME
-        + " (" + StringUtils.join(PHOENIX_STORAGE_PK_LIST, ",")
-        + ", creation_time, modified_time, configs) "
-        + "VALUES (" + StringUtils.repeat("?,", PHOENIX_STORAGE_PK_LIST.length)
-        + "?, ?, ?)";
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("TimelineEntity write SQL: " + sql);
-    }
-
-    try (Connection conn = getConnection();
-        PreparedStatement ps = conn.prepareStatement(sql)) {
-      for (TimelineEntity entity : entities.getEntities()) {
-        int idx = setStringsForPrimaryKey(ps, currContext, entity, 1);
-        ps.setLong(idx++, entity.getCreatedTime());
-        ps.setLong(idx++, entity.getModifiedTime());
-        String configKeys = StringUtils.join(
-            entity.getConfigs().keySet(), PHOENIX_STORAGE_SEPARATOR);
-        ps.setString(idx++, configKeys);
-        ps.execute();
-
-        storeEntityVariableLengthFields(entity, currContext, conn);
-        storeEvents(entity, currContext, conn);
-        storeMetrics(entity, currContext, conn);
-
-        conn.commit();
-      }
-    } catch (SQLException se) {
-      LOG.error("Failed to add entity to Phoenix " + se.getMessage());
-      throw new IOException(se);
-    } catch (Exception e) {
-      LOG.error("Exception on getting connection: " + e.getMessage());
-      throw new IOException(e);
-    }
-    return response;
-  }
-
-  /**
-   * Aggregates the entity information to the timeline store based on which
-   * track this entity is to be rolled up to The tracks along which aggregations
-   * are to be done are given by {@link TimelineAggregationTrack}
-   *
-   * Any errors occurring for individual write request objects will be reported
-   * in the response.
-   *
-   * @param data
-   *          a {@link TimelineEntity} object
-   *          a {@link TimelineAggregationTrack} enum value
-   * @return a {@link TimelineWriteResponse} object.
-   * @throws IOException
-   */
-  @Override
-  public TimelineWriteResponse aggregate(TimelineEntity data,
-      TimelineAggregationTrack track) throws IOException {
-    return null;
-
-  }
-
-  @Override
-  public void flush() throws IOException {
-    // currently no-op
-  }
-
-  // Utility functions
-  @Private
-  @VisibleForTesting
-  Connection getConnection() throws IOException {
-    Connection conn;
-    try {
-      Class.forName(DRIVER_CLASS_NAME);
-      conn = DriverManager.getConnection(connString, connProperties);
-      conn.setAutoCommit(false);
-    } catch (SQLException se) {
-      LOG.error("Failed to connect to phoenix server! "
-          + se.getLocalizedMessage());
-      throw new IOException(se);
-    } catch (ClassNotFoundException e) {
-      LOG.error("Class not found! " + e.getLocalizedMessage());
-      throw new IOException(e);
-    }
-    return conn;
-  }
-
-  private void createTables() throws Exception {
-    // Create tables if necessary
-    try (Connection conn = getConnection();
-        Statement stmt = conn.createStatement()) {
-      // Table schema defined as in YARN-3134.
-      String sql = "CREATE TABLE IF NOT EXISTS " + ENTITY_TABLE_NAME
-          + "(user VARCHAR NOT NULL, cluster VARCHAR NOT NULL, "
-          + "flow_name VARCHAR NOT NULL, flow_version VARCHAR NOT NULL, "
-          + "flow_run UNSIGNED_LONG NOT NULL, "
-          + "app_id VARCHAR NOT NULL, type VARCHAR NOT NULL, "
-          + "entity_id VARCHAR NOT NULL, "
-          + "creation_time UNSIGNED_LONG, modified_time UNSIGNED_LONG, "
-          + "configs VARCHAR, "
-          + CONFIG_COLUMN_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER + " VARCHAR, "
-          + INFO_COLUMN_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER + " VARBINARY, "
-          + IS_RELATED_TO_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER + " VARCHAR, "
-          + RELATES_TO_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER + " VARCHAR "
-          + "CONSTRAINT pk PRIMARY KEY("
-          + "user, cluster, flow_name, flow_version, flow_run DESC, app_id, "
-          + "type, entity_id))";
-      stmt.executeUpdate(sql);
-      sql = "CREATE TABLE IF NOT EXISTS " + EVENT_TABLE_NAME
-          + "(user VARCHAR NOT NULL, cluster VARCHAR NOT NULL, "
-          + "flow_name VARCHAR NOT NULL, flow_version VARCHAR NOT NULL, "
-          + "flow_run UNSIGNED_LONG NOT NULL, "
-          + "app_id VARCHAR NOT NULL, type VARCHAR NOT NULL, "
-          + "entity_id VARCHAR NOT NULL, "
-          + "timestamp UNSIGNED_LONG NOT NULL, event_id VARCHAR NOT NULL, "
-          + EVENT_INFO_COLUMN_FAMILY + PHOENIX_COL_FAMILY_PLACE_HOLDER + " VARBINARY "
-          + "CONSTRAINT pk PRIMARY KEY("
-          + "user, cluster, flow_name, flow_version, flow_run DESC, app_id, "
-          + "type, entity_id, timestamp DESC, event_id))";
-      stmt.executeUpdate(sql);
-      sql = "CREATE TABLE IF NOT EXISTS " + METRIC_TABLE_NAME
-          + "(user VARCHAR NOT NULL, cluster VARCHAR NOT NULL, "
-          + "flow_name VARCHAR NOT NULL, flow_version VARCHAR NOT NULL, "
-          + "flow_run UNSIGNED_LONG NOT NULL, "
-          + "app_id VARCHAR NOT NULL, type VARCHAR NOT NULL, "
-          + "entity_id VARCHAR NOT NULL, "
-          + "metric_id VARCHAR NOT NULL, "
-          + "singledata VARBINARY, "
-          + "time UNSIGNED_LONG "
-          + "CONSTRAINT pk PRIMARY KEY("
-          + "user, cluster, flow_name, flow_version, flow_run DESC, app_id, "
-          + "type, entity_id, metric_id))";
-      stmt.executeUpdate(sql);
-      conn.commit();
-    } catch (SQLException se) {
-      LOG.error("Failed in init data " + se.getLocalizedMessage());
-      throw se;
-    }
-    return;
-  }
-
-  private static class DynamicColumns<K> {
-    static final String COLUMN_FAMILY_TYPE_BYTES = " VARBINARY";
-    static final String COLUMN_FAMILY_TYPE_STRING = " VARCHAR";
-    String columnFamilyPrefix;
-    String type;
-    Set<K> columns;
-
-    public DynamicColumns(String columnFamilyPrefix, String type,
-        Set<K> keyValues) {
-      this.columnFamilyPrefix = columnFamilyPrefix;
-      this.columns = keyValues;
-      this.type = type;
-    }
-  }
-
-  private static <K> StringBuilder appendColumnsSQL(
-      StringBuilder colNames, DynamicColumns<K> cfInfo) {
-    // Prepare the sql template by iterating through all keys
-    for (K key : cfInfo.columns) {
-      colNames.append(",").append(cfInfo.columnFamilyPrefix)
-          .append(key.toString()).append(cfInfo.type);
-    }
-    return colNames;
-  }
-
-  private static <K, V> int setValuesForColumnFamily(
-      PreparedStatement ps, Map<K, V> keyValues, int startPos,
-      boolean converToBytes) throws SQLException {
-    int idx = startPos;
-    for (Map.Entry<K, V> entry : keyValues.entrySet()) {
-      V value = entry.getValue();
-      if (value instanceof Collection) {
-        ps.setString(idx++, StringUtils.join(
-            (Collection) value, PHOENIX_STORAGE_SEPARATOR));
-      } else {
-        if (converToBytes) {
-          try {
-            ps.setBytes(idx++, GenericObjectMapper.write(entry.getValue()));
-          } catch (IOException ie) {
-            LOG.error("Exception in converting values into bytes "
-                + ie.getMessage());
-            throw new SQLException(ie);
-          }
-        } else {
-          ps.setString(idx++, value.toString());
-        }
-      }
-    }
-    return idx;
-  }
-
-  private static <K, V> int setBytesForColumnFamily(
-      PreparedStatement ps, Map<K, V> keyValues, int startPos)
-      throws SQLException {
-    return setValuesForColumnFamily(ps, keyValues, startPos, true);
-  }
-
-  private static <K, V> int setStringsForColumnFamily(
-      PreparedStatement ps, Map<K, V> keyValues, int startPos)
-      throws SQLException {
-    return setValuesForColumnFamily(ps, keyValues, startPos, false);
-  }
-
-  private static int setStringsForPrimaryKey(PreparedStatement ps,
-      TimelineCollectorContext context, TimelineEntity entity, int startPos)
-      throws SQLException {
-    int idx = startPos;
-    ps.setString(idx++, context.getClusterId());
-    ps.setString(idx++, context.getUserId());
-    ps.setString(idx++,
-        context.getFlowName());
-    ps.setString(idx++, context.getFlowVersion());
-    ps.setLong(idx++, context.getFlowRunId());
-    ps.setString(idx++, context.getAppId());
-    ps.setString(idx++, entity.getType());
-    ps.setString(idx++, entity.getId());
-    return idx;
-  }
-
-  private static void storeEntityVariableLengthFields(TimelineEntity entity,
-      TimelineCollectorContext context, Connection conn) throws SQLException {
-    int numPlaceholders = 0;
-    StringBuilder columnDefs = new StringBuilder(
-        StringUtils.join(PHOENIX_STORAGE_PK_LIST, ","));
-    if (entity.getConfigs() != null) {
-      Set<String> keySet = entity.getConfigs().keySet();
-      appendColumnsSQL(columnDefs, new DynamicColumns<>(
-          CONFIG_COLUMN_FAMILY, DynamicColumns.COLUMN_FAMILY_TYPE_STRING,
-          keySet));
-      numPlaceholders += keySet.size();
-    }
-    if (entity.getInfo() != null) {
-      Set<String> keySet = entity.getInfo().keySet();
-      appendColumnsSQL(columnDefs, new DynamicColumns<>(
-          INFO_COLUMN_FAMILY, DynamicColumns.COLUMN_FAMILY_TYPE_BYTES,
-          keySet));
-      numPlaceholders += keySet.size();
-    }
-    if (entity.getIsRelatedToEntities() != null) {
-      Set<String> keySet = entity.getIsRelatedToEntities().keySet();
-      appendColumnsSQL(columnDefs, new DynamicColumns<>(
-          IS_RELATED_TO_FAMILY, DynamicColumns.COLUMN_FAMILY_TYPE_STRING,
-          keySet));
-      numPlaceholders += keySet.size();
-    }
-    if (entity.getRelatesToEntities() != null) {
-      Set<String> keySet = entity.getRelatesToEntities().keySet();
-      appendColumnsSQL(columnDefs, new DynamicColumns<>(
-          RELATES_TO_FAMILY, DynamicColumns.COLUMN_FAMILY_TYPE_STRING,
-          keySet));
-      numPlaceholders += keySet.size();
-    }
-    if (numPlaceholders == 0) {
-      return;
-    }
-    StringBuilder placeholders = new StringBuilder();
-    placeholders.append(
-        StringUtils.repeat("?,", PHOENIX_STORAGE_PK_LIST.length));
-    // numPlaceholders >= 1 now
-    placeholders.append("?")
-        .append(StringUtils.repeat(",?", numPlaceholders - 1));
-    String sqlVariableLengthFields = new StringBuilder("UPSERT INTO ")
-        .append(ENTITY_TABLE_NAME).append(" (").append(columnDefs)
-        .append(") VALUES(").append(placeholders).append(")").toString();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("SQL statement for variable length fields: "
-          + sqlVariableLengthFields);
-    }
-    // Use try with resource statement for the prepared statement
-    try (PreparedStatement psVariableLengthFields =
-        conn.prepareStatement(sqlVariableLengthFields)) {
-      int idx = setStringsForPrimaryKey(
-          psVariableLengthFields, context, entity, 1);
-      if (entity.getConfigs() != null) {
-        idx = setStringsForColumnFamily(
-            psVariableLengthFields, entity.getConfigs(), idx);
-      }
-      if (entity.getInfo() != null) {
-        idx = setBytesForColumnFamily(
-            psVariableLengthFields, entity.getInfo(), idx);
-      }
-      if (entity.getIsRelatedToEntities() != null) {
-        idx = setStringsForColumnFamily(
-            psVariableLengthFields, entity.getIsRelatedToEntities(), idx);
-      }
-      if (entity.getRelatesToEntities() != null) {
-        idx = setStringsForColumnFamily(
-            psVariableLengthFields, entity.getRelatesToEntities(), idx);
-      }
-      psVariableLengthFields.execute();
-    }
-  }
-
-  private static void storeMetrics(TimelineEntity entity,
-      TimelineCollectorContext context, Connection conn) throws SQLException {
-    if (entity.getMetrics() == null) {
-      return;
-    }
-    Set<TimelineMetric> metrics = entity.getMetrics();
-    for (TimelineMetric metric : metrics) {
-      StringBuilder sqlColumns = new StringBuilder(
-          StringUtils.join(PHOENIX_STORAGE_PK_LIST, ","));
-      sqlColumns.append(",")
-          .append(StringUtils.join(TIMELINE_METRIC_EXTRA_PK_LIST, ","));
-      sqlColumns.append(",").append("singledata, time");
-      StringBuilder placeholders = new StringBuilder();
-      placeholders.append(
-          StringUtils.repeat("?,", PHOENIX_STORAGE_PK_LIST.length))
-          .append(StringUtils.repeat("?,", TIMELINE_METRIC_EXTRA_PK_LIST.length));
-      placeholders.append("?, ?");
-      String sqlMetric = new StringBuilder("UPSERT INTO ")
-          .append(METRIC_TABLE_NAME).append(" (").append(sqlColumns)
-          .append(") VALUES(").append(placeholders).append(")").toString();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("SQL statement for metric: " + sqlMetric);
-      }
-      try (PreparedStatement psMetrics = conn.prepareStatement(sqlMetric)) {
-        if (metric.getType().equals(TimelineMetric.Type.TIME_SERIES)) {
-          LOG.warn("The incoming timeline metric contains time series data, "
-              + "which is currently not supported by Phoenix storage. "
-              + "Time series will be truncated. ");
-        }
-        int idx = setStringsForPrimaryKey(psMetrics, context, entity, 1);
-        psMetrics.setString(idx++, metric.getId());
-        Iterator<Map.Entry<Long, Number>> currNumIter =
-            metric.getValues().entrySet().iterator();
-        if (currNumIter.hasNext()) {
-          // TODO: support time series storage
-          Map.Entry<Long, Number> currEntry = currNumIter.next();
-          psMetrics.setBytes(idx++,
-              GenericObjectMapper.write(currEntry.getValue()));
-          psMetrics.setLong(idx++, currEntry.getKey());
-        } else {
-          psMetrics.setBytes(idx++, GenericObjectMapper.write(null));
-          LOG.warn("The incoming metric contains an empty value set. ");
-        }
-        psMetrics.execute();
-      } catch (IOException ie) {
-        LOG.error("Exception on converting single data to bytes: "
-            + ie.getMessage());
-        throw new SQLException(ie);
-      }
-    }
-  }
-
-  private static void storeEvents(TimelineEntity entity,
-      TimelineCollectorContext context, Connection conn) throws SQLException {
-    if (entity.getEvents() == null) {
-      return;
-    }
-    Set<TimelineEvent> events = entity.getEvents();
-    for (TimelineEvent event : events) {
-      // We need this number to check if the incoming event's info field is empty
-      int numPlaceholders = 0;
-      StringBuilder sqlColumns = new StringBuilder(
-          StringUtils.join(PHOENIX_STORAGE_PK_LIST, ","));
-      sqlColumns.append(",")
-          .append(StringUtils.join(TIMELINE_EVENT_EXTRA_PK_LIST, ","));
-      appendColumnsSQL(sqlColumns, new DynamicColumns<>(
-          EVENT_INFO_COLUMN_FAMILY, DynamicColumns.COLUMN_FAMILY_TYPE_BYTES,
-          event.getInfo().keySet()));
-      numPlaceholders += event.getInfo().keySet().size();
-      if (numPlaceholders == 0) {
-        continue;
-      }
-      StringBuilder placeholders = new StringBuilder();
-      placeholders.append(
-          StringUtils.repeat("?,", PHOENIX_STORAGE_PK_LIST.length))
-          .append(StringUtils.repeat("?,", TIMELINE_EVENT_EXTRA_PK_LIST.length));
-      // numPlaceholders >= 1 now
-      placeholders.append("?")
-            .append(StringUtils.repeat(",?", numPlaceholders - 1));
-      String sqlEvents = new StringBuilder("UPSERT INTO ")
-          .append(EVENT_TABLE_NAME).append(" (").append(sqlColumns)
-          .append(") VALUES(").append(placeholders).append(")").toString();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("SQL statement for events: " + sqlEvents);
-      }
-      try (PreparedStatement psEvent = conn.prepareStatement(sqlEvents)) {
-        int idx = setStringsForPrimaryKey(psEvent, context, entity, 1);
-        psEvent.setLong(idx++, event.getTimestamp());
-        psEvent.setString(idx++, event.getId());
-        setBytesForColumnFamily(psEvent, event.getInfo(), idx);
-        psEvent.execute();
-      }
-    }
-  }
-
-  // WARNING: This method will permanently drop a table!
-  @Private
-  @VisibleForTesting
-  void dropTable(String tableName) throws Exception {
-    try (Connection conn = getConnection();
-         Statement stmt = conn.createStatement()) {
-      String sql = "DROP TABLE " + tableName;
-      stmt.executeUpdate(sql);
-    } catch (SQLException se) {
-      LOG.error("Failed in dropping entity table " + se.getLocalizedMessage());
-      throw se;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a87a00ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
index 3a22ed6..5120856 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
@@ -51,6 +53,7 @@ public class TimelineSchemaCreator {
 
   final static String NAME = TimelineSchemaCreator.class.getSimpleName();
   private static final Log LOG = LogFactory.getLog(TimelineSchemaCreator.class);
+  private static final String PHOENIX_OPTION_SHORT = "p";
 
   public static void main(String[] args) throws Exception {
 
@@ -83,7 +86,41 @@ public class TimelineSchemaCreator {
       hbaseConf.set(ApplicationTable.TABLE_NAME_CONF_NAME,
           applicationTableName);
     }
-    createAllTables(hbaseConf);
+
+    List<Exception> exceptions = new ArrayList<>();
+    try {
+      createAllTables(hbaseConf);
+      LOG.info("Successfully created HBase schema. ");
+    } catch (IOException e) {
+      LOG.error("Error in creating hbase tables: " + e.getMessage());
+      exceptions.add(e);
+    }
+
+    // Create Phoenix data schema if needed
+    if (commandLine.hasOption(PHOENIX_OPTION_SHORT)) {
+      Configuration phoenixConf = new Configuration();
+      try {
+        PhoenixOfflineAggregationWriterImpl phoenixWriter =
+            new PhoenixOfflineAggregationWriterImpl();
+        phoenixWriter.init(phoenixConf);
+        phoenixWriter.start();
+        phoenixWriter.createPhoenixTables();
+        phoenixWriter.stop();
+        LOG.info("Successfully created Phoenix offline aggregation schema. ");
+      } catch (IOException e) {
+        LOG.error("Error in creating phoenix 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");
+    }
   }
 
   /**
@@ -115,6 +152,12 @@ public class TimelineSchemaCreator {
     o.setRequired(false);
     options.addOption(o);
 
+    o = new Option(PHOENIX_OPTION_SHORT, "usePhoenix", false,
+        "create Phoenix offline aggregation tables");
+    // No need to set arg name since we do not need an argument here
+    o.setRequired(false);
+    options.addOption(o);
+
     CommandLineParser parser = new PosixParser();
     CommandLine commandLine = null;
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a87a00ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/OfflineAggregationInfo.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/OfflineAggregationInfo.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/OfflineAggregationInfo.java
new file mode 100644
index 0000000..16c03a3
--- /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/common/OfflineAggregationInfo.java
@@ -0,0 +1,110 @@
+/**
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
+
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+
+/**
+ * Class to carry the offline aggregation information for storage level
+ * implementations. There are currently two predefined aggregation info
+ * instances that represent flow and user level offline aggregations. Depend on
+ * its implementation, a storage class may use an OfflineAggregationInfo object
+ * to decide behaviors dynamically.
+ */
+public final class OfflineAggregationInfo {
+  /**
+   * Default flow level aggregation table name
+   */
+  @VisibleForTesting
+  public static final String FLOW_AGGREGATION_TABLE_NAME
+      = "yarn_timeline_flow_aggregation";
+  /**
+   * Default user level aggregation table name
+   */
+  public static final String USER_AGGREGATION_TABLE_NAME
+      = "yarn_timeline_user_aggregation";
+
+  // These lists are not taking effects in table creations.
+  private static final String[] FLOW_AGGREGATION_PK_LIST =
+      { "user", "cluster", "flow_name" };
+  private static final String[] USER_AGGREGATION_PK_LIST = { "user", "cluster"};
+
+  private final String tableName;
+  private final String[] primaryKeyList;
+  private final PrimaryKeyStringSetter primaryKeyStringSetter;
+
+  private OfflineAggregationInfo(String table, String[] pkList,
+      PrimaryKeyStringSetter formatter) {
+    tableName = table;
+    primaryKeyList = pkList;
+    primaryKeyStringSetter = formatter;
+  }
+
+  private interface PrimaryKeyStringSetter {
+    int setValues(PreparedStatement ps, TimelineCollectorContext context,
+        String[] extraInfo, int startPos) throws SQLException;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public String[] getPrimaryKeyList() {
+    return primaryKeyList.clone();
+  }
+
+  public int setStringsForPrimaryKey(PreparedStatement ps,
+      TimelineCollectorContext context, String[] extraInfo, int startPos)
+      throws SQLException {
+    return primaryKeyStringSetter.setValues(ps, context, extraInfo, startPos);
+  }
+
+  public static final OfflineAggregationInfo FLOW_AGGREGATION =
+      new OfflineAggregationInfo(FLOW_AGGREGATION_TABLE_NAME,
+          FLOW_AGGREGATION_PK_LIST, new PrimaryKeyStringSetter() {
+        @Override
+        public int setValues(PreparedStatement ps,
+            TimelineCollectorContext context, String[] extraInfo, int startPos)
+            throws SQLException {
+          int idx = startPos;
+          ps.setString(idx++, context.getUserId());
+          ps.setString(idx++, context.getClusterId());
+          ps.setString(idx++, context.getFlowName());
+          return idx;
+        }
+      });
+
+  public static final OfflineAggregationInfo USER_AGGREGATION =
+      new OfflineAggregationInfo(USER_AGGREGATION_TABLE_NAME,
+          USER_AGGREGATION_PK_LIST, new PrimaryKeyStringSetter() {
+        @Override
+        public int setValues(PreparedStatement ps,
+            TimelineCollectorContext context, String[] extraInfo, int startPos)
+            throws SQLException {
+          int idx = startPos;
+          ps.setString(idx++, context.getUserId());
+          ps.setString(idx++, context.getClusterId());
+          return idx;
+        }
+      });
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a87a00ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixOfflineAggregationWriterImpl.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/TestPhoenixOfflineAggregationWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixOfflineAggregationWriterImpl.java
new file mode 100644
index 0000000..de66a17
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixOfflineAggregationWriterImpl.java
@@ -0,0 +1,162 @@
+/**
+ * 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;
+
+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 org.apache.hadoop.hbase.IntegrationTestingUtility;
+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.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.OfflineAggregationInfo;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.apache.phoenix.hbase.index.write.IndexWriterUtils;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.ReadOnlyProps;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+
+public class TestPhoenixOfflineAggregationWriterImpl extends BaseTest {
+  private static PhoenixOfflineAggregationWriterImpl storage;
+  private static final int BATCH_SIZE = 3;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    YarnConfiguration conf = new YarnConfiguration();
+    storage = setupPhoenixClusterAndWriterForTest(conf);
+  }
+
+  @Test(timeout = 90000)
+  public void testFlowLevelAggregationStorage() throws Exception {
+    testAggregator(OfflineAggregationInfo.FLOW_AGGREGATION);
+  }
+
+  @Test(timeout = 90000)
+  public void testUserLevelAggregationStorage() throws Exception {
+    testAggregator(OfflineAggregationInfo.USER_AGGREGATION);
+  }
+
+  @AfterClass
+  public static void cleanup() throws Exception {
+    storage.dropTable(OfflineAggregationInfo.FLOW_AGGREGATION_TABLE_NAME);
+    storage.dropTable(OfflineAggregationInfo.USER_AGGREGATION_TABLE_NAME);
+    tearDownMiniCluster();
+  }
+
+  private static PhoenixOfflineAggregationWriterImpl
+    setupPhoenixClusterAndWriterForTest(YarnConfiguration conf)
+      throws Exception{
+    Map<String, String> props = new HashMap<>();
+    // Must update config before starting server
+    props.put(QueryServices.STATS_USE_CURRENT_TIME_ATTRIB,
+        Boolean.FALSE.toString());
+    props.put("java.security.krb5.realm", "");
+    props.put("java.security.krb5.kdc", "");
+    props.put(IntegrationTestingUtility.IS_DISTRIBUTED_CLUSTER,
+        Boolean.FALSE.toString());
+    props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(5000));
+    props.put(IndexWriterUtils.HTABLE_THREAD_KEY, Integer.toString(100));
+    // Make a small batch size to test multiple calls to reserve sequences
+    props.put(QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB,
+        Long.toString(BATCH_SIZE));
+    // Must update config before starting server
+    setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+
+    // Change connection settings for test
+    conf.set(
+        YarnConfiguration.PHOENIX_OFFLINE_STORAGE_CONN_STR,
+        getUrl());
+    PhoenixOfflineAggregationWriterImpl
+        myWriter = new PhoenixOfflineAggregationWriterImpl(TEST_PROPERTIES);
+    myWriter.init(conf);
+    myWriter.start();
+    myWriter.createPhoenixTables();
+    return myWriter;
+  }
+
+  private static TimelineEntity getTestAggregationTimelineEntity() {
+    TimelineEntity entity = new TimelineEntity();
+    String id = "hello1";
+    String type = "testAggregationType";
+    entity.setId(id);
+    entity.setType(type);
+    entity.setCreatedTime(1425016501000L);
+    entity.setModifiedTime(1425016502000L);
+
+    TimelineMetric metric = new TimelineMetric();
+    metric.setId("HDFS_BYTES_READ");
+    metric.addValue(1425016501100L, 8000);
+    entity.addMetric(metric);
+
+    return entity;
+  }
+
+  private void testAggregator(OfflineAggregationInfo aggregationInfo)
+      throws Exception {
+    // Set up a list of timeline entities and write them back to Phoenix
+    int numEntity = 1;
+    TimelineEntities te = new TimelineEntities();
+    te.addEntity(getTestAggregationTimelineEntity());
+    TimelineCollectorContext context = new TimelineCollectorContext("cluster_1",
+        "user1", "testFlow", null, 0, null);
+    storage.writeAggregatedEntity(context, te,
+        aggregationInfo);
+
+    // Verify if we're storing all entities
+    String[] primaryKeyList = aggregationInfo.getPrimaryKeyList();
+    String sql = "SELECT COUNT(" + primaryKeyList[primaryKeyList.length - 1]
+        +") FROM " + aggregationInfo.getTableName();
+    verifySQLWithCount(sql, numEntity, "Number of entities should be ");
+    // Check metric
+    sql = "SELECT COUNT(m.HDFS_BYTES_READ) FROM "
+        + aggregationInfo.getTableName() + "(m.HDFS_BYTES_READ VARBINARY) ";
+    verifySQLWithCount(sql, numEntity,
+        "Number of entities with info should be ");
+  }
+
+
+  private void verifySQLWithCount(String sql, int targetCount, String message)
+      throws Exception {
+    try (
+        Statement stmt =
+          storage.getConnection().createStatement();
+        ResultSet rs = stmt.executeQuery(sql)) {
+      assertTrue("Result set empty on statement " + sql, rs.next());
+      assertNotNull("Fail to execute query " + sql, rs);
+      assertEquals(message + " " + targetCount, targetCount, rs.getInt(1));
+    } catch (SQLException se) {
+      fail("SQL exception on query: " + sql
+          + " With exception message: " + se.getLocalizedMessage());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a87a00ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixTimelineWriterImpl.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/TestPhoenixTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixTimelineWriterImpl.java
deleted file mode 100644
index dece83d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixTimelineWriterImpl.java
+++ /dev/null
@@ -1,152 +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;
-
-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 org.apache.hadoop.hbase.IntegrationTestingUtility;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.apache.phoenix.hbase.index.write.IndexWriterUtils;
-import org.apache.phoenix.query.BaseTest;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.PropertiesUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
-
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-
-public class TestPhoenixTimelineWriterImpl extends BaseTest {
-  private static PhoenixTimelineWriterImpl writer;
-  private static final int BATCH_SIZE = 3;
-
-  @BeforeClass
-  public static void setup() throws Exception {
-    YarnConfiguration conf = new YarnConfiguration();
-    writer = setupPhoenixClusterAndWriterForTest(conf);
-  }
-
-  @Test(timeout = 90000)
-  public void testPhoenixWriterBasic() throws Exception {
-    // Set up a list of timeline entities and write them back to Phoenix
-    int numEntity = 12;
-    TimelineEntities te =
-        TestTimelineWriterImpl.getStandardTestTimelineEntities(numEntity);
-    writer.write("cluster_1", "user1", "testFlow", "version1", 1l, "app_test_1", te);
-    // Verify if we're storing all entities
-    String sql = "SELECT COUNT(entity_id) FROM "
-        + PhoenixTimelineWriterImpl.ENTITY_TABLE_NAME;
-    verifySQLWithCount(sql, numEntity, "Number of entities should be ");
-    // Check config (half of all entities)
-    sql = "SELECT COUNT(c.config) FROM "
-        + PhoenixTimelineWriterImpl.ENTITY_TABLE_NAME + "(c.config VARCHAR) ";
-    verifySQLWithCount(sql, (numEntity / 2),
-        "Number of entities with config should be ");
-    // Check info (half of all entities)
-    sql = "SELECT COUNT(i.info1) FROM "
-        + PhoenixTimelineWriterImpl.ENTITY_TABLE_NAME + "(i.info1 VARBINARY) ";
-    verifySQLWithCount(sql, (numEntity / 2),
-        "Number of entities with info should be ");
-    // Check config and info (a quarter of all entities)
-    sql = "SELECT COUNT(entity_id) FROM "
-        + PhoenixTimelineWriterImpl.ENTITY_TABLE_NAME
-        + "(c.config VARCHAR, i.info1 VARBINARY) "
-        + "WHERE c.config IS NOT NULL AND i.info1 IS NOT NULL";
-    verifySQLWithCount(sql, (numEntity / 4),
-        "Number of entities with both config and info should be ");
-    // Check relatesToEntities and isRelatedToEntities
-    sql = "SELECT COUNT(entity_id) FROM "
-        + PhoenixTimelineWriterImpl.ENTITY_TABLE_NAME
-        + "(rt.testType VARCHAR, ir.testType VARCHAR) "
-        + "WHERE rt.testType IS NOT NULL AND ir.testType IS NOT NULL";
-    verifySQLWithCount(sql, numEntity - 2,
-        "Number of entities with both relatesTo and isRelatedTo should be ");
-    // Check event
-    sql = "SELECT COUNT(entity_id) FROM "
-        + PhoenixTimelineWriterImpl.EVENT_TABLE_NAME;
-    verifySQLWithCount(sql, (numEntity / 4), "Number of events should be ");
-    // Check metrics
-    sql = "SELECT COUNT(entity_id) FROM "
-        + PhoenixTimelineWriterImpl.METRIC_TABLE_NAME;
-    verifySQLWithCount(sql, (numEntity / 4), "Number of events should be ");
-  }
-
-  @AfterClass
-  public static void cleanup() throws Exception {
-    writer.dropTable(PhoenixTimelineWriterImpl.ENTITY_TABLE_NAME);
-    writer.dropTable(PhoenixTimelineWriterImpl.EVENT_TABLE_NAME);
-    writer.dropTable(PhoenixTimelineWriterImpl.METRIC_TABLE_NAME);
-    writer.serviceStop();
-    tearDownMiniCluster();
-  }
-
-  private static PhoenixTimelineWriterImpl setupPhoenixClusterAndWriterForTest(
-      YarnConfiguration conf) throws Exception{
-    Map<String, String> props = new HashMap<>();
-    // Must update config before starting server
-    props.put(QueryServices.STATS_USE_CURRENT_TIME_ATTRIB,
-        Boolean.FALSE.toString());
-    props.put("java.security.krb5.realm", "");
-    props.put("java.security.krb5.kdc", "");
-    props.put(IntegrationTestingUtility.IS_DISTRIBUTED_CLUSTER,
-        Boolean.FALSE.toString());
-    props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(5000));
-    props.put(IndexWriterUtils.HTABLE_THREAD_KEY, Integer.toString(100));
-    // Make a small batch size to test multiple calls to reserve sequences
-    props.put(QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB,
-        Long.toString(BATCH_SIZE));
-    // Must update config before starting server
-    setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
-
-    PhoenixTimelineWriterImpl myWriter = new PhoenixTimelineWriterImpl();
-    // Change connection settings for test
-    conf.set(
-        PhoenixTimelineWriterImpl.TIMELINE_SERVICE_PHOENIX_STORAGE_CONN_STR,
-        getUrl());
-    myWriter.connProperties = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    myWriter.serviceInit(conf);
-    return myWriter;
-  }
-
-  private void verifySQLWithCount(String sql, int targetCount, String message)
-      throws Exception {
-    try (
-        Statement stmt =
-          writer.getConnection().createStatement();
-        ResultSet rs = stmt.executeQuery(sql)) {
-      assertTrue("Result set empty on statement " + sql, rs.next());
-      assertNotNull("Fail to execute query " + sql, rs);
-      assertEquals(message + " " + targetCount, targetCount, rs.getInt(1));
-    } catch (SQLException se) {
-      fail("SQL exception on query: " + sql
-          + " With exception message: " + se.getLocalizedMessage());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a87a00ee/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestTimelineWriterImpl.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/TestTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestTimelineWriterImpl.java
deleted file mode 100644
index 7a7afc0..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestTimelineWriterImpl.java
+++ /dev/null
@@ -1,74 +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;
-
-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.TimelineEvent;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
-
-public class TestTimelineWriterImpl {
-  static TimelineEntities getStandardTestTimelineEntities(int listSize) {
-    TimelineEntities te = new TimelineEntities();
-    for (int i = 0; i < listSize; i++) {
-      TimelineEntity entity = new TimelineEntity();
-      String id = "hello" + i;
-      String type = "testType";
-      entity.setId(id);
-      entity.setType(type);
-      entity.setCreatedTime(1425016501000L + i);
-      entity.setModifiedTime(1425016502000L + i);
-      if (i > 0) {
-        entity.addRelatesToEntity(type, "hello" + i);
-        entity.addRelatesToEntity(type, "hello" + (i - 1));
-      }
-      if (i < listSize - 1) {
-        entity.addIsRelatedToEntity(type, "hello" + i);
-        entity.addIsRelatedToEntity(type, "hello" + (i + 1));
-      }
-      int category = i % 4;
-      switch (category) {
-      case 0:
-        entity.addConfig("config", "config" + i);
-        // Fall through deliberately
-      case 1:
-        entity.addInfo("info1", new Integer(i));
-        entity.addInfo("info2", "helloworld");
-        // Fall through deliberately
-      case 2:
-        break;
-      case 3:
-        entity.addConfig("config", "config" + i);
-        TimelineEvent event = new TimelineEvent();
-        event.setId("test event");
-        event.setTimestamp(1425016501100L + i);
-        event.addInfo("test_info", "content for " + entity.getId());
-        event.addInfo("test_info1", new Integer(i));
-        entity.addEvent(event);
-        TimelineMetric metric = new TimelineMetric();
-        metric.setId("HDFS_BYTES_READ");
-        metric.addValue(1425016501100L + i, 8000 + i);
-        entity.addMetric(metric);
-        break;
-      }
-      te.addEntity(entity);
-    }
-    return te;
-  }
-}


[16/50] [abbrv] hadoop git commit: YARN-4064. build is broken at TestHBaseTimelineWriterImpl.java (sjlee)

Posted by gt...@apache.org.
YARN-4064. build is broken at TestHBaseTimelineWriterImpl.java (sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: a057b2895993787983623191f3ac008391de4e80
Parents: 233bfc9
Author: Sangjin Lee <sj...@apache.org>
Authored: Wed Aug 19 17:46:03 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:57 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../storage/TestHBaseTimelineStorage.java       | 770 +++++++++++++++++++
 .../storage/TestHBaseTimelineWriterImpl.java    | 770 -------------------
 3 files changed, 772 insertions(+), 770 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a057b289/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b72c4a6..68ef1f7 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -139,6 +139,8 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3984. Adjusted the event column key schema and avoided missing empty
     event. (Vrushali C via zjshen)
 
+    YARN-4064. build is broken at TestHBaseTimelineWriterImpl.java (sjlee)
+
 Trunk - Unreleased
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a057b289/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.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/TestHBaseTimelineStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
new file mode 100644
index 0000000..2875e01
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -0,0 +1,770 @@
+/**
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+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.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
+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.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.storage.application.ApplicationColumn;
+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.AppToFlowTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Various tests to test writing entities to HBase and reading them back from
+ * it.
+ *
+ * It uses a single HBase mini-cluster for all tests which is a little more
+ * realistic, and helps test correctness in the presence of other data.
+ *
+ * Each test uses a different cluster name to be able to handle its own data
+ * even if other records exist in the table. Use a different cluster name if
+ * you add a new test.
+ */
+public class TestHBaseTimelineStorage {
+
+  private static HBaseTestingUtility util;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    util = new HBaseTestingUtility();
+    util.startMiniCluster();
+    createSchema();
+  }
+
+  private static void createSchema() throws IOException {
+    new EntityTable()
+        .createTable(util.getHBaseAdmin(), util.getConfiguration());
+    new AppToFlowTable()
+        .createTable(util.getHBaseAdmin(), util.getConfiguration());
+    new ApplicationTable()
+        .createTable(util.getHBaseAdmin(), util.getConfiguration());
+  }
+
+  @Test
+  public void testWriteApplicationToHBase() throws Exception {
+    TimelineEntities te = new TimelineEntities();
+    ApplicationEntity entity = new ApplicationEntity();
+    String id = "hello";
+    entity.setId(id);
+    long cTime = 1425016501000L;
+    long mTime = 1425026901000L;
+    entity.setCreatedTime(cTime);
+    entity.setModifiedTime(mTime);
+
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<String, Object>();
+    infoMap.put("infoMapKey1", "infoMapValue1");
+    infoMap.put("infoMapKey2", 10);
+    entity.addInfo(infoMap);
+
+    // add the isRelatedToEntity info
+    String key = "task";
+    String value = "is_related_to_entity_id_here";
+    Set<String> isRelatedToSet = new HashSet<String>();
+    isRelatedToSet.add(value);
+    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
+    isRelatedTo.put(key, isRelatedToSet);
+    entity.setIsRelatedToEntities(isRelatedTo);
+
+    // add the relatesTo info
+    key = "container";
+    value = "relates_to_entity_id_here";
+    Set<String> relatesToSet = new HashSet<String>();
+    relatesToSet.add(value);
+    value = "relates_to_entity_id_here_Second";
+    relatesToSet.add(value);
+    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
+    relatesTo.put(key, relatesToSet);
+    entity.setRelatesToEntities(relatesTo);
+
+    // add some config entries
+    Map<String, String> conf = new HashMap<String, String>();
+    conf.put("config_param1", "value1");
+    conf.put("config_param2", "value2");
+    entity.addConfigs(conf);
+
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    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);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    entity.addMetrics(metrics);
+
+    te.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+      String cluster = "cluster_test_write_app";
+      String user = "user1";
+      String flow = "some_flow_name";
+      String flowVersion = "AB7822C10F1111";
+      long runid = 1002345678919L;
+      hbi.write(cluster, user, flow, flowVersion, runid, id, te);
+      hbi.stop();
+
+      // retrieve the row
+      byte[] rowKey =
+          ApplicationRowKey.getRowKey(cluster, user, flow, runid, id);
+      Get get = new Get(rowKey);
+      get.setMaxVersions(Integer.MAX_VALUE);
+      Connection conn = ConnectionFactory.createConnection(c1);
+      Result result = new ApplicationTable().getResult(c1, conn, get);
+
+      assertTrue(result != null);
+      assertEquals(16, result.size());
+
+      // check the row key
+      byte[] row1 = result.getRow();
+      assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
+          id));
+
+      // check info column family
+      String id1 = ApplicationColumn.ID.readResult(result).toString();
+      assertEquals(id, id1);
+
+      Number val =
+          (Number) ApplicationColumn.CREATED_TIME.readResult(result);
+      long cTime1 = val.longValue();
+      assertEquals(cTime1, cTime);
+
+      val = (Number) ApplicationColumn.MODIFIED_TIME.readResult(result);
+      long mTime1 = val.longValue();
+      assertEquals(mTime1, mTime);
+
+      Map<String, Object> infoColumns =
+          ApplicationColumnPrefix.INFO.readResults(result);
+      assertEquals(infoMap, infoColumns);
+
+      // Remember isRelatedTo is of type Map<String, Set<String>>
+      for (String isRelatedToKey : isRelatedTo.keySet()) {
+        Object isRelatedToValue =
+            ApplicationColumnPrefix.IS_RELATED_TO.readResult(result,
+                isRelatedToKey);
+        String compoundValue = isRelatedToValue.toString();
+        // id7?id9?id6
+        Set<String> isRelatedToValues =
+            new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
+        assertEquals(isRelatedTo.get(isRelatedToKey).size(),
+            isRelatedToValues.size());
+        for (String v : isRelatedTo.get(isRelatedToKey)) {
+          assertTrue(isRelatedToValues.contains(v));
+        }
+      }
+
+      // RelatesTo
+      for (String relatesToKey : relatesTo.keySet()) {
+        String compoundValue =
+            ApplicationColumnPrefix.RELATES_TO.readResult(result,
+                relatesToKey).toString();
+        // id3?id4?id5
+        Set<String> relatesToValues =
+            new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
+        assertEquals(relatesTo.get(relatesToKey).size(),
+            relatesToValues.size());
+        for (String v : relatesTo.get(relatesToKey)) {
+          assertTrue(relatesToValues.contains(v));
+        }
+      }
+
+      // Configuration
+      Map<String, Object> configColumns =
+          ApplicationColumnPrefix.CONFIG.readResults(result);
+      assertEquals(conf, configColumns);
+
+      NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
+          ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result);
+
+      NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
+      assertEquals(metricValues, metricMap);
+
+      // read the timeline entity using the reader this time
+      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, id,
+          entity.getType(), entity.getId(),
+          EnumSet.of(TimelineReader.Field.ALL));
+      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
+          id, entity.getType(), null, null, null, null, null, null, null,
+          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
+      assertNotNull(e1);
+      assertEquals(1, es1.size());
+
+      // verify attributes
+      assertEquals(id, e1.getId());
+      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
+          e1.getType());
+      assertEquals(cTime, e1.getCreatedTime());
+      assertEquals(mTime, e1.getModifiedTime());
+      Map<String, Object> infoMap2 = e1.getInfo();
+      assertEquals(infoMap, infoMap2);
+
+      Map<String, Set<String>> isRelatedTo2 = e1.getIsRelatedToEntities();
+      assertEquals(isRelatedTo, isRelatedTo2);
+
+      Map<String, Set<String>> relatesTo2 = e1.getRelatesToEntities();
+      assertEquals(relatesTo, relatesTo2);
+
+      Map<String, String> conf2 = e1.getConfigs();
+      assertEquals(conf, conf2);
+
+      Set<TimelineMetric> metrics2 = e1.getMetrics();
+      assertEquals(metrics, metrics2);
+      for (TimelineMetric metric2 : metrics2) {
+        Map<Long, Number> metricValues2 = metric2.getValues();
+        assertEquals(metricValues, metricValues2);
+      }
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+      if (hbr != null) {
+        hbr.stop();
+        hbr.close();
+      }
+    }
+  }
+
+  @Test
+  public void testWriteEntityToHBase() throws Exception {
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entity = new TimelineEntity();
+    String id = "hello";
+    String type = "world";
+    entity.setId(id);
+    entity.setType(type);
+    long cTime = 1425016501000L;
+    long mTime = 1425026901000L;
+    entity.setCreatedTime(cTime);
+    entity.setModifiedTime(mTime);
+
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<String, Object>();
+    infoMap.put("infoMapKey1", "infoMapValue1");
+    infoMap.put("infoMapKey2", 10);
+    entity.addInfo(infoMap);
+
+    // add the isRelatedToEntity info
+    String key = "task";
+    String value = "is_related_to_entity_id_here";
+    Set<String> isRelatedToSet = new HashSet<String>();
+    isRelatedToSet.add(value);
+    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
+    isRelatedTo.put(key, isRelatedToSet);
+    entity.setIsRelatedToEntities(isRelatedTo);
+
+    // add the relatesTo info
+    key = "container";
+    value = "relates_to_entity_id_here";
+    Set<String> relatesToSet = new HashSet<String>();
+    relatesToSet.add(value);
+    value = "relates_to_entity_id_here_Second";
+    relatesToSet.add(value);
+    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
+    relatesTo.put(key, relatesToSet);
+    entity.setRelatesToEntities(relatesTo);
+
+    // add some config entries
+    Map<String, String> conf = new HashMap<String, String>();
+    conf.put("config_param1", "value1");
+    conf.put("config_param2", "value2");
+    entity.addConfigs(conf);
+
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    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);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    entity.addMetrics(metrics);
+
+    te.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+      String cluster = "cluster_test_write_entity";
+      String user = "user1";
+      String flow = "some_flow_name";
+      String flowVersion = "AB7822C10F1111";
+      long runid = 1002345678919L;
+      String appName = "some app name";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.stop();
+
+      // scan the table and see that entity exists
+      Scan s = new Scan();
+      byte[] startRow =
+          EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
+      s.setStartRow(startRow);
+      s.setMaxVersions(Integer.MAX_VALUE);
+      Connection conn = ConnectionFactory.createConnection(c1);
+      ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
+
+      int rowCount = 0;
+      int colCount = 0;
+      for (Result result : scanner) {
+        if (result != null && !result.isEmpty()) {
+          rowCount++;
+          colCount += result.size();
+          byte[] row1 = result.getRow();
+          assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
+              entity));
+
+          // check info column family
+          String id1 = EntityColumn.ID.readResult(result).toString();
+          assertEquals(id, id1);
+
+          String type1 = EntityColumn.TYPE.readResult(result).toString();
+          assertEquals(type, type1);
+
+          Number val = (Number) EntityColumn.CREATED_TIME.readResult(result);
+          long cTime1 = val.longValue();
+          assertEquals(cTime1, cTime);
+
+          val = (Number) EntityColumn.MODIFIED_TIME.readResult(result);
+          long mTime1 = val.longValue();
+          assertEquals(mTime1, mTime);
+
+          Map<String, Object> infoColumns =
+              EntityColumnPrefix.INFO.readResults(result);
+          assertEquals(infoMap, infoColumns);
+
+          // Remember isRelatedTo is of type Map<String, Set<String>>
+          for (String isRelatedToKey : isRelatedTo.keySet()) {
+            Object isRelatedToValue =
+                EntityColumnPrefix.IS_RELATED_TO.readResult(result,
+                    isRelatedToKey);
+            String compoundValue = isRelatedToValue.toString();
+            // id7?id9?id6
+            Set<String> isRelatedToValues =
+                new HashSet<String>(
+                    Separator.VALUES.splitEncoded(compoundValue));
+            assertEquals(isRelatedTo.get(isRelatedToKey).size(),
+                isRelatedToValues.size());
+            for (String v : isRelatedTo.get(isRelatedToKey)) {
+              assertTrue(isRelatedToValues.contains(v));
+            }
+          }
+
+          // RelatesTo
+          for (String relatesToKey : relatesTo.keySet()) {
+            String compoundValue =
+                EntityColumnPrefix.RELATES_TO.readResult(result, relatesToKey)
+                    .toString();
+            // id3?id4?id5
+            Set<String> relatesToValues =
+                new HashSet<String>(
+                    Separator.VALUES.splitEncoded(compoundValue));
+            assertEquals(relatesTo.get(relatesToKey).size(),
+                relatesToValues.size());
+            for (String v : relatesTo.get(relatesToKey)) {
+              assertTrue(relatesToValues.contains(v));
+            }
+          }
+
+          // Configuration
+          Map<String, Object> configColumns =
+              EntityColumnPrefix.CONFIG.readResults(result);
+          assertEquals(conf, configColumns);
+
+          NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
+              EntityColumnPrefix.METRIC.readResultsWithTimestamps(result);
+
+          NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
+          assertEquals(metricValues, metricMap);
+        }
+      }
+      assertEquals(1, rowCount);
+      assertEquals(17, colCount);
+
+      // read the timeline entity using the reader this time
+      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
+          entity.getType(), entity.getId(),
+          EnumSet.of(TimelineReader.Field.ALL));
+      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
+          appName, entity.getType(), null, null, null, null, null, null, null,
+          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
+      assertNotNull(e1);
+      assertEquals(1, es1.size());
+
+      // verify attributes
+      assertEquals(id, e1.getId());
+      assertEquals(type, e1.getType());
+      assertEquals(cTime, e1.getCreatedTime());
+      assertEquals(mTime, e1.getModifiedTime());
+      Map<String, Object> infoMap2 = e1.getInfo();
+      assertEquals(infoMap, infoMap2);
+
+      Map<String, Set<String>> isRelatedTo2 = e1.getIsRelatedToEntities();
+      assertEquals(isRelatedTo, isRelatedTo2);
+
+      Map<String, Set<String>> relatesTo2 = e1.getRelatesToEntities();
+      assertEquals(relatesTo, relatesTo2);
+
+      Map<String, String> conf2 = e1.getConfigs();
+      assertEquals(conf, conf2);
+
+      Set<TimelineMetric> metrics2 = e1.getMetrics();
+      assertEquals(metrics, metrics2);
+      for (TimelineMetric metric2 : metrics2) {
+        Map<Long, Number> metricValues2 = metric2.getValues();
+        assertEquals(metricValues, metricValues2);
+      }
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+      if (hbr != null) {
+        hbr.stop();
+        hbr.close();
+      }
+    }
+  }
+
+  private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user,
+      String flow, long runid, String appName, TimelineEntity te) {
+
+    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
+
+    assertTrue(rowKeyComponents.length == 7);
+    assertEquals(user, Bytes.toString(rowKeyComponents[0]));
+    assertEquals(cluster, Bytes.toString(rowKeyComponents[1]));
+    assertEquals(flow, Bytes.toString(rowKeyComponents[2]));
+    assertEquals(TimelineWriterUtils.invert(runid),
+        Bytes.toLong(rowKeyComponents[3]));
+    assertEquals(appName, Bytes.toString(rowKeyComponents[4]));
+    assertEquals(te.getType(), Bytes.toString(rowKeyComponents[5]));
+    assertEquals(te.getId(), Bytes.toString(rowKeyComponents[6]));
+    return true;
+  }
+
+  private boolean isApplicationRowKeyCorrect(byte[] rowKey, String cluster,
+      String user, String flow, long runid, String appName) {
+
+    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
+
+    assertTrue(rowKeyComponents.length == 5);
+    assertEquals(cluster, Bytes.toString(rowKeyComponents[0]));
+    assertEquals(user, Bytes.toString(rowKeyComponents[1]));
+    assertEquals(flow, Bytes.toString(rowKeyComponents[2]));
+    assertEquals(TimelineWriterUtils.invert(runid),
+        Bytes.toLong(rowKeyComponents[3]));
+    assertEquals(appName, Bytes.toString(rowKeyComponents[4]));
+    return true;
+  }
+
+  @Test
+  public void testEvents() throws IOException {
+    TimelineEvent event = new TimelineEvent();
+    String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE;
+    event.setId(eventId);
+    long expTs = 1436512802000L;
+    event.setTimestamp(expTs);
+    String expKey = "foo_event";
+    Object expVal = "test";
+    event.addInfo(expKey, expVal);
+
+    final TimelineEntity entity = new ApplicationEntity();
+    entity.setId(ApplicationId.newInstance(0, 1).toString());
+    entity.addEvent(event);
+
+    TimelineEntities entities = new TimelineEntities();
+    entities.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+      String cluster = "cluster_test_events";
+      String user = "user2";
+      String flow = "other_flow_name";
+      String flowVersion = "1111F01C2287BA";
+      long runid = 1009876543218L;
+      String appName = "some app name";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
+      hbi.stop();
+
+      // retrieve the row
+      byte[] rowKey =
+          ApplicationRowKey.getRowKey(cluster, user, flow, runid, appName);
+      Get get = new Get(rowKey);
+      get.setMaxVersions(Integer.MAX_VALUE);
+      Connection conn = ConnectionFactory.createConnection(c1);
+      Result result = new ApplicationTable().getResult(c1, conn, get);
+
+      assertTrue(result != null);
+
+      // check the row key
+      byte[] row1 = result.getRow();
+      assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
+          appName));
+
+      Map<?, Object> eventsResult =
+          ApplicationColumnPrefix.EVENT.
+              readResultsHavingCompoundColumnQualifiers(result);
+      // there should be only one event
+      assertEquals(1, eventsResult.size());
+      for (Map.Entry<?, Object> e : eventsResult.entrySet()) {
+        // the qualifier is a compound key
+        // hence match individual values
+        byte[][] karr = (byte[][])e.getKey();
+        assertEquals(3, karr.length);
+        assertEquals(eventId, Bytes.toString(karr[0]));
+        assertEquals(TimelineWriterUtils.invert(expTs), Bytes.toLong(karr[1]));
+        assertEquals(expKey, Bytes.toString(karr[2]));
+        Object value = e.getValue();
+        // there should be only one timestamp and value
+        assertEquals(expVal, value.toString());
+      }
+
+      // read the timeline entity using the reader this time
+      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
+          entity.getType(), entity.getId(),
+          EnumSet.of(TimelineReader.Field.ALL));
+      TimelineEntity e2 = hbr.getEntity(user, cluster, null, null, appName,
+          entity.getType(), entity.getId(),
+          EnumSet.of(TimelineReader.Field.ALL));
+      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
+          appName, entity.getType(), null, null, null, null, null, null, null,
+          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
+      Set<TimelineEntity> es2 = hbr.getEntities(user, cluster, null, null,
+          appName, entity.getType(), null, null, null, null, null, null, null,
+          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
+      assertNotNull(e1);
+      assertNotNull(e2);
+      assertEquals(e1, e2);
+      assertEquals(1, es1.size());
+      assertEquals(1, es2.size());
+      assertEquals(es1, es2);
+
+      // check the events
+      NavigableSet<TimelineEvent> events = e1.getEvents();
+      // there should be only one event
+      assertEquals(1, events.size());
+      for (TimelineEvent e : events) {
+        assertEquals(eventId, e.getId());
+        assertEquals(expTs, e.getTimestamp());
+        Map<String,Object> info = e.getInfo();
+        assertEquals(1, info.size());
+        for (Map.Entry<String, Object> infoEntry : info.entrySet()) {
+          assertEquals(expKey, infoEntry.getKey());
+          assertEquals(expVal, infoEntry.getValue());
+        }
+      }
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+      if (hbr != null) {
+        hbr.stop();
+        hbr.close();
+      }
+    }
+  }
+
+  @Test
+  public void testEventsWithEmptyInfo() throws IOException {
+    TimelineEvent event = new TimelineEvent();
+    String eventId = "foo_event_id";
+    event.setId(eventId);
+    long expTs = 1436512802000L;
+    event.setTimestamp(expTs);
+
+    final TimelineEntity entity = new TimelineEntity();
+    entity.setId("attempt_1329348432655_0001_m_000008_18");
+    entity.setType("FOO_ATTEMPT");
+    entity.addEvent(event);
+
+    TimelineEntities entities = new TimelineEntities();
+    entities.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+      String cluster = "cluster_test_empty_eventkey";
+      String user = "user_emptyeventkey";
+      String flow = "other_flow_name";
+      String flowVersion = "1111F01C2287BA";
+      long runid = 1009876543218L;
+      String appName = "some app name";
+      byte[] startRow =
+          EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
+      hbi.stop();
+      // scan the table and see that entity exists
+      Scan s = new Scan();
+      s.setStartRow(startRow);
+      s.addFamily(EntityColumnFamily.INFO.getBytes());
+      Connection conn = ConnectionFactory.createConnection(c1);
+      ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
+
+      int rowCount = 0;
+      for (Result result : scanner) {
+        if (result != null && !result.isEmpty()) {
+          rowCount++;
+
+          // check the row key
+          byte[] row1 = result.getRow();
+          assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
+              entity));
+
+          Map<?, Object> eventsResult =
+              EntityColumnPrefix.EVENT.
+                  readResultsHavingCompoundColumnQualifiers(result);
+          // there should be only one event
+          assertEquals(1, eventsResult.size());
+          for (Map.Entry<?, Object> e : eventsResult.entrySet()) {
+            // the qualifier is a compound key
+            // hence match individual values
+            byte[][] karr = (byte[][])e.getKey();
+            assertEquals(3, karr.length);
+            assertEquals(eventId, Bytes.toString(karr[0]));
+            assertEquals(TimelineWriterUtils.invert(expTs),
+                Bytes.toLong(karr[1]));
+            // key must be empty
+            assertEquals(0, karr[2].length);
+            Object value = e.getValue();
+            // value should be empty
+            assertEquals("", value.toString());
+          }
+        }
+      }
+      assertEquals(1, rowCount);
+
+      // read the timeline entity using the reader this time
+      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
+          entity.getType(), entity.getId(),
+          EnumSet.of(TimelineReader.Field.ALL));
+      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
+          appName, entity.getType(), null, null, null, null, null, null, null,
+          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
+      assertNotNull(e1);
+      assertEquals(1, es1.size());
+
+      // check the events
+      NavigableSet<TimelineEvent> events = e1.getEvents();
+      // there should be only one event
+      assertEquals(1, events.size());
+      for (TimelineEvent e : events) {
+        assertEquals(eventId, e.getId());
+        assertEquals(expTs, e.getTimestamp());
+        Map<String,Object> info = e.getInfo();
+        assertTrue(info == null || info.isEmpty());
+      }
+    } finally {
+      hbi.stop();
+      hbi.close();
+      hbr.stop();;
+      hbr.close();
+    }
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    util.shutdownMiniCluster();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a057b289/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.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/TestHBaseTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
deleted file mode 100644
index 2875e01..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
+++ /dev/null
@@ -1,770 +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;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Get;
-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.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
-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.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.storage.application.ApplicationColumn;
-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.AppToFlowTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnFamily;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-/**
- * Various tests to test writing entities to HBase and reading them back from
- * it.
- *
- * It uses a single HBase mini-cluster for all tests which is a little more
- * realistic, and helps test correctness in the presence of other data.
- *
- * Each test uses a different cluster name to be able to handle its own data
- * even if other records exist in the table. Use a different cluster name if
- * you add a new test.
- */
-public class TestHBaseTimelineStorage {
-
-  private static HBaseTestingUtility util;
-
-  @BeforeClass
-  public static void setupBeforeClass() throws Exception {
-    util = new HBaseTestingUtility();
-    util.startMiniCluster();
-    createSchema();
-  }
-
-  private static void createSchema() throws IOException {
-    new EntityTable()
-        .createTable(util.getHBaseAdmin(), util.getConfiguration());
-    new AppToFlowTable()
-        .createTable(util.getHBaseAdmin(), util.getConfiguration());
-    new ApplicationTable()
-        .createTable(util.getHBaseAdmin(), util.getConfiguration());
-  }
-
-  @Test
-  public void testWriteApplicationToHBase() throws Exception {
-    TimelineEntities te = new TimelineEntities();
-    ApplicationEntity entity = new ApplicationEntity();
-    String id = "hello";
-    entity.setId(id);
-    long cTime = 1425016501000L;
-    long mTime = 1425026901000L;
-    entity.setCreatedTime(cTime);
-    entity.setModifiedTime(mTime);
-
-    // add the info map in Timeline Entity
-    Map<String, Object> infoMap = new HashMap<String, Object>();
-    infoMap.put("infoMapKey1", "infoMapValue1");
-    infoMap.put("infoMapKey2", 10);
-    entity.addInfo(infoMap);
-
-    // add the isRelatedToEntity info
-    String key = "task";
-    String value = "is_related_to_entity_id_here";
-    Set<String> isRelatedToSet = new HashSet<String>();
-    isRelatedToSet.add(value);
-    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
-    isRelatedTo.put(key, isRelatedToSet);
-    entity.setIsRelatedToEntities(isRelatedTo);
-
-    // add the relatesTo info
-    key = "container";
-    value = "relates_to_entity_id_here";
-    Set<String> relatesToSet = new HashSet<String>();
-    relatesToSet.add(value);
-    value = "relates_to_entity_id_here_Second";
-    relatesToSet.add(value);
-    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
-    relatesTo.put(key, relatesToSet);
-    entity.setRelatesToEntities(relatesTo);
-
-    // add some config entries
-    Map<String, String> conf = new HashMap<String, String>();
-    conf.put("config_param1", "value1");
-    conf.put("config_param2", "value2");
-    entity.addConfigs(conf);
-
-    // add metrics
-    Set<TimelineMetric> metrics = new HashSet<>();
-    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);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    entity.addMetrics(metrics);
-
-    te.addEntity(entity);
-
-    HBaseTimelineWriterImpl hbi = null;
-    HBaseTimelineReaderImpl hbr = null;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
-      String cluster = "cluster_test_write_app";
-      String user = "user1";
-      String flow = "some_flow_name";
-      String flowVersion = "AB7822C10F1111";
-      long runid = 1002345678919L;
-      hbi.write(cluster, user, flow, flowVersion, runid, id, te);
-      hbi.stop();
-
-      // retrieve the row
-      byte[] rowKey =
-          ApplicationRowKey.getRowKey(cluster, user, flow, runid, id);
-      Get get = new Get(rowKey);
-      get.setMaxVersions(Integer.MAX_VALUE);
-      Connection conn = ConnectionFactory.createConnection(c1);
-      Result result = new ApplicationTable().getResult(c1, conn, get);
-
-      assertTrue(result != null);
-      assertEquals(16, result.size());
-
-      // check the row key
-      byte[] row1 = result.getRow();
-      assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
-          id));
-
-      // check info column family
-      String id1 = ApplicationColumn.ID.readResult(result).toString();
-      assertEquals(id, id1);
-
-      Number val =
-          (Number) ApplicationColumn.CREATED_TIME.readResult(result);
-      long cTime1 = val.longValue();
-      assertEquals(cTime1, cTime);
-
-      val = (Number) ApplicationColumn.MODIFIED_TIME.readResult(result);
-      long mTime1 = val.longValue();
-      assertEquals(mTime1, mTime);
-
-      Map<String, Object> infoColumns =
-          ApplicationColumnPrefix.INFO.readResults(result);
-      assertEquals(infoMap, infoColumns);
-
-      // Remember isRelatedTo is of type Map<String, Set<String>>
-      for (String isRelatedToKey : isRelatedTo.keySet()) {
-        Object isRelatedToValue =
-            ApplicationColumnPrefix.IS_RELATED_TO.readResult(result,
-                isRelatedToKey);
-        String compoundValue = isRelatedToValue.toString();
-        // id7?id9?id6
-        Set<String> isRelatedToValues =
-            new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
-        assertEquals(isRelatedTo.get(isRelatedToKey).size(),
-            isRelatedToValues.size());
-        for (String v : isRelatedTo.get(isRelatedToKey)) {
-          assertTrue(isRelatedToValues.contains(v));
-        }
-      }
-
-      // RelatesTo
-      for (String relatesToKey : relatesTo.keySet()) {
-        String compoundValue =
-            ApplicationColumnPrefix.RELATES_TO.readResult(result,
-                relatesToKey).toString();
-        // id3?id4?id5
-        Set<String> relatesToValues =
-            new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
-        assertEquals(relatesTo.get(relatesToKey).size(),
-            relatesToValues.size());
-        for (String v : relatesTo.get(relatesToKey)) {
-          assertTrue(relatesToValues.contains(v));
-        }
-      }
-
-      // Configuration
-      Map<String, Object> configColumns =
-          ApplicationColumnPrefix.CONFIG.readResults(result);
-      assertEquals(conf, configColumns);
-
-      NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
-          ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result);
-
-      NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
-      assertEquals(metricValues, metricMap);
-
-      // read the timeline entity using the reader this time
-      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, id,
-          entity.getType(), entity.getId(),
-          EnumSet.of(TimelineReader.Field.ALL));
-      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
-          id, entity.getType(), null, null, null, null, null, null, null,
-          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
-      assertNotNull(e1);
-      assertEquals(1, es1.size());
-
-      // verify attributes
-      assertEquals(id, e1.getId());
-      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
-          e1.getType());
-      assertEquals(cTime, e1.getCreatedTime());
-      assertEquals(mTime, e1.getModifiedTime());
-      Map<String, Object> infoMap2 = e1.getInfo();
-      assertEquals(infoMap, infoMap2);
-
-      Map<String, Set<String>> isRelatedTo2 = e1.getIsRelatedToEntities();
-      assertEquals(isRelatedTo, isRelatedTo2);
-
-      Map<String, Set<String>> relatesTo2 = e1.getRelatesToEntities();
-      assertEquals(relatesTo, relatesTo2);
-
-      Map<String, String> conf2 = e1.getConfigs();
-      assertEquals(conf, conf2);
-
-      Set<TimelineMetric> metrics2 = e1.getMetrics();
-      assertEquals(metrics, metrics2);
-      for (TimelineMetric metric2 : metrics2) {
-        Map<Long, Number> metricValues2 = metric2.getValues();
-        assertEquals(metricValues, metricValues2);
-      }
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-      if (hbr != null) {
-        hbr.stop();
-        hbr.close();
-      }
-    }
-  }
-
-  @Test
-  public void testWriteEntityToHBase() throws Exception {
-    TimelineEntities te = new TimelineEntities();
-    TimelineEntity entity = new TimelineEntity();
-    String id = "hello";
-    String type = "world";
-    entity.setId(id);
-    entity.setType(type);
-    long cTime = 1425016501000L;
-    long mTime = 1425026901000L;
-    entity.setCreatedTime(cTime);
-    entity.setModifiedTime(mTime);
-
-    // add the info map in Timeline Entity
-    Map<String, Object> infoMap = new HashMap<String, Object>();
-    infoMap.put("infoMapKey1", "infoMapValue1");
-    infoMap.put("infoMapKey2", 10);
-    entity.addInfo(infoMap);
-
-    // add the isRelatedToEntity info
-    String key = "task";
-    String value = "is_related_to_entity_id_here";
-    Set<String> isRelatedToSet = new HashSet<String>();
-    isRelatedToSet.add(value);
-    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
-    isRelatedTo.put(key, isRelatedToSet);
-    entity.setIsRelatedToEntities(isRelatedTo);
-
-    // add the relatesTo info
-    key = "container";
-    value = "relates_to_entity_id_here";
-    Set<String> relatesToSet = new HashSet<String>();
-    relatesToSet.add(value);
-    value = "relates_to_entity_id_here_Second";
-    relatesToSet.add(value);
-    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
-    relatesTo.put(key, relatesToSet);
-    entity.setRelatesToEntities(relatesTo);
-
-    // add some config entries
-    Map<String, String> conf = new HashMap<String, String>();
-    conf.put("config_param1", "value1");
-    conf.put("config_param2", "value2");
-    entity.addConfigs(conf);
-
-    // add metrics
-    Set<TimelineMetric> metrics = new HashSet<>();
-    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);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    entity.addMetrics(metrics);
-
-    te.addEntity(entity);
-
-    HBaseTimelineWriterImpl hbi = null;
-    HBaseTimelineReaderImpl hbr = null;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
-      String cluster = "cluster_test_write_entity";
-      String user = "user1";
-      String flow = "some_flow_name";
-      String flowVersion = "AB7822C10F1111";
-      long runid = 1002345678919L;
-      String appName = "some app name";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
-      hbi.stop();
-
-      // scan the table and see that entity exists
-      Scan s = new Scan();
-      byte[] startRow =
-          EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
-      s.setStartRow(startRow);
-      s.setMaxVersions(Integer.MAX_VALUE);
-      Connection conn = ConnectionFactory.createConnection(c1);
-      ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
-
-      int rowCount = 0;
-      int colCount = 0;
-      for (Result result : scanner) {
-        if (result != null && !result.isEmpty()) {
-          rowCount++;
-          colCount += result.size();
-          byte[] row1 = result.getRow();
-          assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
-              entity));
-
-          // check info column family
-          String id1 = EntityColumn.ID.readResult(result).toString();
-          assertEquals(id, id1);
-
-          String type1 = EntityColumn.TYPE.readResult(result).toString();
-          assertEquals(type, type1);
-
-          Number val = (Number) EntityColumn.CREATED_TIME.readResult(result);
-          long cTime1 = val.longValue();
-          assertEquals(cTime1, cTime);
-
-          val = (Number) EntityColumn.MODIFIED_TIME.readResult(result);
-          long mTime1 = val.longValue();
-          assertEquals(mTime1, mTime);
-
-          Map<String, Object> infoColumns =
-              EntityColumnPrefix.INFO.readResults(result);
-          assertEquals(infoMap, infoColumns);
-
-          // Remember isRelatedTo is of type Map<String, Set<String>>
-          for (String isRelatedToKey : isRelatedTo.keySet()) {
-            Object isRelatedToValue =
-                EntityColumnPrefix.IS_RELATED_TO.readResult(result,
-                    isRelatedToKey);
-            String compoundValue = isRelatedToValue.toString();
-            // id7?id9?id6
-            Set<String> isRelatedToValues =
-                new HashSet<String>(
-                    Separator.VALUES.splitEncoded(compoundValue));
-            assertEquals(isRelatedTo.get(isRelatedToKey).size(),
-                isRelatedToValues.size());
-            for (String v : isRelatedTo.get(isRelatedToKey)) {
-              assertTrue(isRelatedToValues.contains(v));
-            }
-          }
-
-          // RelatesTo
-          for (String relatesToKey : relatesTo.keySet()) {
-            String compoundValue =
-                EntityColumnPrefix.RELATES_TO.readResult(result, relatesToKey)
-                    .toString();
-            // id3?id4?id5
-            Set<String> relatesToValues =
-                new HashSet<String>(
-                    Separator.VALUES.splitEncoded(compoundValue));
-            assertEquals(relatesTo.get(relatesToKey).size(),
-                relatesToValues.size());
-            for (String v : relatesTo.get(relatesToKey)) {
-              assertTrue(relatesToValues.contains(v));
-            }
-          }
-
-          // Configuration
-          Map<String, Object> configColumns =
-              EntityColumnPrefix.CONFIG.readResults(result);
-          assertEquals(conf, configColumns);
-
-          NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
-              EntityColumnPrefix.METRIC.readResultsWithTimestamps(result);
-
-          NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
-          assertEquals(metricValues, metricMap);
-        }
-      }
-      assertEquals(1, rowCount);
-      assertEquals(17, colCount);
-
-      // read the timeline entity using the reader this time
-      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
-          entity.getType(), entity.getId(),
-          EnumSet.of(TimelineReader.Field.ALL));
-      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
-          appName, entity.getType(), null, null, null, null, null, null, null,
-          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
-      assertNotNull(e1);
-      assertEquals(1, es1.size());
-
-      // verify attributes
-      assertEquals(id, e1.getId());
-      assertEquals(type, e1.getType());
-      assertEquals(cTime, e1.getCreatedTime());
-      assertEquals(mTime, e1.getModifiedTime());
-      Map<String, Object> infoMap2 = e1.getInfo();
-      assertEquals(infoMap, infoMap2);
-
-      Map<String, Set<String>> isRelatedTo2 = e1.getIsRelatedToEntities();
-      assertEquals(isRelatedTo, isRelatedTo2);
-
-      Map<String, Set<String>> relatesTo2 = e1.getRelatesToEntities();
-      assertEquals(relatesTo, relatesTo2);
-
-      Map<String, String> conf2 = e1.getConfigs();
-      assertEquals(conf, conf2);
-
-      Set<TimelineMetric> metrics2 = e1.getMetrics();
-      assertEquals(metrics, metrics2);
-      for (TimelineMetric metric2 : metrics2) {
-        Map<Long, Number> metricValues2 = metric2.getValues();
-        assertEquals(metricValues, metricValues2);
-      }
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-      if (hbr != null) {
-        hbr.stop();
-        hbr.close();
-      }
-    }
-  }
-
-  private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user,
-      String flow, long runid, String appName, TimelineEntity te) {
-
-    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
-
-    assertTrue(rowKeyComponents.length == 7);
-    assertEquals(user, Bytes.toString(rowKeyComponents[0]));
-    assertEquals(cluster, Bytes.toString(rowKeyComponents[1]));
-    assertEquals(flow, Bytes.toString(rowKeyComponents[2]));
-    assertEquals(TimelineWriterUtils.invert(runid),
-        Bytes.toLong(rowKeyComponents[3]));
-    assertEquals(appName, Bytes.toString(rowKeyComponents[4]));
-    assertEquals(te.getType(), Bytes.toString(rowKeyComponents[5]));
-    assertEquals(te.getId(), Bytes.toString(rowKeyComponents[6]));
-    return true;
-  }
-
-  private boolean isApplicationRowKeyCorrect(byte[] rowKey, String cluster,
-      String user, String flow, long runid, String appName) {
-
-    byte[][] rowKeyComponents = Separator.QUALIFIERS.split(rowKey);
-
-    assertTrue(rowKeyComponents.length == 5);
-    assertEquals(cluster, Bytes.toString(rowKeyComponents[0]));
-    assertEquals(user, Bytes.toString(rowKeyComponents[1]));
-    assertEquals(flow, Bytes.toString(rowKeyComponents[2]));
-    assertEquals(TimelineWriterUtils.invert(runid),
-        Bytes.toLong(rowKeyComponents[3]));
-    assertEquals(appName, Bytes.toString(rowKeyComponents[4]));
-    return true;
-  }
-
-  @Test
-  public void testEvents() throws IOException {
-    TimelineEvent event = new TimelineEvent();
-    String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE;
-    event.setId(eventId);
-    long expTs = 1436512802000L;
-    event.setTimestamp(expTs);
-    String expKey = "foo_event";
-    Object expVal = "test";
-    event.addInfo(expKey, expVal);
-
-    final TimelineEntity entity = new ApplicationEntity();
-    entity.setId(ApplicationId.newInstance(0, 1).toString());
-    entity.addEvent(event);
-
-    TimelineEntities entities = new TimelineEntities();
-    entities.addEntity(entity);
-
-    HBaseTimelineWriterImpl hbi = null;
-    HBaseTimelineReaderImpl hbr = null;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
-      String cluster = "cluster_test_events";
-      String user = "user2";
-      String flow = "other_flow_name";
-      String flowVersion = "1111F01C2287BA";
-      long runid = 1009876543218L;
-      String appName = "some app name";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
-      hbi.stop();
-
-      // retrieve the row
-      byte[] rowKey =
-          ApplicationRowKey.getRowKey(cluster, user, flow, runid, appName);
-      Get get = new Get(rowKey);
-      get.setMaxVersions(Integer.MAX_VALUE);
-      Connection conn = ConnectionFactory.createConnection(c1);
-      Result result = new ApplicationTable().getResult(c1, conn, get);
-
-      assertTrue(result != null);
-
-      // check the row key
-      byte[] row1 = result.getRow();
-      assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
-          appName));
-
-      Map<?, Object> eventsResult =
-          ApplicationColumnPrefix.EVENT.
-              readResultsHavingCompoundColumnQualifiers(result);
-      // there should be only one event
-      assertEquals(1, eventsResult.size());
-      for (Map.Entry<?, Object> e : eventsResult.entrySet()) {
-        // the qualifier is a compound key
-        // hence match individual values
-        byte[][] karr = (byte[][])e.getKey();
-        assertEquals(3, karr.length);
-        assertEquals(eventId, Bytes.toString(karr[0]));
-        assertEquals(TimelineWriterUtils.invert(expTs), Bytes.toLong(karr[1]));
-        assertEquals(expKey, Bytes.toString(karr[2]));
-        Object value = e.getValue();
-        // there should be only one timestamp and value
-        assertEquals(expVal, value.toString());
-      }
-
-      // read the timeline entity using the reader this time
-      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
-          entity.getType(), entity.getId(),
-          EnumSet.of(TimelineReader.Field.ALL));
-      TimelineEntity e2 = hbr.getEntity(user, cluster, null, null, appName,
-          entity.getType(), entity.getId(),
-          EnumSet.of(TimelineReader.Field.ALL));
-      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
-          appName, entity.getType(), null, null, null, null, null, null, null,
-          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
-      Set<TimelineEntity> es2 = hbr.getEntities(user, cluster, null, null,
-          appName, entity.getType(), null, null, null, null, null, null, null,
-          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
-      assertNotNull(e1);
-      assertNotNull(e2);
-      assertEquals(e1, e2);
-      assertEquals(1, es1.size());
-      assertEquals(1, es2.size());
-      assertEquals(es1, es2);
-
-      // check the events
-      NavigableSet<TimelineEvent> events = e1.getEvents();
-      // there should be only one event
-      assertEquals(1, events.size());
-      for (TimelineEvent e : events) {
-        assertEquals(eventId, e.getId());
-        assertEquals(expTs, e.getTimestamp());
-        Map<String,Object> info = e.getInfo();
-        assertEquals(1, info.size());
-        for (Map.Entry<String, Object> infoEntry : info.entrySet()) {
-          assertEquals(expKey, infoEntry.getKey());
-          assertEquals(expVal, infoEntry.getValue());
-        }
-      }
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-      if (hbr != null) {
-        hbr.stop();
-        hbr.close();
-      }
-    }
-  }
-
-  @Test
-  public void testEventsWithEmptyInfo() throws IOException {
-    TimelineEvent event = new TimelineEvent();
-    String eventId = "foo_event_id";
-    event.setId(eventId);
-    long expTs = 1436512802000L;
-    event.setTimestamp(expTs);
-
-    final TimelineEntity entity = new TimelineEntity();
-    entity.setId("attempt_1329348432655_0001_m_000008_18");
-    entity.setType("FOO_ATTEMPT");
-    entity.addEvent(event);
-
-    TimelineEntities entities = new TimelineEntities();
-    entities.addEntity(entity);
-
-    HBaseTimelineWriterImpl hbi = null;
-    HBaseTimelineReaderImpl hbr = null;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
-      String cluster = "cluster_test_empty_eventkey";
-      String user = "user_emptyeventkey";
-      String flow = "other_flow_name";
-      String flowVersion = "1111F01C2287BA";
-      long runid = 1009876543218L;
-      String appName = "some app name";
-      byte[] startRow =
-          EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
-      hbi.stop();
-      // scan the table and see that entity exists
-      Scan s = new Scan();
-      s.setStartRow(startRow);
-      s.addFamily(EntityColumnFamily.INFO.getBytes());
-      Connection conn = ConnectionFactory.createConnection(c1);
-      ResultScanner scanner = new EntityTable().getResultScanner(c1, conn, s);
-
-      int rowCount = 0;
-      for (Result result : scanner) {
-        if (result != null && !result.isEmpty()) {
-          rowCount++;
-
-          // check the row key
-          byte[] row1 = result.getRow();
-          assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
-              entity));
-
-          Map<?, Object> eventsResult =
-              EntityColumnPrefix.EVENT.
-                  readResultsHavingCompoundColumnQualifiers(result);
-          // there should be only one event
-          assertEquals(1, eventsResult.size());
-          for (Map.Entry<?, Object> e : eventsResult.entrySet()) {
-            // the qualifier is a compound key
-            // hence match individual values
-            byte[][] karr = (byte[][])e.getKey();
-            assertEquals(3, karr.length);
-            assertEquals(eventId, Bytes.toString(karr[0]));
-            assertEquals(TimelineWriterUtils.invert(expTs),
-                Bytes.toLong(karr[1]));
-            // key must be empty
-            assertEquals(0, karr[2].length);
-            Object value = e.getValue();
-            // value should be empty
-            assertEquals("", value.toString());
-          }
-        }
-      }
-      assertEquals(1, rowCount);
-
-      // read the timeline entity using the reader this time
-      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
-          entity.getType(), entity.getId(),
-          EnumSet.of(TimelineReader.Field.ALL));
-      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
-          appName, entity.getType(), null, null, null, null, null, null, null,
-          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
-      assertNotNull(e1);
-      assertEquals(1, es1.size());
-
-      // check the events
-      NavigableSet<TimelineEvent> events = e1.getEvents();
-      // there should be only one event
-      assertEquals(1, events.size());
-      for (TimelineEvent e : events) {
-        assertEquals(eventId, e.getId());
-        assertEquals(expTs, e.getTimestamp());
-        Map<String,Object> info = e.getInfo();
-        assertTrue(info == null || info.isEmpty());
-      }
-    } finally {
-      hbi.stop();
-      hbi.close();
-      hbr.stop();;
-      hbr.close();
-    }
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    util.shutdownMiniCluster();
-  }
-}


[04/50] [abbrv] hadoop git commit: YARN-3049. [Storage Implementation] Implement storage reader interface to fetch raw data from HBase backend (Zhijie Shen via sjlee)

Posted by gt...@apache.org.
YARN-3049. [Storage Implementation] Implement storage reader interface to fetch raw data from HBase backend (Zhijie Shen via sjlee)

(cherry picked from commit 07433c2ad52df9e844dbd90020c277d3df844dcd)


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

Branch: refs/heads/feature-YARN-2928
Commit: 0bed3fb3b3793fc0fc4b838c024cf5da7c2cc291
Parents: 2d97f86
Author: Sangjin Lee <sj...@apache.org>
Authored: Fri Aug 7 10:00:22 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:37:46 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../dev-support/findbugs-exclude.xml            |   5 +
 .../records/timelineservice/TimelineEntity.java |   9 +-
 .../storage/FileSystemTimelineReaderImpl.java   | 164 +++----
 .../storage/HBaseTimelineReaderImpl.java        | 424 +++++++++++++++++++
 .../storage/HBaseTimelineWriterImpl.java        |  43 +-
 .../storage/TimelineSchemaCreator.java          |  12 +
 .../storage/apptoflow/AppToFlowColumn.java      | 126 ++++++
 .../apptoflow/AppToFlowColumnFamily.java        |  51 +++
 .../storage/apptoflow/AppToFlowRowKey.java      |  39 ++
 .../storage/apptoflow/AppToFlowTable.java       | 110 +++++
 .../storage/apptoflow/package-info.java         |  23 +
 .../storage/common/BaseTable.java               |  16 +
 .../storage/common/ColumnPrefix.java            |   2 +-
 .../common/TimelineEntitySchemaConstants.java   |  68 ---
 .../common/TimelineHBaseSchemaConstants.java    |  68 +++
 .../storage/common/TimelineReaderUtils.java     | 112 +++++
 .../storage/entity/EntityColumn.java            |   2 +-
 .../storage/entity/EntityColumnFamily.java      |   2 +-
 .../storage/entity/EntityColumnPrefix.java      |   2 +-
 .../storage/entity/EntityRowKey.java            |  36 +-
 .../storage/entity/EntityTable.java             |   8 +-
 .../storage/TestHBaseTimelineWriterImpl.java    |  82 +++-
 23 files changed, 1197 insertions(+), 210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f40cfe8..0378df8 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -82,6 +82,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3949. Ensure timely flush of timeline writes. (Sangjin Lee via
     junping_du)
 
+    YARN-3049. [Storage Implementation] Implement storage reader interface to
+    fetch raw data from HBase backend (Zhijie Shen via sjlee)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index 7505645..0dcdd15 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -522,4 +522,9 @@
     <Package name="org.apache.hadoop.yarn.api.records.impl.pb" />
     <Bug pattern="NP_BOOLEAN_RETURN_NULL" />
   </Match>
+  <!-- Object cast is based on the event type -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.server.resourcemanager.metrics.AbstractTimelineServicePublisher" />
+     <Bug pattern="BC_UNCONFIRMED_CAST" />
+  </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java
index 9ef2d90..0701001 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineEntity.java
@@ -29,7 +29,9 @@ import javax.xml.bind.annotation.XmlRootElement;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import java.util.NavigableSet;
 import java.util.Set;
+import java.util.TreeSet;
 
 /**
  * The basic timeline entity data structure for timeline service v2. Timeline
@@ -133,7 +135,8 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
   private HashMap<String, Object> info = new HashMap<>();
   private HashMap<String, String> configs = new HashMap<>();
   private Set<TimelineMetric> metrics = new HashSet<>();
-  private Set<TimelineEvent> events = new HashSet<>();
+  // events should be sorted by timestamp in descending order
+  private NavigableSet<TimelineEvent> events = new TreeSet<>();
   private HashMap<String, Set<String>> isRelatedToEntities = new HashMap<>();
   private HashMap<String, Set<String>> relatesToEntities = new HashMap<>();
   private long createdTime;
@@ -334,7 +337,7 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
   }
 
   @XmlElement(name = "events")
-  public Set<TimelineEvent> getEvents() {
+  public NavigableSet<TimelineEvent> getEvents() {
     if (real == null) {
       return events;
     } else {
@@ -342,7 +345,7 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
     }
   }
 
-  public void setEvents(Set<TimelineEvent> events) {
+  public void setEvents(NavigableSet<TimelineEvent> events) {
     if (real == null) {
       this.events = events;
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/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 f9f1d1d..45ddd1d 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
@@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineReaderUtils;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.map.JsonMappingException;
@@ -119,59 +120,32 @@ public class FileSystemTimelineReaderImpl extends AbstractService
   private static void fillFields(TimelineEntity finalEntity,
       TimelineEntity real, EnumSet<Field> fields) {
     if (fields.contains(Field.ALL)) {
-      finalEntity.setConfigs(real.getConfigs());
-      finalEntity.setMetrics(real.getMetrics());
-      finalEntity.setInfo(real.getInfo());
-      finalEntity.setIsRelatedToEntities(real.getIsRelatedToEntities());
-      finalEntity.setIsRelatedToEntities(real.getIsRelatedToEntities());
-      finalEntity.setEvents(real.getEvents());
-      return;
+      fields = EnumSet.allOf(Field.class);
     }
     for (Field field : fields) {
       switch(field) {
-      case CONFIGS:
-        finalEntity.setConfigs(real.getConfigs());
-        break;
-      case METRICS:
-        finalEntity.setMetrics(real.getMetrics());
-        break;
-      case INFO:
-        finalEntity.setInfo(real.getInfo());
-        break;
-      case IS_RELATED_TO:
-        finalEntity.setIsRelatedToEntities(real.getIsRelatedToEntities());
-        break;
-      case RELATES_TO:
-        finalEntity.setIsRelatedToEntities(real.getIsRelatedToEntities());
-        break;
-      case EVENTS:
-        finalEntity.setEvents(real.getEvents());
-        break;
-      default:
-        continue;
-      }
-    }
-  }
-
-  private static boolean matchFilter(Object infoValue, Object filterValue) {
-    return infoValue.equals(filterValue);
-  }
-
-  private static boolean matchFilters(Map<String, ? extends Object> entityInfo,
-      Map<String, ? extends Object> filters) {
-    if (entityInfo == null || entityInfo.isEmpty()) {
-      return false;
-    }
-    for (Map.Entry<String, ? extends Object> filter : filters.entrySet()) {
-      Object infoValue = entityInfo.get(filter.getKey());
-      if (infoValue == null) {
-        return false;
-      }
-      if (!matchFilter(infoValue, filter.getValue())) {
-        return false;
+        case CONFIGS:
+          finalEntity.setConfigs(real.getConfigs());
+          break;
+        case METRICS:
+          finalEntity.setMetrics(real.getMetrics());
+          break;
+        case INFO:
+          finalEntity.setInfo(real.getInfo());
+          break;
+        case IS_RELATED_TO:
+          finalEntity.setIsRelatedToEntities(real.getIsRelatedToEntities());
+          break;
+        case RELATES_TO:
+          finalEntity.setIsRelatedToEntities(real.getIsRelatedToEntities());
+          break;
+        case EVENTS:
+          finalEntity.setEvents(real.getEvents());
+          break;
+        default:
+          continue;
       }
     }
-    return true;
   }
 
   private String getFlowRunPath(String userId, String clusterId, String flowId,
@@ -186,10 +160,10 @@ public class FileSystemTimelineReaderImpl extends AbstractService
     String appFlowMappingFile = rootPath + "/" +  ENTITIES_DIR + "/" +
         clusterId + "/" + APP_FLOW_MAPPING_FILE;
     try (BufferedReader reader =
-        new BufferedReader(new InputStreamReader(
-            new FileInputStream(
-                appFlowMappingFile), Charset.forName("UTF-8")));
-        CSVParser parser = new CSVParser(reader, csvFormat)) {
+             new BufferedReader(new InputStreamReader(
+                 new FileInputStream(
+                     appFlowMappingFile), Charset.forName("UTF-8")));
+         CSVParser parser = new CSVParser(reader, csvFormat)) {
       for (CSVRecord record : parser.getRecords()) {
         if (record.size() < 4) {
           continue;
@@ -207,36 +181,6 @@ public class FileSystemTimelineReaderImpl extends AbstractService
     throw new IOException("Unable to get flow info");
   }
 
-  private static boolean matchMetricFilters(Set<TimelineMetric> metrics,
-      Set<String> metricFilters) {
-    Set<String> tempMetrics = new HashSet<String>();
-    for (TimelineMetric metric : metrics) {
-      tempMetrics.add(metric.getId());
-    }
-
-    for (String metricFilter : metricFilters) {
-      if (!tempMetrics.contains(metricFilter)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  private static boolean matchEventFilters(Set<TimelineEvent> entityEvents,
-      Set<String> eventFilters) {
-    Set<String> tempEvents = new HashSet<String>();
-    for (TimelineEvent event : entityEvents) {
-      tempEvents.add(event.getId());
-    }
-
-    for (String eventFilter : eventFilters) {
-      if (!tempEvents.contains(eventFilter)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
   private static TimelineEntity createEntityToBeReturned(TimelineEntity entity,
       EnumSet<Field> fieldsToRetrieve) {
     TimelineEntity entityToBeReturned = new TimelineEntity();
@@ -254,23 +198,6 @@ public class FileSystemTimelineReaderImpl extends AbstractService
     return (time >= timeBegin) && (time <= timeEnd);
   }
 
-  private static boolean matchRelations(
-      Map<String, Set<String>> entityRelations,
-      Map<String, Set<String>> relations) {
-    for (Map.Entry<String, Set<String>> relation : relations.entrySet()) {
-      Set<String> ids = entityRelations.get(relation.getKey());
-      if (ids == null) {
-        return false;
-      }
-      for (String id : relation.getValue()) {
-        if (!ids.contains(id)) {
-          return false;
-        }
-      }
-    }
-    return true;
-  }
-
   private static void mergeEntities(TimelineEntity entity1,
       TimelineEntity entity2) {
     // Ideally created time wont change except in the case of issue from client.
@@ -364,22 +291,22 @@ public class FileSystemTimelineReaderImpl extends AbstractService
     // First sort the selected entities based on created/start time.
     Map<Long, Set<TimelineEntity>> sortedEntities =
         new TreeMap<>(
-          new Comparator<Long>() {
-            @Override
-            public int compare(Long l1, Long l2) {
-              return l2.compareTo(l1);
+            new Comparator<Long>() {
+              @Override
+              public int compare(Long l1, Long l2) {
+                return l2.compareTo(l1);
+              }
             }
-          }
         );
     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")))) {
+               new BufferedReader(
+                   new InputStreamReader(
+                       new FileInputStream(
+                           entityFile), Charset.forName("UTF-8")))) {
         TimelineEntity entity = readEntityFromFile(reader);
         if (!entity.getType().equals(entityType)) {
           continue;
@@ -393,27 +320,32 @@ public class FileSystemTimelineReaderImpl extends AbstractService
           continue;
         }
         if (relatesTo != null && !relatesTo.isEmpty() &&
-            !matchRelations(entity.getRelatesToEntities(), relatesTo)) {
+            !TimelineReaderUtils
+                .matchRelations(entity.getRelatesToEntities(), relatesTo)) {
           continue;
         }
         if (isRelatedTo != null && !isRelatedTo.isEmpty() &&
-            !matchRelations(entity.getIsRelatedToEntities(), isRelatedTo)) {
+            !TimelineReaderUtils
+                .matchRelations(entity.getIsRelatedToEntities(), isRelatedTo)) {
           continue;
         }
         if (infoFilters != null && !infoFilters.isEmpty() &&
-            !matchFilters(entity.getInfo(), infoFilters)) {
+            !TimelineReaderUtils.matchFilters(entity.getInfo(), infoFilters)) {
           continue;
         }
         if (configFilters != null && !configFilters.isEmpty() &&
-            !matchFilters(entity.getConfigs(), configFilters)) {
+            !TimelineReaderUtils.matchFilters(
+                entity.getConfigs(), configFilters)) {
           continue;
         }
         if (metricFilters != null && !metricFilters.isEmpty() &&
-            !matchMetricFilters(entity.getMetrics(), metricFilters)) {
+            !TimelineReaderUtils.matchMetricFilters(
+                entity.getMetrics(), metricFilters)) {
           continue;
         }
         if (eventFilters != null && !eventFilters.isEmpty() &&
-            !matchEventFilters(entity.getEvents(), eventFilters)) {
+            !TimelineReaderUtils.matchEventFilters(
+                entity.getEvents(), eventFilters)) {
           continue;
         }
         TimelineEntity entityToBeReturned =
@@ -461,8 +393,8 @@ public class FileSystemTimelineReaderImpl extends AbstractService
     File entityFile =
         new File(dir, entityId + TIMELINE_SERVICE_STORAGE_EXTENSION);
     try (BufferedReader reader =
-        new BufferedReader(new InputStreamReader(
-            new FileInputStream(entityFile), Charset.forName("UTF-8")))) {
+             new BufferedReader(new InputStreamReader(
+                 new FileInputStream(entityFile), Charset.forName("UTF-8")))) {
       TimelineEntity entity = readEntityFromFile(reader);
       return createEntityToBeReturned(entity, fieldsToRetrieve);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.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/HBaseTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java
new file mode 100644
index 0000000..5258b9c
--- /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/HBaseTimelineReaderImpl.java
@@ -0,0 +1,424 @@
+/**
+ * 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;
+
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Get;
+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.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumn;
+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.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineReaderUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.TreeSet;
+
+public class HBaseTimelineReaderImpl
+    extends AbstractService implements TimelineReader {
+
+  private static final Log LOG = LogFactory
+      .getLog(HBaseTimelineReaderImpl.class);
+  private static final long DEFAULT_BEGIN_TIME = 0L;
+  private static final long DEFAULT_END_TIME = Long.MAX_VALUE;
+
+  private Configuration hbaseConf = null;
+  private Connection conn;
+  private EntityTable entityTable;
+  private AppToFlowTable appToFlowTable;
+
+  public HBaseTimelineReaderImpl() {
+    super(HBaseTimelineReaderImpl.class.getName());
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    hbaseConf = HBaseConfiguration.create(conf);
+    conn = ConnectionFactory.createConnection(hbaseConf);
+    entityTable = new EntityTable();
+    appToFlowTable = new AppToFlowTable();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (conn != null) {
+      LOG.info("closing the hbase Connection");
+      conn.close();
+    }
+    super.serviceStop();
+  }
+
+  @Override
+  public TimelineEntity getEntity(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      String entityId, EnumSet<Field> fieldsToRetrieve)
+      throws IOException {
+    validateParams(userId, clusterId, appId, entityType, entityId, true);
+    // In reality both should be null or neither should be null
+    if (flowId == null || flowRunId == null) {
+      FlowContext context = lookupFlowContext(clusterId, appId);
+      flowId = context.flowId;
+      flowRunId = context.flowRunId;
+    }
+    if (fieldsToRetrieve == null) {
+      fieldsToRetrieve = EnumSet.noneOf(Field.class);
+    }
+
+    byte[] rowKey = EntityRowKey.getRowKey(
+        clusterId, userId, flowId, flowRunId, appId, entityType, entityId);
+    Get get = new Get(rowKey);
+    get.setMaxVersions(Integer.MAX_VALUE);
+    return parseEntity(
+        entityTable.getResult(hbaseConf, conn, get), fieldsToRetrieve,
+        false, DEFAULT_BEGIN_TIME, DEFAULT_END_TIME, false, DEFAULT_BEGIN_TIME,
+        DEFAULT_END_TIME, null, null, null, null, null, null);
+  }
+
+  @Override
+  public Set<TimelineEntity> getEntities(String userId, String clusterId,
+      String flowId, Long flowRunId, String appId, String entityType,
+      Long limit, Long createdTimeBegin, Long createdTimeEnd,
+      Long modifiedTimeBegin, Long modifiedTimeEnd,
+      Map<String, Set<String>> relatesTo, Map<String, Set<String>> isRelatedTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> metricFilters, Set<String> eventFilters,
+      EnumSet<Field> fieldsToRetrieve) throws IOException {
+    validateParams(userId, clusterId, appId, entityType, null, false);
+    // In reality both should be null or neither should be null
+    if (flowId == null || flowRunId == null) {
+      FlowContext context = lookupFlowContext(clusterId, appId);
+      flowId = context.flowId;
+      flowRunId = context.flowRunId;
+    }
+    if (limit == null) {
+      limit = TimelineReader.DEFAULT_LIMIT;
+    }
+    if (createdTimeBegin == null) {
+      createdTimeBegin = DEFAULT_BEGIN_TIME;
+    }
+    if (createdTimeEnd == null) {
+      createdTimeEnd = DEFAULT_END_TIME;
+    }
+    if (modifiedTimeBegin == null) {
+      modifiedTimeBegin = DEFAULT_BEGIN_TIME;
+    }
+    if (modifiedTimeEnd == null) {
+      modifiedTimeEnd = DEFAULT_END_TIME;
+    }
+    if (fieldsToRetrieve == null) {
+      fieldsToRetrieve = EnumSet.noneOf(Field.class);
+    }
+
+    NavigableSet<TimelineEntity> entities = new TreeSet<>();
+    // Scan through part of the table to find the entities belong to one app and
+    // one type
+    Scan scan = new Scan();
+    scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
+        clusterId, userId, flowId, flowRunId, appId, entityType));
+    scan.setMaxVersions(Integer.MAX_VALUE);
+    ResultScanner scanner = entityTable.getResultScanner(hbaseConf, conn, scan);
+    for (Result result : scanner) {
+      TimelineEntity entity = parseEntity(result, fieldsToRetrieve,
+          true, createdTimeBegin, createdTimeEnd,
+          true, modifiedTimeBegin, modifiedTimeEnd,
+          isRelatedTo, relatesTo, infoFilters, configFilters, eventFilters,
+          metricFilters);
+      if (entity == null) {
+        continue;
+      }
+      if (entities.size() > limit) {
+        entities.pollLast();
+      }
+      entities.add(entity);
+    }
+    return entities;
+  }
+
+  private FlowContext lookupFlowContext(String clusterId, String appId)
+      throws IOException {
+    byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
+    Get get = new Get(rowKey);
+    Result result = appToFlowTable.getResult(hbaseConf, conn, get);
+    if (result != null && !result.isEmpty()) {
+      return new FlowContext(
+          AppToFlowColumn.FLOW_ID.readResult(result).toString(),
+          ((Number) AppToFlowColumn.FLOW_RUN_ID.readResult(result)).longValue());
+    } else {
+       throw new IOException(
+           "Unable to find the context flow ID and flow run ID for clusterId=" +
+           clusterId + ", appId=" + appId);
+    }
+  }
+
+  private static class FlowContext {
+    private String flowId;
+    private Long flowRunId;
+    public FlowContext(String flowId, Long flowRunId) {
+      this.flowId = flowId;
+      this.flowRunId = flowRunId;
+    }
+  }
+
+  private static void validateParams(String userId, String clusterId,
+      String appId, String entityType, String entityId, boolean checkEntityId) {
+    Preconditions.checkNotNull(userId, "userId shouldn't be null");
+    Preconditions.checkNotNull(clusterId, "clusterId shouldn't be null");
+    Preconditions.checkNotNull(appId, "appId shouldn't be null");
+    Preconditions.checkNotNull(entityType, "entityType shouldn't be null");
+    if (checkEntityId) {
+      Preconditions.checkNotNull(entityId, "entityId shouldn't be null");
+    }
+  }
+
+  private static TimelineEntity parseEntity(
+      Result result, EnumSet<Field> fieldsToRetrieve,
+      boolean checkCreatedTime, long createdTimeBegin, long createdTimeEnd,
+      boolean checkModifiedTime, long modifiedTimeBegin, long modifiedTimeEnd,
+      Map<String, Set<String>> isRelatedTo, Map<String, Set<String>> relatesTo,
+      Map<String, Object> infoFilters, Map<String, String> configFilters,
+      Set<String> eventFilters, Set<String> metricFilters)
+          throws IOException {
+    if (result == null || result.isEmpty()) {
+      return null;
+    }
+    TimelineEntity entity = new TimelineEntity();
+    entity.setType(EntityColumn.TYPE.readResult(result).toString());
+    entity.setId(EntityColumn.ID.readResult(result).toString());
+
+    // fetch created time
+    entity.setCreatedTime(
+        ((Number) EntityColumn.CREATED_TIME.readResult(result)).longValue());
+    if (checkCreatedTime && (entity.getCreatedTime() < createdTimeBegin ||
+        entity.getCreatedTime() > createdTimeEnd)) {
+      return null;
+    }
+
+    // fetch modified time
+    entity.setCreatedTime(
+        ((Number) EntityColumn.MODIFIED_TIME.readResult(result)).longValue());
+    if (checkModifiedTime && (entity.getModifiedTime() < modifiedTimeBegin ||
+        entity.getModifiedTime() > modifiedTimeEnd)) {
+      return null;
+    }
+
+    // fetch is related to entities
+    boolean checkIsRelatedTo = isRelatedTo != null && isRelatedTo.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.IS_RELATED_TO) || checkIsRelatedTo) {
+      readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO);
+      if (checkIsRelatedTo && !TimelineReaderUtils.matchRelations(
+          entity.getIsRelatedToEntities(), isRelatedTo)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.IS_RELATED_TO)) {
+        entity.getIsRelatedToEntities().clear();
+      }
+    }
+
+    // fetch relates to entities
+    boolean checkRelatesTo = relatesTo != null && relatesTo.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.RELATES_TO) || checkRelatesTo) {
+      readRelationship(entity, result, EntityColumnPrefix.RELATES_TO);
+      if (checkRelatesTo && !TimelineReaderUtils.matchRelations(
+          entity.getRelatesToEntities(), relatesTo)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.RELATES_TO)) {
+        entity.getRelatesToEntities().clear();
+      }
+    }
+
+    // fetch info
+    boolean checkInfo = infoFilters != null && infoFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
+      readKeyValuePairs(entity, result, EntityColumnPrefix.INFO);
+      if (checkInfo &&
+          !TimelineReaderUtils.matchFilters(entity.getInfo(), infoFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.INFO)) {
+        entity.getInfo().clear();
+      }
+    }
+
+    // fetch configs
+    boolean checkConfigs = configFilters != null && configFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
+      readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG);
+      if (checkConfigs && !TimelineReaderUtils.matchFilters(
+          entity.getConfigs(), configFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.CONFIGS)) {
+        entity.getConfigs().clear();
+      }
+    }
+
+    // fetch events
+    boolean checkEvents = eventFilters != null && eventFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
+      readEvents(entity, result);
+      if (checkEvents && !TimelineReaderUtils.matchEventFilters(
+          entity.getEvents(), eventFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.EVENTS)) {
+        entity.getEvents().clear();
+      }
+    }
+
+    // fetch metrics
+    boolean checkMetrics = metricFilters != null && metricFilters.size() > 0;
+    if (fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
+      readMetrics(entity, result);
+      if (checkMetrics && !TimelineReaderUtils.matchMetricFilters(
+          entity.getMetrics(), metricFilters)) {
+        return null;
+      }
+      if (!fieldsToRetrieve.contains(Field.ALL) &&
+          !fieldsToRetrieve.contains(Field.METRICS)) {
+        entity.getMetrics().clear();
+      }
+    }
+    return entity;
+  }
+
+  private static void readRelationship(
+      TimelineEntity entity, Result result, EntityColumnPrefix prefix)
+          throws IOException {
+    // isRelatedTo and relatesTo are of type Map<String, Set<String>>
+    Map<String, Object> columns = prefix.readResults(result);
+    for (Map.Entry<String, Object> column : columns.entrySet()) {
+      for (String id : Separator.VALUES.splitEncoded(
+          column.getValue().toString())) {
+        if (prefix.equals(EntityColumnPrefix.IS_RELATED_TO)) {
+          entity.addIsRelatedToEntity(column.getKey(), id);
+        } else {
+          entity.addRelatesToEntity(column.getKey(), id);
+        }
+      }
+    }
+  }
+
+  private static void readKeyValuePairs(
+      TimelineEntity entity, Result result, EntityColumnPrefix prefix)
+          throws IOException {
+    // info and configuration are of type Map<String, Object or String>
+    Map<String, Object> columns = prefix.readResults(result);
+    if (prefix.equals(EntityColumnPrefix.CONFIG)) {
+      for (Map.Entry<String, Object> column : columns.entrySet()) {
+        entity.addConfig(column.getKey(), column.getKey().toString());
+      }
+    } else {
+      entity.addInfo(columns);
+    }
+  }
+
+  private static void readEvents(TimelineEntity entity, Result result)
+      throws IOException {
+    Map<String, TimelineEvent> eventsMap = new HashMap<>();
+    Map<String, Object> eventsResult =
+        EntityColumnPrefix.EVENT.readResults(result);
+    for (Map.Entry<String,Object> eventResult : eventsResult.entrySet()) {
+      Collection<String> tokens =
+          Separator.VALUES.splitEncoded(eventResult.getKey());
+      if (tokens.size() != 2 && tokens.size() != 3) {
+        throw new IOException(
+            "Invalid event column name: " + eventResult.getKey());
+      }
+      Iterator<String> idItr = tokens.iterator();
+      String id = idItr.next();
+      String tsStr = idItr.next();
+      // TODO: timestamp is not correct via ser/des through UTF-8 string
+      Long ts =
+          TimelineWriterUtils.invert(Bytes.toLong(tsStr.getBytes(
+              StandardCharsets.UTF_8)));
+      String key = Separator.VALUES.joinEncoded(id, ts.toString());
+      TimelineEvent event = eventsMap.get(key);
+      if (event == null) {
+        event = new TimelineEvent();
+        event.setId(id);
+        event.setTimestamp(ts);
+        eventsMap.put(key, event);
+      }
+      if (tokens.size() == 3) {
+        String infoKey = idItr.next();
+        event.addInfo(infoKey, eventResult.getValue());
+      }
+    }
+    Set<TimelineEvent> eventsSet = new HashSet<>(eventsMap.values());
+    entity.addEvents(eventsSet);
+  }
+
+  private static void readMetrics(TimelineEntity entity, Result result)
+      throws IOException {
+    NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
+        EntityColumnPrefix.METRIC.readResultsWithTimestamps(result);
+    for (Map.Entry<String, NavigableMap<Long, Number>> metricResult:
+        metricsResult.entrySet()) {
+      TimelineMetric metric = new TimelineMetric();
+      metric.setId(metricResult.getKey());
+      // Simply assume that if the value set contains more than 1 elements, the
+      // metric is a TIME_SERIES metric, otherwise, it's a SINGLE_VALUE metric
+      metric.setType(metricResult.getValue().size() > 1 ?
+          TimelineMetric.Type.TIME_SERIES : TimelineMetric.Type.SINGLE_VALUE);
+      metric.addValues(metricResult.getValue());
+      entity.addMetric(metric);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/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 3173e87..5290415 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
@@ -33,9 +33,14 @@ import org.apache.hadoop.hbase.util.Bytes;
 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.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.TimelineWriteResponse;
+import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowColumn;
+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.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
@@ -55,6 +60,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
 
   private Connection conn;
   private TypedBufferedMutator<EntityTable> entityTable;
+  private TypedBufferedMutator<AppToFlowTable> appToFlowTable;
 
   private static final Log LOG = LogFactory
       .getLog(HBaseTimelineWriterImpl.class);
@@ -77,6 +83,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     Configuration hbaseConf = HBaseConfiguration.create(conf);
     conn = ConnectionFactory.createConnection(hbaseConf);
     entityTable = new EntityTable().getTableMutator(hbaseConf, conn);
+    appToFlowTable = new AppToFlowTable().getTableMutator(hbaseConf, conn);
   }
 
   /**
@@ -97,7 +104,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
 
       byte[] rowKey =
           EntityRowKey.getRowKey(clusterId, userId, flowName, flowRunId, appId,
-              te);
+              te.getType(), te.getId());
 
       storeInfo(rowKey, te, flowVersion);
       storeEvents(rowKey, te.getEvents());
@@ -107,11 +114,37 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
           EntityColumnPrefix.IS_RELATED_TO);
       storeRelations(rowKey, te.getRelatesToEntities(),
           EntityColumnPrefix.RELATES_TO);
-    }
 
+      if (isApplicationCreated(te)) {
+        onApplicationCreated(
+            clusterId, userId, flowName, flowVersion, flowRunId, appId, te);
+      }
+    }
     return putStatus;
   }
 
+  private static boolean isApplicationCreated(TimelineEntity te) {
+    if (te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString())) {
+      boolean isAppCreated = false;
+      for (TimelineEvent event : te.getEvents()) {
+        if (event.getId().equals(
+            ApplicationMetricsConstants.CREATED_EVENT_TYPE)) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  private void onApplicationCreated(String clusterId, String userId,
+      String flowName, String flowVersion, long flowRunId, String appId,
+      TimelineEntity te) throws IOException {
+    byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
+    AppToFlowColumn.FLOW_ID.store(rowKey, appToFlowTable, null, flowName);
+    AppToFlowColumn.FLOW_RUN_ID.store(
+        rowKey, appToFlowTable, null, flowRunId);
+  }
+
   /**
    * Stores the Relations from the {@linkplain TimelineEntity} object
    */
@@ -245,6 +278,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   public void flush() throws IOException {
     // flush all buffered mutators
     entityTable.flush();
+    appToFlowTable.flush();
   }
 
   /**
@@ -258,6 +292,11 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       // The close API performs flushing and releases any resources held
       entityTable.close();
     }
+    if (appToFlowTable != null) {
+      LOG.info("closing app_flow table");
+      // The close API performs flushing and releases any resources held
+      appToFlowTable.close();
+    }
     if (conn != null) {
       LOG.info("closing the hbase Connection");
       conn.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
index a5cc2ab..2c3897d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
@@ -37,6 +37,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.server.timelineservice.storage.apptoflow.AppToFlowTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
 
 /**
@@ -70,6 +71,11 @@ public class TimelineSchemaCreator {
       int metricsTTL = Integer.parseInt(entityTableTTLMetrics);
       new EntityTable().setMetricsTTL(metricsTTL, hbaseConf);
     }
+    // Grab the appToflowTableName argument
+    String appToflowTableName = commandLine.getOptionValue("a2f");
+    if (StringUtils.isNotBlank(appToflowTableName)) {
+      hbaseConf.set(AppToFlowTable.TABLE_NAME_CONF_NAME, appToflowTableName);
+    }
     createAllTables(hbaseConf);
   }
 
@@ -95,6 +101,11 @@ public class TimelineSchemaCreator {
     o.setRequired(false);
     options.addOption(o);
 
+    o = new Option("a2f", "appToflowTableName", true, "app to flow table name");
+    o.setArgName("appToflowTableName");
+    o.setRequired(false);
+    options.addOption(o);
+
     CommandLineParser parser = new PosixParser();
     CommandLine commandLine = null;
     try {
@@ -120,6 +131,7 @@ public class TimelineSchemaCreator {
         throw new IOException("Cannot create table since admin is null");
       }
       new EntityTable().createTable(admin, hbaseConf);
+      new AppToFlowTable().createTable(admin, hbaseConf);
     } finally {
       if (conn != null) {
         conn.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/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
new file mode 100644
index 0000000..423037a
--- /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/AppToFlowColumn.java
@@ -0,0 +1,126 @@
+/**
+ * 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 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");
+
+  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;
+  }
+
+  public void store(byte[] rowKey,
+      TypedBufferedMutator<AppToFlowTable> tableMutator, Long timestamp,
+      Object inputValue) throws IOException {
+    column.store(rowKey, tableMutator, columnQualifierBytes, timestamp,
+        inputValue);
+  }
+
+  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/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowColumnFamily.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/AppToFlowColumnFamily.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/AppToFlowColumnFamily.java
new file mode 100644
index 0000000..e74235f
--- /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/AppToFlowColumnFamily.java
@@ -0,0 +1,51 @@
+/**
+ * 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.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+
+/**
+ * Represents the app_flow table column families.
+ */
+public enum AppToFlowColumnFamily implements ColumnFamily<AppToFlowTable> {
+  /**
+   * Mapping column family houses known columns such as flowId and flowRunId
+   */
+  MAPPING("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.
+   */
+  AppToFlowColumnFamily(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/0bed3fb3/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
new file mode 100644
index 0000000..ad4fec6
--- /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/AppToFlowRowKey.java
@@ -0,0 +1,39 @@
+/**
+ * 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.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+
+/**
+ * Represents a rowkey for the app_flow table.
+ */
+public class AppToFlowRowKey {
+  /**
+   * Constructs a row key prefix for the app_flow table as follows:
+   * {@code clusterId!AppId}
+   *
+   * @param clusterId
+   * @param appId
+   * @return byte array with the row key
+   */
+  public static byte[] getRowKey(String clusterId, String appId) {
+    return Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId, appId));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/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
new file mode 100644
index 0000000..2467856
--- /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/AppToFlowTable.java
@@ -0,0 +1,110 @@
+/**
+ * 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.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;
+
+import java.io.IOException;
+
+/**
+ * The app_flow table as column families mapping. Mapping stores
+ * appId to flowId and flowRunId mapping information
+ *
+ * Example app_flow table record:
+ *
+ * <pre>
+ * |--------------------------------------|
+ * |  Row       | Column Family           |
+ * |  key       | info                    |
+ * |--------------------------------------|
+ * | clusterId! | flowId:                 |
+ * | AppId      | foo@daily_hive_report   |
+ * |            |                         |
+ * |            | flowRunId:              |
+ * |            | 1452828720457           |
+ * |            |                         |
+ * |            |                         |
+ * |            |                         |
+ * |--------------------------------------|
+ * </pre>
+ */
+public class AppToFlowTable extends BaseTable<AppToFlowTable> {
+  /** app_flow prefix */
+  private static final String PREFIX =
+      YarnConfiguration.TIMELINE_SERVICE_PREFIX + "app-flow";
+
+  /** config param name that specifies the app_flow table name */
+  public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name";
+
+  /** default value for app_flow table name */
+  private static final String DEFAULT_TABLE_NAME = "timelineservice.app_flow";
+
+  private static final Log LOG = LogFactory.getLog(AppToFlowTable.class);
+
+  public AppToFlowTable() {
+    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 appToFlowTableDescp = new HTableDescriptor(table);
+    HColumnDescriptor mappCF =
+        new HColumnDescriptor(AppToFlowColumnFamily.MAPPING.getBytes());
+    mappCF.setBloomFilterType(BloomType.ROWCOL);
+    appToFlowTableDescp.addFamily(mappCF);
+
+    appToFlowTableDescp
+        .setRegionSplitPolicyClassName(
+            "org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy");
+    appToFlowTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length",
+        TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH);
+    admin.createTable(appToFlowTableDescp,
+        TimelineHBaseSchemaConstants.getUsernameSplits());
+    LOG.info("Status of table creation for " + table.getNameAsString() + "="
+        + admin.tableExists(table));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/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/storage/apptoflow/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/storage/apptoflow/package-info.java
new file mode 100644
index 0000000..df7ffc1
--- /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/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/BaseTable.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/BaseTable.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/BaseTable.java
index e8d8b5c..abba79a 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/BaseTable.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/BaseTable.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.BufferedMutator;
 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.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
@@ -94,6 +96,20 @@ public abstract class BaseTable<T> {
   }
 
   /**
+   *
+   * @param hbaseConf used to read settings that override defaults
+   * @param conn used to create table from
+   * @param get that specifies what single row you want to get from this table
+   * @return result of get operation
+   * @throws IOException
+   */
+  public Result getResult(Configuration hbaseConf, Connection conn, Get get)
+      throws IOException {
+    Table table = conn.getTable(getTableName(hbaseConf));
+    return table.get(get);
+  }
+
+  /**
    * Get the table name for this table.
    *
    * @param hbaseConf

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ColumnPrefix.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/ColumnPrefix.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/ColumnPrefix.java
index 671c824..509ff49 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/ColumnPrefix.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/ColumnPrefix.java
@@ -64,7 +64,7 @@ public interface ColumnPrefix<T> {
   public Object readResult(Result result, String qualifier) throws IOException;
 
   /**
-   * @param resultfrom which to read columns
+   * @param result from which to read columns
    * @return the latest values of columns in the column family with this prefix
    *         (or all of them if the prefix value is null).
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntitySchemaConstants.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/TimelineEntitySchemaConstants.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/TimelineEntitySchemaConstants.java
deleted file mode 100644
index 5518a27..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntitySchemaConstants.java
+++ /dev/null
@@ -1,68 +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.common;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * contains the constants used in the context of schema accesses for
- * {@link org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity}
- * information
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TimelineEntitySchemaConstants {
-
-  /**
-   * Used to create a pre-split for tables starting with a username in the
-   * prefix. TODO: this may have to become a config variable (string with
-   * separators) so that different installations can presplit based on their own
-   * commonly occurring names.
-   */
-  private final static byte[][] USERNAME_SPLITS = { Bytes.toBytes("a"),
-      Bytes.toBytes("ad"), Bytes.toBytes("an"), Bytes.toBytes("b"),
-      Bytes.toBytes("ca"), Bytes.toBytes("cl"), Bytes.toBytes("d"),
-      Bytes.toBytes("e"), Bytes.toBytes("f"), Bytes.toBytes("g"),
-      Bytes.toBytes("h"), Bytes.toBytes("i"), Bytes.toBytes("j"),
-      Bytes.toBytes("k"), Bytes.toBytes("l"), Bytes.toBytes("m"),
-      Bytes.toBytes("n"), Bytes.toBytes("o"), Bytes.toBytes("q"),
-      Bytes.toBytes("r"), Bytes.toBytes("s"), Bytes.toBytes("se"),
-      Bytes.toBytes("t"), Bytes.toBytes("u"), Bytes.toBytes("v"),
-      Bytes.toBytes("w"), Bytes.toBytes("x"), Bytes.toBytes("y"),
-      Bytes.toBytes("z") };
-
-  /**
-   * The length at which keys auto-split
-   */
-  public static final String USERNAME_SPLIT_KEY_PREFIX_LENGTH = "4";
-
-  /**
-   * @return splits for splits where a user is a prefix.
-   */
-  public final static byte[][] getUsernameSplits() {
-    byte[][] kloon = USERNAME_SPLITS.clone();
-    // Deep copy.
-    for (int row = 0; row < USERNAME_SPLITS.length; row++) {
-      kloon[row] = Bytes.copy(USERNAME_SPLITS[row]);
-    }
-    return kloon;
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineHBaseSchemaConstants.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/TimelineHBaseSchemaConstants.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/TimelineHBaseSchemaConstants.java
new file mode 100644
index 0000000..bbf498a
--- /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/common/TimelineHBaseSchemaConstants.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.common;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * contains the constants used in the context of schema accesses for
+ * {@link org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity}
+ * information
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class TimelineHBaseSchemaConstants {
+
+  /**
+   * Used to create a pre-split for tables starting with a username in the
+   * prefix. TODO: this may have to become a config variable (string with
+   * separators) so that different installations can presplit based on their own
+   * commonly occurring names.
+   */
+  private final static byte[][] USERNAME_SPLITS = { Bytes.toBytes("a"),
+      Bytes.toBytes("ad"), Bytes.toBytes("an"), Bytes.toBytes("b"),
+      Bytes.toBytes("ca"), Bytes.toBytes("cl"), Bytes.toBytes("d"),
+      Bytes.toBytes("e"), Bytes.toBytes("f"), Bytes.toBytes("g"),
+      Bytes.toBytes("h"), Bytes.toBytes("i"), Bytes.toBytes("j"),
+      Bytes.toBytes("k"), Bytes.toBytes("l"), Bytes.toBytes("m"),
+      Bytes.toBytes("n"), Bytes.toBytes("o"), Bytes.toBytes("q"),
+      Bytes.toBytes("r"), Bytes.toBytes("s"), Bytes.toBytes("se"),
+      Bytes.toBytes("t"), Bytes.toBytes("u"), Bytes.toBytes("v"),
+      Bytes.toBytes("w"), Bytes.toBytes("x"), Bytes.toBytes("y"),
+      Bytes.toBytes("z") };
+
+  /**
+   * The length at which keys auto-split
+   */
+  public static final String USERNAME_SPLIT_KEY_PREFIX_LENGTH = "4";
+
+  /**
+   * @return splits for splits where a user is a prefix.
+   */
+  public final static byte[][] getUsernameSplits() {
+    byte[][] kloon = USERNAME_SPLITS.clone();
+    // Deep copy.
+    for (int row = 0; row < USERNAME_SPLITS.length; row++) {
+      kloon[row] = Bytes.copy(USERNAME_SPLITS[row]);
+    }
+    return kloon;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/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/storage/common/TimelineReaderUtils.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/TimelineReaderUtils.java
new file mode 100644
index 0000000..91d7ba4
--- /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/common/TimelineReaderUtils.java
@@ -0,0 +1,112 @@
+/**
+ * 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.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+
+public class TimelineReaderUtils {
+  /**
+   *
+   * @param entityRelations the relations of an entity
+   * @param relationFilters the relations for filtering
+   * @return a boolean flag to indicate if both match
+   */
+  public static boolean matchRelations(
+      Map<String, Set<String>> entityRelations,
+      Map<String, Set<String>> relationFilters) {
+    for (Map.Entry<String, Set<String>> relation : relationFilters.entrySet()) {
+      Set<String> ids = entityRelations.get(relation.getKey());
+      if (ids == null) {
+        return false;
+      }
+      for (String id : relation.getValue()) {
+        if (!ids.contains(id)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  /**
+   *
+   * @param map the map of key/value pairs in an entity
+   * @param filters the map of key/value pairs for filtering
+   * @return a boolean flag to indicate if both match
+   */
+  public static boolean matchFilters(Map<String, ? extends Object> map,
+      Map<String, ? extends Object> filters) {
+    for (Map.Entry<String, ? extends Object> filter : filters.entrySet()) {
+      Object value = map.get(filter.getKey());
+      if (value == null) {
+        return false;
+      }
+      if (!value.equals(filter.getValue())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   *
+   * @param entityEvents the set of event objects in an entity
+   * @param eventFilters the set of event Ids for filtering
+   * @return a boolean flag to indicate if both match
+   */
+  public static boolean matchEventFilters(Set<TimelineEvent> entityEvents,
+      Set<String> eventFilters) {
+    Set<String> eventIds = new HashSet<String>();
+    for (TimelineEvent event : entityEvents) {
+      eventIds.add(event.getId());
+    }
+    for (String eventFilter : eventFilters) {
+      if (!eventIds.contains(eventFilter)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   *
+   * @param metrics the set of metric objects in an entity
+   * @param metricFilters the set of metric Ids for filtering
+   * @return a boolean flag to indicate if both match
+   */
+  public static boolean matchMetricFilters(Set<TimelineMetric> metrics,
+      Set<String> metricFilters) {
+    Set<String> metricIds = new HashSet<String>();
+    for (TimelineMetric metric : metrics) {
+      metricIds.add(metric.getId());
+    }
+
+    for (String metricFilter : metricFilters) {
+      if (!metricIds.contains(metricFilter)) {
+        return false;
+      }
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java
index 90da966..26e7748 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java
@@ -62,7 +62,7 @@ public enum EntityColumn implements Column<EntityTable> {
   private final String columnQualifier;
   private final byte[] columnQualifierBytes;
 
-  private EntityColumn(ColumnFamily<EntityTable> columnFamily,
+  EntityColumn(ColumnFamily<EntityTable> columnFamily,
       String columnQualifier) {
     this.columnFamily = columnFamily;
     this.columnQualifier = columnQualifier;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.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/entity/EntityColumnFamily.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java
index 8a95d12..7c63727 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java
@@ -53,7 +53,7 @@ public enum EntityColumnFamily implements ColumnFamily<EntityTable> {
    * @param value create a column family with this name. Must be lower case and
    *          without spaces.
    */
-  private EntityColumnFamily(String value) {
+  EntityColumnFamily(String value) {
     // column families should be lower case and not contain any spaces.
     this.bytes = Bytes.toBytes(Separator.SPACE.encode(value));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
index 8b7bc3e..58272ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java
@@ -80,7 +80,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    * @param columnFamily that this column is stored in.
    * @param columnPrefix for this column.
    */
-  private EntityColumnPrefix(ColumnFamily<EntityTable> columnFamily,
+  EntityColumnPrefix(ColumnFamily<EntityTable> columnFamily,
       String columnPrefix) {
     column = new ColumnHelper<EntityTable>(columnFamily);
     this.columnFamily = columnFamily;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java
index 3e17ad0..9a72be0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java
@@ -55,17 +55,45 @@ public class EntityRowKey {
 
   /**
    * Constructs a row key prefix for the entity table as follows:
-   * {@code userName!clusterId!flowId!flowRunId!AppId}
+   * {@code userName!clusterId!flowId!flowRunId!AppId!entityType!}
    *
    * @param clusterId
    * @param userId
    * @param flowId
    * @param flowRunId
    * @param appId
+   * @param entityType
    * @return byte array with the row key prefix
    */
+  public static byte[] getRowKeyPrefix(String clusterId, String userId,
+      String flowId, Long flowRunId, String appId, String entityType) {
+    byte[] first =
+        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(userId, clusterId,
+            flowId));
+    // Note that flowRunId is a long, so we can't encode them all at the same
+    // time.
+    byte[] second = Bytes.toBytes(TimelineWriterUtils.invert(flowRunId));
+    byte[] third =
+        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(appId, entityType, ""));
+    return Separator.QUALIFIERS.join(first, second, third);
+  }
+
+  /**
+   * Constructs a row key for the entity table as follows:
+   * {@code userName!clusterId!flowId!flowRunId!AppId!entityType!entityId}
+   *
+   * @param clusterId
+   * @param userId
+   * @param flowId
+   * @param flowRunId
+   * @param appId
+   * @param entityType
+   * @param entityId
+   * @return byte array with the row key
+   */
   public static byte[] getRowKey(String clusterId, String userId,
-      String flowId, Long flowRunId, String appId, TimelineEntity te) {
+      String flowId, Long flowRunId, String appId, String entityType,
+      String entityId) {
     byte[] first =
         Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(userId, clusterId,
             flowId));
@@ -73,8 +101,8 @@ public class EntityRowKey {
     // time.
     byte[] second = Bytes.toBytes(TimelineWriterUtils.invert(flowRunId));
     byte[] third =
-        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(appId, te.getType(),
-            te.getId()));
+        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(appId, entityType,
+            entityId));
     return Separator.QUALIFIERS.join(first, second, third);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.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/entity/EntityTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
index 2ae7d39..f657a14 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java
@@ -29,7 +29,7 @@ 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.TimelineEntitySchemaConstants;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants;
 
 /**
  * The entity table as column families info, config and metrics. Info stores
@@ -75,7 +75,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineEnti
 public class EntityTable extends BaseTable<EntityTable> {
   /** entity prefix */
   private static final String PREFIX =
-      YarnConfiguration.TIMELINE_SERVICE_PREFIX + ".entity";
+      YarnConfiguration.TIMELINE_SERVICE_PREFIX + "entity";
 
   /** config param name that specifies the entity table name */
   public static final String TABLE_NAME_CONF_NAME = PREFIX + ".table.name";
@@ -146,9 +146,9 @@ public class EntityTable extends BaseTable<EntityTable> {
     entityTableDescp
         .setRegionSplitPolicyClassName("org.apache.hadoop.hbase.regionserver.KeyPrefixRegionSplitPolicy");
     entityTableDescp.setValue("KeyPrefixRegionSplitPolicy.prefix_length",
-        TimelineEntitySchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH);
+        TimelineHBaseSchemaConstants.USERNAME_SPLIT_KEY_PREFIX_LENGTH);
     admin.createTable(entityTableDescp,
-        TimelineEntitySchemaConstants.getUsernameSplits());
+        TimelineHBaseSchemaConstants.getUsernameSplits());
     LOG.info("Status of table creation for " + table.getNameAsString() + "="
         + admin.tableExists(table));
   }


[15/50] [abbrv] hadoop git commit: YARN-4058. Miscellaneous issues in NodeManager project (Naganarasimha G R via sjlee)

Posted by gt...@apache.org.
YARN-4058. Miscellaneous issues in NodeManager project (Naganarasimha G R via sjlee)


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

Branch: refs/heads/feature-YARN-2928
Commit: c59a6ac60d24481c7b93a3210b7c7d645afb8ad1
Parents: 01f5e18
Author: Sangjin Lee <sj...@apache.org>
Authored: Mon Aug 24 17:36:31 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:57 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../containermanager/ContainerManagerImpl.java  | 33 +++++++++++---------
 .../metrics/TimelineServiceV2Publisher.java     |  2 --
 .../TestSystemMetricsPublisherForV2.java        |  8 ++++-
 4 files changed, 28 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59a6ac6/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f8775fc..9c1ad80 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -144,6 +144,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
 
     YARN-4064. build is broken at TestHBaseTimelineWriterImpl.java (sjlee)
 
+    YARN-4058. Miscellaneous issues in NodeManager project (Naganarasimha G R
+    via sjlee)
+
 Trunk - Unreleased
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59a6ac6/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 12aab79..b010eee 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
@@ -951,21 +951,24 @@ public class ContainerManagerImpl extends CompositeService implements
         if (flowRunIdStr != null && !flowRunIdStr.isEmpty()) {
           flowRunId = Long.parseLong(flowRunIdStr);
         }
-        Application application = new ApplicationImpl(dispatcher, user,
-            flowName, flowVersion, flowRunId, applicationID, credentials, context);
-        if (null == context.getApplications().putIfAbsent(applicationID,
-          application)) {
-          LOG.info("Creating a new application reference for app " + applicationID);
-          LogAggregationContext logAggregationContext =
-              containerTokenIdentifier.getLogAggregationContext();
-          Map<ApplicationAccessType, String> appAcls =
-              container.getLaunchContext().getApplicationACLs();
-          context.getNMStateStore().storeApplication(applicationID,
-              buildAppProto(applicationID, user, credentials, appAcls,
-                logAggregationContext));
-          dispatcher.getEventHandler().handle(
-            new ApplicationInitEvent(applicationID, appAcls,
-              logAggregationContext));
+        if (!context.getApplications().containsKey(applicationID)) {
+          Application application =
+              new ApplicationImpl(dispatcher, user, flowName, flowVersion,
+                  flowRunId, applicationID, credentials, context);
+          if (context.getApplications().putIfAbsent(applicationID,
+              application) == null) {
+            LOG.info("Creating a new application reference for app "
+                + applicationID);
+            LogAggregationContext logAggregationContext =
+                containerTokenIdentifier.getLogAggregationContext();
+            Map<ApplicationAccessType, String> appAcls =
+                container.getLaunchContext().getApplicationACLs();
+            context.getNMStateStore().storeApplication(applicationID,
+                buildAppProto(applicationID, user, credentials, appAcls,
+                    logAggregationContext));
+            dispatcher.getEventHandler().handle(new ApplicationInitEvent(
+                applicationID, appAcls, logAggregationContext));
+          }
         }
 
         dispatcher.getEventHandler().handle(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59a6ac6/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 5d4d5c8..3a6c678 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
@@ -39,8 +39,6 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.Identif
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
 import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c59a6ac6/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 9830a80..ac20335 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
@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -45,7 +46,6 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -332,6 +332,12 @@ public class TestSystemMetricsPublisherForV2 {
     when(app.getRMAppMetrics()).thenReturn(
         new RMAppMetrics(null, 0, 0, Integer.MAX_VALUE, Long.MAX_VALUE));
     when(app.getApplicationTags()).thenReturn(Collections.<String> emptySet());
+    ApplicationSubmissionContext appSubmissionContext =
+        mock(ApplicationSubmissionContext.class);
+    when(appSubmissionContext.getPriority())
+        .thenReturn(Priority.newInstance(0));
+    when(app.getApplicationSubmissionContext())
+        .thenReturn(appSubmissionContext);
     return app;
   }
 


[07/50] [abbrv] hadoop git commit: MAPREDUCE-6337. Added a mode to replay MR job history files and put them into the timeline service v2. Contributed by Sangjin Lee.

Posted by gt...@apache.org.
MAPREDUCE-6337. Added a mode to replay MR job history files and put them into the timeline service v2. Contributed by Sangjin Lee.

(cherry picked from commit 463e070a8e7c882706a96eaa20ea49bfe9982875)


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

Branch: refs/heads/feature-YARN-2928
Commit: a7ac8b2174454fc8656891c7820d73df50559d4d
Parents: 3007622
Author: Zhijie Shen <zj...@apache.org>
Authored: Thu May 14 15:16:33 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:37:47 2016 -0800

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |   3 +
 .../hadoop/mapred/JobHistoryFileParser.java     |  53 ++++
 .../mapred/JobHistoryFileReplayMapper.java      | 301 +++++++++++++++++++
 .../hadoop/mapred/SimpleEntityWriter.java       | 139 +++++++++
 .../hadoop/mapred/TimelineEntityConverter.java  | 207 +++++++++++++
 .../mapred/TimelineServicePerformanceV2.java    | 191 ++++--------
 .../collector/TimelineCollectorManager.java     |   8 +-
 .../storage/FileSystemTimelineWriterImpl.java   |  23 +-
 .../timelineservice/storage/package-info.java   |  24 ++
 9 files changed, 809 insertions(+), 140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7ac8b21/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index c9c1f6b..b5b6104 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -12,6 +12,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     MAPREDUCE-6335. Created MR job based performance test driver for the
     timeline service v2. (Sangjin Lee via zjshen)
 
+    MAPREDUCE-6337. Added a mode to replay MR job history files and put them
+    into the timeline service v2. (Sangjin Lee via zjshen)
+
   IMPROVEMENTS
 
   OPTIMIZATIONS

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7ac8b21/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobHistoryFileParser.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobHistoryFileParser.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobHistoryFileParser.java
new file mode 100644
index 0000000..9d051df
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobHistoryFileParser.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.mapred;
+
+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.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
+
+class JobHistoryFileParser {
+  private static final Log LOG = LogFactory.getLog(JobHistoryFileParser.class);
+
+  private final FileSystem fs;
+
+  public JobHistoryFileParser(FileSystem fs) {
+    LOG.info("JobHistoryFileParser created with " + fs);
+    this.fs = fs;
+  }
+
+  public JobInfo parseHistoryFile(Path path) throws IOException {
+    LOG.info("parsing job history file " + path);
+    JobHistoryParser parser = new JobHistoryParser(fs, path);
+    return parser.parse();
+  }
+
+  public Configuration parseConfiguration(Path path) throws IOException {
+    LOG.info("parsing job configuration file " + path);
+    Configuration conf = new Configuration(false);
+    conf.addResource(fs.open(path));
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7ac8b21/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobHistoryFileReplayMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobHistoryFileReplayMapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobHistoryFileReplayMapper.java
new file mode 100644
index 0000000..802b78f
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/JobHistoryFileReplayMapper.java
@@ -0,0 +1,301 @@
+/**
+ * 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.mapred;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.mapred.TimelineServicePerformanceV2.EntityWriter;
+import org.apache.hadoop.mapred.TimelineServicePerformanceV2.PerfCounters;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TypeConverter;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+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.server.timelineservice.collector.AppLevelTimelineCollector;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager;
+
+/**
+ * Mapper for TimelineServicePerformanceV2 that replays job history files to the
+ * timeline service.
+ *
+ */
+class JobHistoryFileReplayMapper extends EntityWriter {
+  private static final Log LOG =
+      LogFactory.getLog(JobHistoryFileReplayMapper.class);
+
+  static final String PROCESSING_PATH = "processing path";
+  static final String REPLAY_MODE = "replay mode";
+  static final int WRITE_ALL_AT_ONCE = 1;
+  static final int WRITE_PER_ENTITY = 2;
+  static final int REPLAY_MODE_DEFAULT = WRITE_ALL_AT_ONCE;
+
+  private static final Pattern JOB_ID_PARSER =
+      Pattern.compile("^(job_[0-9]+_([0-9]+)).*");
+
+  public static class JobFiles {
+    private final String jobId;
+    private Path jobHistoryFilePath;
+    private Path jobConfFilePath;
+
+    public JobFiles(String jobId) {
+      this.jobId = jobId;
+    }
+
+    public String getJobId() {
+      return jobId;
+    }
+
+    public Path getJobHistoryFilePath() {
+      return jobHistoryFilePath;
+    }
+
+    public void setJobHistoryFilePath(Path jobHistoryFilePath) {
+      this.jobHistoryFilePath = jobHistoryFilePath;
+    }
+
+    public Path getJobConfFilePath() {
+      return jobConfFilePath;
+    }
+
+    public void setJobConfFilePath(Path jobConfFilePath) {
+      this.jobConfFilePath = jobConfFilePath;
+    }
+
+    @Override
+    public int hashCode() {
+      return jobId.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (getClass() != obj.getClass()) {
+        return false;
+      }
+      JobFiles other = (JobFiles) obj;
+      return jobId.equals(other.jobId);
+    }
+  }
+
+  private enum FileType { JOB_HISTORY_FILE, JOB_CONF_FILE, UNKNOWN }
+
+
+  @Override
+  protected void writeEntities(Configuration tlConf,
+      TimelineCollectorManager manager, Context context) throws IOException {
+    // collect the apps it needs to process
+    Configuration conf = context.getConfiguration();
+    int taskId = context.getTaskAttemptID().getTaskID().getId();
+    int size = conf.getInt(MRJobConfig.NUM_MAPS,
+        TimelineServicePerformanceV2.NUM_MAPS_DEFAULT);
+    String processingDir =
+        conf.get(JobHistoryFileReplayMapper.PROCESSING_PATH);
+    int replayMode =
+        conf.getInt(JobHistoryFileReplayMapper.REPLAY_MODE,
+        JobHistoryFileReplayMapper.REPLAY_MODE_DEFAULT);
+    Path processingPath = new Path(processingDir);
+    FileSystem processingFs = processingPath.getFileSystem(conf);
+    JobHistoryFileParser parser = new JobHistoryFileParser(processingFs);
+    TimelineEntityConverter converter = new TimelineEntityConverter();
+
+    Collection<JobFiles> jobs =
+        selectJobFiles(processingFs, processingPath, taskId, size);
+    if (jobs.isEmpty()) {
+      LOG.info(context.getTaskAttemptID().getTaskID() +
+          " will process no jobs");
+    } else {
+      LOG.info(context.getTaskAttemptID().getTaskID() + " will process " +
+          jobs.size() + " jobs");
+    }
+    for (JobFiles job: jobs) {
+      // process each job
+      String jobIdStr = job.getJobId();
+      LOG.info("processing " + jobIdStr + "...");
+      JobId jobId = TypeConverter.toYarn(JobID.forName(jobIdStr));
+      ApplicationId appId = jobId.getAppId();
+
+      // create the app level timeline collector and start it
+      AppLevelTimelineCollector collector =
+          new AppLevelTimelineCollector(appId);
+      manager.putIfAbsent(appId, collector);
+      try {
+        // parse the job info and configuration
+        JobInfo jobInfo =
+            parser.parseHistoryFile(job.getJobHistoryFilePath());
+        Configuration jobConf =
+            parser.parseConfiguration(job.getJobConfFilePath());
+        LOG.info("parsed the job history file and the configuration file for job"
+            + jobIdStr);
+
+        // set the context
+        // flow id: job name, flow run id: timestamp, user id
+        TimelineCollectorContext tlContext =
+            collector.getTimelineEntityContext();
+        tlContext.setFlowName(jobInfo.getJobname());
+        tlContext.setFlowRunId(jobInfo.getSubmitTime());
+        tlContext.setUserId(jobInfo.getUsername());
+
+        // create entities from job history and write them
+        long totalTime = 0;
+        Set<TimelineEntity> entitySet =
+            converter.createTimelineEntities(jobInfo, jobConf);
+        LOG.info("converted them into timeline entities for job " + jobIdStr);
+        // use the current user for this purpose
+        UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+        long startWrite = System.nanoTime();
+        try {
+          switch (replayMode) {
+          case JobHistoryFileReplayMapper.WRITE_ALL_AT_ONCE:
+            writeAllEntities(collector, entitySet, ugi);
+            break;
+          case JobHistoryFileReplayMapper.WRITE_PER_ENTITY:
+            writePerEntity(collector, entitySet, ugi);
+            break;
+          default:
+            break;
+          }
+        } catch (Exception e) {
+          context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_FAILURES).
+              increment(1);
+          LOG.error("writing to the timeline service failed", e);
+        }
+        long endWrite = System.nanoTime();
+        totalTime += TimeUnit.NANOSECONDS.toMillis(endWrite-startWrite);
+        int numEntities = entitySet.size();
+        LOG.info("wrote " + numEntities + " entities in " + totalTime + " ms");
+
+        context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_TIME).
+            increment(totalTime);
+        context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_COUNTER).
+            increment(numEntities);
+      } finally {
+        manager.remove(appId);
+        context.progress(); // move it along
+      }
+    }
+  }
+
+  private void writeAllEntities(AppLevelTimelineCollector collector,
+      Set<TimelineEntity> entitySet, UserGroupInformation ugi)
+      throws IOException {
+    TimelineEntities entities = new TimelineEntities();
+    entities.setEntities(entitySet);
+    collector.putEntities(entities, ugi);
+  }
+
+  private void writePerEntity(AppLevelTimelineCollector collector,
+      Set<TimelineEntity> entitySet, UserGroupInformation ugi)
+      throws IOException {
+    for (TimelineEntity entity : entitySet) {
+      TimelineEntities entities = new TimelineEntities();
+      entities.addEntity(entity);
+      collector.putEntities(entities, ugi);
+      LOG.info("wrote entity " + entity.getId());
+    }
+  }
+
+  private Collection<JobFiles> selectJobFiles(FileSystem fs,
+      Path processingRoot, int i, int size) throws IOException {
+    Map<String,JobFiles> jobs = new HashMap<>();
+    RemoteIterator<LocatedFileStatus> it = fs.listFiles(processingRoot, true);
+    while (it.hasNext()) {
+      LocatedFileStatus status = it.next();
+      Path path = status.getPath();
+      String fileName = path.getName();
+      Matcher m = JOB_ID_PARSER.matcher(fileName);
+      if (!m.matches()) {
+        continue;
+      }
+      String jobId = m.group(1);
+      int lastId = Integer.parseInt(m.group(2));
+      int mod = lastId % size;
+      if (mod != i) {
+        continue;
+      }
+      LOG.info("this mapper will process file " + fileName);
+      // it's mine
+      JobFiles jobFiles = jobs.get(jobId);
+      if (jobFiles == null) {
+        jobFiles = new JobFiles(jobId);
+        jobs.put(jobId, jobFiles);
+      }
+      setFilePath(fileName, path, jobFiles);
+    }
+    return jobs.values();
+  }
+
+  private void setFilePath(String fileName, Path path,
+      JobFiles jobFiles) {
+    // determine if we're dealing with a job history file or a job conf file
+    FileType type = getFileType(fileName);
+    switch (type) {
+    case JOB_HISTORY_FILE:
+      if (jobFiles.getJobHistoryFilePath() == null) {
+        jobFiles.setJobHistoryFilePath(path);
+      } else {
+        LOG.warn("we already have the job history file " +
+            jobFiles.getJobHistoryFilePath() + ": skipping " + path);
+      }
+      break;
+    case JOB_CONF_FILE:
+      if (jobFiles.getJobConfFilePath() == null) {
+        jobFiles.setJobConfFilePath(path);
+      } else {
+        LOG.warn("we already have the job conf file " +
+            jobFiles.getJobConfFilePath() + ": skipping " + path);
+      }
+      break;
+    case UNKNOWN:
+      LOG.warn("unknown type: " + path);
+    }
+  }
+
+  private FileType getFileType(String fileName) {
+    if (fileName.endsWith(".jhist")) {
+      return FileType.JOB_HISTORY_FILE;
+    }
+    if (fileName.endsWith("_conf.xml")) {
+      return FileType.JOB_CONF_FILE;
+    }
+    return FileType.UNKNOWN;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7ac8b21/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/SimpleEntityWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/SimpleEntityWriter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/SimpleEntityWriter.java
new file mode 100644
index 0000000..4ef0a14
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/SimpleEntityWriter.java
@@ -0,0 +1,139 @@
+/**
+ * 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.mapred;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.TimelineServicePerformanceV2.EntityWriter;
+import org.apache.hadoop.mapred.TimelineServicePerformanceV2.PerfCounters;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+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.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineCollector;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager;
+
+/**
+   * Adds simple entities with random string payload, events, metrics, and
+   * configuration.
+   */
+class SimpleEntityWriter extends EntityWriter {
+  private static final Log LOG = LogFactory.getLog(SimpleEntityWriter.class);
+
+  // constants for mtype = 1
+  static final String KBS_SENT = "kbs sent";
+  static final int KBS_SENT_DEFAULT = 1;
+  static final String TEST_TIMES = "testtimes";
+  static final int TEST_TIMES_DEFAULT = 100;
+  static final String TIMELINE_SERVICE_PERFORMANCE_RUN_ID =
+      "timeline.server.performance.run.id";
+
+  protected void writeEntities(Configuration tlConf,
+      TimelineCollectorManager manager, Context context) throws IOException {
+    Configuration conf = context.getConfiguration();
+    // simulate the app id with the task id
+    int taskId = context.getTaskAttemptID().getTaskID().getId();
+    long timestamp = conf.getLong(TIMELINE_SERVICE_PERFORMANCE_RUN_ID, 0);
+    ApplicationId appId = ApplicationId.newInstance(timestamp, taskId);
+
+    // create the app level timeline collector
+    AppLevelTimelineCollector collector =
+        new AppLevelTimelineCollector(appId);
+    manager.putIfAbsent(appId, collector);
+
+    try {
+      // set the context
+      // flow id: job name, flow run id: timestamp, user id
+      TimelineCollectorContext tlContext =
+          collector.getTimelineEntityContext();
+      tlContext.setFlowName(context.getJobName());
+      tlContext.setFlowRunId(timestamp);
+      tlContext.setUserId(context.getUser());
+
+      final int kbs = conf.getInt(KBS_SENT, KBS_SENT_DEFAULT);
+
+      long totalTime = 0;
+      final int testtimes = conf.getInt(TEST_TIMES, TEST_TIMES_DEFAULT);
+      final Random rand = new Random();
+      final TaskAttemptID taskAttemptId = context.getTaskAttemptID();
+      final char[] payLoad = new char[kbs * 1024];
+
+      for (int i = 0; i < testtimes; i++) {
+        // Generate a fixed length random payload
+        for (int xx = 0; xx < kbs * 1024; xx++) {
+          int alphaNumIdx =
+              rand.nextInt(TimelineServicePerformanceV2.alphaNums.length);
+          payLoad[xx] = TimelineServicePerformanceV2.alphaNums[alphaNumIdx];
+        }
+        String entId = taskAttemptId + "_" + Integer.toString(i);
+        final TimelineEntity entity = new TimelineEntity();
+        entity.setId(entId);
+        entity.setType("FOO_ATTEMPT");
+        entity.addInfo("PERF_TEST", payLoad);
+        // add an event
+        TimelineEvent event = new TimelineEvent();
+        event.setTimestamp(System.currentTimeMillis());
+        event.addInfo("foo_event", "test");
+        entity.addEvent(event);
+        // add a metric
+        TimelineMetric metric = new TimelineMetric();
+        metric.setId("foo_metric");
+        metric.addValue(System.currentTimeMillis(), 123456789L);
+        entity.addMetric(metric);
+        // add a config
+        entity.addConfig("foo", "bar");
+
+        TimelineEntities entities = new TimelineEntities();
+        entities.addEntity(entity);
+        // use the current user for this purpose
+        UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+        long startWrite = System.nanoTime();
+        try {
+          collector.putEntities(entities, ugi);
+        } catch (Exception e) {
+          context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_FAILURES).
+              increment(1);
+          LOG.error("writing to the timeline service failed", e);
+        }
+        long endWrite = System.nanoTime();
+        totalTime += TimeUnit.NANOSECONDS.toMillis(endWrite-startWrite);
+      }
+      LOG.info("wrote " + testtimes + " entities (" + kbs*testtimes +
+          " kB) in " + totalTime + " ms");
+      context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_TIME).
+          increment(totalTime);
+      context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_COUNTER).
+          increment(testtimes);
+      context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_KBS).
+          increment(kbs*testtimes);
+    } finally {
+      // clean up
+      manager.remove(appId);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7ac8b21/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TimelineEntityConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TimelineEntityConverter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TimelineEntityConverter.java
new file mode 100644
index 0000000..80928dc
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TimelineEntityConverter.java
@@ -0,0 +1,207 @@
+/**
+ * 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.mapred;
+
+import java.util.HashSet;
+import java.util.Map;
+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.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.CounterGroup;
+import org.apache.hadoop.mapreduce.Counters;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
+import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+
+class TimelineEntityConverter {
+  private static final Log LOG =
+      LogFactory.getLog(TimelineEntityConverter.class);
+
+  static final String JOB = "MAPREDUCE_JOB";
+  static final String TASK = "MAPREDUCE_TASK";
+  static final String TASK_ATTEMPT = "MAPREDUCE_TASK_ATTEMPT";
+
+  /**
+   * Creates job, task, and task attempt entities based on the job history info
+   * and configuration.
+   *
+   * Note: currently these are plan timeline entities created for mapreduce
+   * types. These are not meant to be the complete and accurate entity set-up
+   * for mapreduce jobs. We do not leverage hierarchical timeline entities. If
+   * we create canonical mapreduce hierarchical timeline entities with proper
+   * parent-child relationship, we could modify this to use that instead.
+   *
+   * Note that we also do not add info to the YARN application entity, which
+   * would be needed for aggregation.
+   */
+  public Set<TimelineEntity> createTimelineEntities(JobInfo jobInfo,
+      Configuration conf) {
+    Set<TimelineEntity> entities = new HashSet<>();
+
+    // create the job entity
+    TimelineEntity job = createJobEntity(jobInfo, conf);
+    entities.add(job);
+
+    // create the task and task attempt entities
+    Set<TimelineEntity> tasksAndAttempts =
+        createTaskAndTaskAttemptEntities(jobInfo);
+    entities.addAll(tasksAndAttempts);
+
+    return entities;
+  }
+
+  private TimelineEntity createJobEntity(JobInfo jobInfo, Configuration conf) {
+    TimelineEntity job = new TimelineEntity();
+    job.setType(JOB);
+    job.setId(jobInfo.getJobId().toString());
+    job.setCreatedTime(jobInfo.getSubmitTime());
+
+    job.addInfo("JOBNAME", jobInfo.getJobname());
+    job.addInfo("USERNAME", jobInfo.getUsername());
+    job.addInfo("JOB_QUEUE_NAME", jobInfo.getJobQueueName());
+    job.addInfo("SUBMIT_TIME", jobInfo.getSubmitTime());
+    job.addInfo("LAUNCH_TIME", jobInfo.getLaunchTime());
+    job.addInfo("FINISH_TIME", jobInfo.getFinishTime());
+    job.addInfo("JOB_STATUS", jobInfo.getJobStatus());
+    job.addInfo("PRIORITY", jobInfo.getPriority());
+    job.addInfo("TOTAL_MAPS", jobInfo.getTotalMaps());
+    job.addInfo("TOTAL_REDUCES", jobInfo.getTotalReduces());
+    job.addInfo("UBERIZED", jobInfo.getUberized());
+    job.addInfo("ERROR_INFO", jobInfo.getErrorInfo());
+
+    // add metrics from total counters
+    // we omit the map counters and reduce counters for now as it's kind of
+    // awkward to put them (map/reduce/total counters are really a group of
+    // related counters)
+    Counters totalCounters = jobInfo.getTotalCounters();
+    if (totalCounters != null) {
+      addMetrics(job, totalCounters);
+    }
+    // finally add configuration to the job
+    addConfiguration(job, conf);
+    LOG.info("converted job " + jobInfo.getJobId() + " to a timeline entity");
+    return job;
+  }
+
+  private void addConfiguration(TimelineEntity job, Configuration conf) {
+    for (Map.Entry<String,String> e: conf) {
+      job.addConfig(e.getKey(), e.getValue());
+    }
+  }
+
+  private void addMetrics(TimelineEntity entity, Counters counters) {
+    for (CounterGroup g: counters) {
+      String groupName = g.getName();
+      for (Counter c: g) {
+        String name = groupName + ":" + c.getName();
+        TimelineMetric metric = new TimelineMetric();
+        metric.setId(name);
+        metric.addValue(System.currentTimeMillis(), c.getValue());
+        entity.addMetric(metric);
+      }
+    }
+  }
+
+  private Set<TimelineEntity> createTaskAndTaskAttemptEntities(JobInfo jobInfo) {
+    Set<TimelineEntity> entities = new HashSet<>();
+    Map<TaskID,TaskInfo> taskInfoMap = jobInfo.getAllTasks();
+    LOG.info("job " + jobInfo.getJobId()+ " has " + taskInfoMap.size() +
+        " tasks");
+    for (TaskInfo taskInfo: taskInfoMap.values()) {
+      TimelineEntity task = createTaskEntity(taskInfo);
+      entities.add(task);
+      // add the task attempts from this task
+      Set<TimelineEntity> taskAttempts = createTaskAttemptEntities(taskInfo);
+      entities.addAll(taskAttempts);
+    }
+    return entities;
+  }
+
+  private TimelineEntity createTaskEntity(TaskInfo taskInfo) {
+    TimelineEntity task = new TimelineEntity();
+    task.setType(TASK);
+    task.setId(taskInfo.getTaskId().toString());
+    task.setCreatedTime(taskInfo.getStartTime());
+
+    task.addInfo("START_TIME", taskInfo.getStartTime());
+    task.addInfo("FINISH_TIME", taskInfo.getFinishTime());
+    task.addInfo("TASK_TYPE", taskInfo.getTaskType());
+    task.addInfo("TASK_STATUS", taskInfo.getTaskStatus());
+    task.addInfo("ERROR_INFO", taskInfo.getError());
+
+    // add metrics from counters
+    Counters counters = taskInfo.getCounters();
+    if (counters != null) {
+      addMetrics(task, counters);
+    }
+    LOG.info("converted task " + taskInfo.getTaskId() +
+        " to a timeline entity");
+    return task;
+  }
+
+  private Set<TimelineEntity> createTaskAttemptEntities(TaskInfo taskInfo) {
+    Set<TimelineEntity> taskAttempts = new HashSet<TimelineEntity>();
+    Map<TaskAttemptID,TaskAttemptInfo> taskAttemptInfoMap =
+        taskInfo.getAllTaskAttempts();
+    LOG.info("task " + taskInfo.getTaskId() + " has " +
+        taskAttemptInfoMap.size() + " task attempts");
+    for (TaskAttemptInfo taskAttemptInfo: taskAttemptInfoMap.values()) {
+      TimelineEntity taskAttempt = createTaskAttemptEntity(taskAttemptInfo);
+      taskAttempts.add(taskAttempt);
+    }
+    return taskAttempts;
+  }
+
+  private TimelineEntity createTaskAttemptEntity(TaskAttemptInfo taskAttemptInfo) {
+    TimelineEntity taskAttempt = new TimelineEntity();
+    taskAttempt.setType(TASK_ATTEMPT);
+    taskAttempt.setId(taskAttemptInfo.getAttemptId().toString());
+    taskAttempt.setCreatedTime(taskAttemptInfo.getStartTime());
+
+    taskAttempt.addInfo("START_TIME", taskAttemptInfo.getStartTime());
+    taskAttempt.addInfo("FINISH_TIME", taskAttemptInfo.getFinishTime());
+    taskAttempt.addInfo("MAP_FINISH_TIME",
+        taskAttemptInfo.getMapFinishTime());
+    taskAttempt.addInfo("SHUFFLE_FINISH_TIME",
+        taskAttemptInfo.getShuffleFinishTime());
+    taskAttempt.addInfo("SORT_FINISH_TIME",
+        taskAttemptInfo.getSortFinishTime());
+    taskAttempt.addInfo("TASK_STATUS", taskAttemptInfo.getTaskStatus());
+    taskAttempt.addInfo("STATE", taskAttemptInfo.getState());
+    taskAttempt.addInfo("ERROR", taskAttemptInfo.getError());
+    taskAttempt.addInfo("CONTAINER_ID",
+        taskAttemptInfo.getContainerId().toString());
+
+    // add metrics from counters
+    Counters counters = taskAttemptInfo.getCounters();
+    if (counters != null) {
+      addMetrics(taskAttempt, counters);
+    }
+    LOG.info("converted task attempt " + taskAttemptInfo.getAttemptId() +
+        " to a timeline entity");
+    return taskAttempt;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7ac8b21/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TimelineServicePerformanceV2.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TimelineServicePerformanceV2.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TimelineServicePerformanceV2.java
index 1c2e28d..f674ae1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TimelineServicePerformanceV2.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TimelineServicePerformanceV2.java
@@ -20,10 +20,7 @@ package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.util.Date;
-import java.util.Random;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.IntWritable;
@@ -31,49 +28,35 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.SleepJob.SleepInputFormat;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-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.TimelineEvent;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineCollector;
-import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager;
 
 public class TimelineServicePerformanceV2 extends Configured implements Tool {
-  private static final Log LOG =
-      LogFactory.getLog(TimelineServicePerformanceV2.class);
-
   static final int NUM_MAPS_DEFAULT = 1;
 
   static final int SIMPLE_ENTITY_WRITER = 1;
-  // constants for mtype = 1
-  static final String KBS_SENT = "kbs sent";
-  static final int KBS_SENT_DEFAULT = 1;
-  static final String TEST_TIMES = "testtimes";
-  static final int TEST_TIMES_DEFAULT = 100;
-  static final String TIMELINE_SERVICE_PERFORMANCE_RUN_ID =
-      "timeline.server.performance.run.id";
-
+  static final int JOB_HISTORY_FILE_REPLAY_MAPPER = 2;
   static int mapperType = SIMPLE_ENTITY_WRITER;
 
   protected static int printUsage() {
-    // TODO is there a way to handle mapper-specific options more gracefully?
     System.err.println(
         "Usage: [-m <maps>] number of mappers (default: " + NUM_MAPS_DEFAULT +
             ")\n" +
-        "     [-mtype <mapper type in integer>] \n" +
+        "     [-mtype <mapper type in integer>]\n" +
         "          1. simple entity write mapper\n" +
-        "     [-s <(KBs)test>] number of KB per put (default: " +
-            KBS_SENT_DEFAULT + " KB)\n" +
-        "     [-t] package sending iterations per mapper (default: " +
-            TEST_TIMES_DEFAULT + ")\n");
+        "          2. job history file replay mapper\n" +
+        "     [-s <(KBs)test>] number of KB per put (mtype=1, default: " +
+             SimpleEntityWriter.KBS_SENT_DEFAULT + " KB)\n" +
+        "     [-t] package sending iterations per mapper (mtype=1, default: " +
+             SimpleEntityWriter.TEST_TIMES_DEFAULT + ")\n" +
+        "     [-d <path>] root path of job history files (mtype=2)\n" +
+        "     [-r <replay mode>] (mtype=2)\n" +
+        "          1. write all entities for a job in one put (default)\n" +
+        "          2. write one entity at a time\n");
     GenericOptionsParser.printGenericCommandUsage(System.err);
     return -1;
   }
@@ -82,11 +65,9 @@ public class TimelineServicePerformanceV2 extends Configured implements Tool {
    * Configure a job given argv.
    */
   public static boolean parseArgs(String[] args, Job job) throws IOException {
-    // set the defaults
+    // set the common defaults
     Configuration conf = job.getConfiguration();
     conf.setInt(MRJobConfig.NUM_MAPS, NUM_MAPS_DEFAULT);
-    conf.setInt(KBS_SENT, KBS_SENT_DEFAULT);
-    conf.setInt(TEST_TIMES, TEST_TIMES_DEFAULT);
 
     for (int i = 0; i < args.length; i++) {
       if (args.length == i + 1) {
@@ -97,25 +78,24 @@ public class TimelineServicePerformanceV2 extends Configured implements Tool {
         if ("-m".equals(args[i])) {
           if (Integer.parseInt(args[++i]) > 0) {
             job.getConfiguration()
-                .setInt(MRJobConfig.NUM_MAPS, (Integer.parseInt(args[i])));
+                .setInt(MRJobConfig.NUM_MAPS, Integer.parseInt(args[i]));
           }
         } else if ("-mtype".equals(args[i])) {
           mapperType = Integer.parseInt(args[++i]);
-          switch (mapperType) {
-          case SIMPLE_ENTITY_WRITER:
-            job.setMapperClass(SimpleEntityWriter.class);
-            break;
-          default:
-            job.setMapperClass(SimpleEntityWriter.class);
-          }
         } else if ("-s".equals(args[i])) {
           if (Integer.parseInt(args[++i]) > 0) {
-            conf.setInt(KBS_SENT, (Integer.parseInt(args[i])));
+            conf.setInt(SimpleEntityWriter.KBS_SENT, Integer.parseInt(args[i]));
           }
         } else if ("-t".equals(args[i])) {
           if (Integer.parseInt(args[++i]) > 0) {
-            conf.setInt(TEST_TIMES, (Integer.parseInt(args[i])));
+            conf.setInt(SimpleEntityWriter.TEST_TIMES,
+                Integer.parseInt(args[i]));
           }
+        } else if ("-d".equals(args[i])) {
+          conf.set(JobHistoryFileReplayMapper.PROCESSING_PATH, args[++i]);
+        } else if ("-r".equals(args[i])) {
+          conf.setInt(JobHistoryFileReplayMapper.REPLAY_MODE,
+              Integer.parseInt(args[++i]));
         } else {
           System.out.println("Unexpected argument: " + args[i]);
           return printUsage() == 0;
@@ -128,6 +108,27 @@ public class TimelineServicePerformanceV2 extends Configured implements Tool {
       }
     }
 
+    // handle mapper-specific settings
+    switch (mapperType) {
+    case JOB_HISTORY_FILE_REPLAY_MAPPER:
+      job.setMapperClass(JobHistoryFileReplayMapper.class);
+      String processingPath =
+          conf.get(JobHistoryFileReplayMapper.PROCESSING_PATH);
+      if (processingPath == null || processingPath.isEmpty()) {
+        System.out.println("processing path is missing while mtype = 2");
+        return printUsage() == 0;
+      }
+      break;
+    case SIMPLE_ENTITY_WRITER:
+    default:
+      job.setMapperClass(SimpleEntityWriter.class);
+      // use the current timestamp as the "run id" of the test: this will
+      // be used as simulating the cluster timestamp for apps
+      conf.setLong(SimpleEntityWriter.TIMELINE_SERVICE_PERFORMANCE_RUN_ID,
+          System.currentTimeMillis());
+      break;
+    }
+
     return true;
   }
 
@@ -153,13 +154,6 @@ public class TimelineServicePerformanceV2 extends Configured implements Tool {
       return -1;
     }
 
-    // for mtype = 1
-    // use the current timestamp as the "run id" of the test: this will be used
-    // as simulating the cluster timestamp for apps
-    Configuration conf = job.getConfiguration();
-    conf.setLong(TIMELINE_SERVICE_PERFORMANCE_RUN_ID,
-        System.currentTimeMillis());
-
     Date startTime = new Date();
     System.out.println("Job started: " + startTime);
     int ret = job.waitForCompletion(true) ? 0 : 1;
@@ -172,7 +166,8 @@ public class TimelineServicePerformanceV2 extends Configured implements Tool {
         counters.findCounter(PerfCounters.TIMELINE_SERVICE_WRITE_KBS).getValue();
     double transacrate = writecounts * 1000 / (double)writetime;
     double iorate = writesize * 1000 / (double)writetime;
-    int numMaps = Integer.parseInt(conf.get(MRJobConfig.NUM_MAPS));
+    int numMaps =
+        Integer.parseInt(job.getConfiguration().get(MRJobConfig.NUM_MAPS));
 
     System.out.println("TRANSACTION RATE (per mapper): " + transacrate +
         " ops/s");
@@ -204,95 +199,31 @@ public class TimelineServicePerformanceV2 extends Configured implements Tool {
     '3', '4', '5', '6', '7', '8', '9', '0', ' ' };
 
   /**
-   * Adds simple entities with random string payload, events, metrics, and
-   * configuration.
+   * Base mapper for writing entities to the timeline service. Subclasses
+   * override {@link #writeEntities(Configuration, TimelineCollectorManager,
+   * org.apache.hadoop.mapreduce.Mapper.Context)} to create and write entities
+   * to the timeline service.
    */
-  public static class SimpleEntityWriter
+  public static abstract class EntityWriter
       extends org.apache.hadoop.mapreduce.Mapper<IntWritable,IntWritable,Writable,Writable> {
+    @Override
     public void map(IntWritable key, IntWritable val, Context context)
         throws IOException {
 
-      Configuration conf = context.getConfiguration();
-      // simulate the app id with the task id
-      int taskId = context.getTaskAttemptID().getTaskID().getId();
-      long timestamp = conf.getLong(TIMELINE_SERVICE_PERFORMANCE_RUN_ID, 0);
-      ApplicationId appId = ApplicationId.newInstance(timestamp, taskId);
-
-      // create the app level timeline collector
+      // create the timeline collector manager wired with the writer
       Configuration tlConf = new YarnConfiguration();
-      AppLevelTimelineCollector collector =
-          new AppLevelTimelineCollector(appId);
-      collector.init(tlConf);
-      collector.start();
-
+      TimelineCollectorManager manager = new TimelineCollectorManager("test");
+      manager.init(tlConf);
+      manager.start();
       try {
-        // set the context
-        // flow id: job name, flow run id: timestamp, user id
-        TimelineCollectorContext tlContext =
-            collector.getTimelineEntityContext();
-        tlContext.setFlowName(context.getJobName());
-        tlContext.setFlowRunId(timestamp);
-        tlContext.setUserId(context.getUser());
-
-        final int kbs = Integer.parseInt(conf.get(KBS_SENT));
-
-        long totalTime = 0;
-        final int testtimes = Integer.parseInt(conf.get(TEST_TIMES));
-        final Random rand = new Random();
-        final TaskAttemptID taskAttemptId = context.getTaskAttemptID();
-        final char[] payLoad = new char[kbs * 1024];
-
-        for (int i = 0; i < testtimes; i++) {
-          // Generate a fixed length random payload
-          for (int xx = 0; xx < kbs * 1024; xx++) {
-            int alphaNumIdx = rand.nextInt(alphaNums.length);
-            payLoad[xx] = alphaNums[alphaNumIdx];
-          }
-          String entId = taskAttemptId + "_" + Integer.toString(i);
-          final TimelineEntity entity = new TimelineEntity();
-          entity.setId(entId);
-          entity.setType("FOO_ATTEMPT");
-          entity.addInfo("PERF_TEST", payLoad);
-          // add an event
-          TimelineEvent event = new TimelineEvent();
-          event.setTimestamp(System.currentTimeMillis());
-          event.addInfo("foo_event", "test");
-          entity.addEvent(event);
-          // add a metric
-          TimelineMetric metric = new TimelineMetric();
-          metric.setId("foo_metric");
-          metric.addValue(System.currentTimeMillis(), 123456789L);
-          entity.addMetric(metric);
-          // add a config
-          entity.addConfig("foo", "bar");
-
-          TimelineEntities entities = new TimelineEntities();
-          entities.addEntity(entity);
-          // use the current user for this purpose
-          UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-          long startWrite = System.nanoTime();
-          try {
-            collector.putEntities(entities, ugi);
-          } catch (Exception e) {
-            context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_FAILURES).
-                increment(1);
-            e.printStackTrace();
-          }
-          long endWrite = System.nanoTime();
-          totalTime += (endWrite-startWrite)/1000000L;
-        }
-        LOG.info("wrote " + testtimes + " entities (" + kbs*testtimes +
-            " kB) in " + totalTime + " ms");
-        context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_TIME).
-            increment(totalTime);
-        context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_COUNTER).
-            increment(testtimes);
-        context.getCounter(PerfCounters.TIMELINE_SERVICE_WRITE_KBS).
-            increment(kbs*testtimes);
+        // invoke the method to have the subclass write entities
+        writeEntities(tlConf, manager, context);
       } finally {
-        // clean up
-        collector.close();
+        manager.close();
       }
     }
+
+    protected abstract void writeEntities(Configuration tlConf,
+        TimelineCollectorManager manager, Context context) throws IOException;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7ac8b21/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 e9f2085..165754d 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
@@ -48,7 +48,7 @@ import com.google.common.annotations.VisibleForTesting;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public abstract class TimelineCollectorManager extends AbstractService {
+public class TimelineCollectorManager extends AbstractService {
   private static final Log LOG =
       LogFactory.getLog(TimelineCollectorManager.class);
 
@@ -90,10 +90,14 @@ public abstract class TimelineCollectorManager extends AbstractService {
       Collections.synchronizedMap(
           new HashMap<ApplicationId, TimelineCollector>());
 
-  protected TimelineCollectorManager(String name) {
+  public TimelineCollectorManager(String name) {
     super(name);
   }
 
+  protected TimelineWriter getWriter() {
+    return writer;
+  }
+
   /**
    * Put the collector into the collection if an collector mapped by id does
    * not exist.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7ac8b21/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 b22b39f..4385bbc 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
@@ -47,17 +47,17 @@ public class FileSystemTimelineWriterImpl extends AbstractService
 
   private String outputRoot;
 
-  /** Config param for timeline service storage tmp root for FILE YARN-3264 */
+  /** Config param for timeline service storage tmp root for FILE YARN-3264. */
   public static final String TIMELINE_SERVICE_STORAGE_DIR_ROOT
-    = YarnConfiguration.TIMELINE_SERVICE_PREFIX + "fs-writer.root-dir";
+      = YarnConfiguration.TIMELINE_SERVICE_PREFIX + "fs-writer.root-dir";
 
-  /** default value for storage location on local disk */
+  /** default value for storage location on local disk. */
   public static final String DEFAULT_TIMELINE_SERVICE_STORAGE_DIR_ROOT
-    = "/tmp/timeline_service_data";
+      = "/tmp/timeline_service_data";
 
   public static final String ENTITIES_DIR = "entities";
 
-  /** Default extension for output files */
+  /** Default extension for output files. */
   public static final String TIMELINE_SERVICE_STORAGE_EXTENSION = ".thist";
 
   FileSystemTimelineWriterImpl() {
@@ -81,9 +81,11 @@ public class FileSystemTimelineWriterImpl extends AbstractService
       TimelineWriteResponse response) throws IOException {
     PrintWriter out = null;
     try {
-      String dir = mkdirs(outputRoot, ENTITIES_DIR, clusterId, userId,flowName,
-          flowVersion, String.valueOf(flowRun), appId, entity.getType());
-      String fileName = dir + entity.getId() + TIMELINE_SERVICE_STORAGE_EXTENSION;
+      String dir = mkdirs(outputRoot, ENTITIES_DIR, clusterId, userId,
+          escape(flowName), escape(flowVersion), String.valueOf(flowRun), appId,
+          entity.getType());
+      String fileName = dir + entity.getId() +
+          TIMELINE_SERVICE_STORAGE_EXTENSION;
       out =
           new PrintWriter(new BufferedWriter(new OutputStreamWriter(
               new FileOutputStream(fileName, true), "UTF-8")));
@@ -145,4 +147,9 @@ public class FileSystemTimelineWriterImpl extends AbstractService
     }
     return path.toString();
   }
+
+  // specifically escape the separator character
+  private static String escape(String str) {
+    return str.replace(File.separatorChar, '_');
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a7ac8b21/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/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/storage/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/storage/package-info.java
new file mode 100644
index 0000000..f652ffd
--- /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/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.storage;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;


[44/50] [abbrv] hadoop git commit: Updating CHANGES.txt

Posted by gt...@apache.org.
Updating CHANGES.txt


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

Branch: refs/heads/feature-YARN-2928
Commit: e9b93fc7c39e9c3e6092a44f42bd93dfa6ee235e
Parents: dfaa7c5
Author: Varun Saxena <va...@apache.org>
Authored: Sun Dec 20 02:08:14 2015 +0530
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 18:03:30 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt | 11 ++++++++++-
 1 file changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e9b93fc7/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 169a907..611e521 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -121,6 +121,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3864. Implement support for querying single app and all apps for a
     flow run (Varun Saxena via sjlee)
 
+    YARN-4237 Support additional queries for ATSv2 Web UI. (Varun Saxena via
+    gtcarrera9)
+ 
     YARN-4179. [reader implementation] support flow activity queries based on
     time (Varun Saxena via sjlee)
 
@@ -146,6 +149,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-3862. Support for fetching specific configs and metrics based on
     prefixes (Varun Saxena via sjlee)
 
+    YARN-4356. Ensure the timeline service v.2 is disabled cleanly and has no
+    impact when it's turned off. (Sangjin Lee via gtcarrera9)
+
     YARN-4445. Unify the term flowId and flowName in timeline v2 codebase. 
     (Zhan Zhang via gtcarrera9). 
 
@@ -191,7 +197,10 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4297. TestJobHistoryEventHandler and TestRMContainerAllocator failing
     on YARN-2928 branch (Varun Saxena via sjlee)
 
-    YARN-4460. [Bug fix] RM fails to start when SMP is enabled. (Li Lu via Varun Saxena)
+    YARN-4450. TestTimelineAuthenticationFilter and TestYarnConfigurationFields
+    fail. (Sangjin Lee via gtcarrera9)
+
+    YARN-4460. [Bug fix] RM fails to start when SMP is enabled. (Li Lu via varunsaxena)
 
 Trunk - Unreleased
 


[22/50] [abbrv] hadoop git commit: YARN-4075 [reader REST API] implement support for querying for flows and flow runs (Varun Saxena via vrushali)

Posted by gt...@apache.org.
YARN-4075 [reader REST API] implement support for querying for flows and flow runs (Varun Saxena via vrushali)


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

Branch: refs/heads/feature-YARN-2928
Commit: 4552598da618d536dfd375e6bb77974e21287666
Parents: 25f58b4
Author: Vrushali Channapattan <vr...@apache.org>
Authored: Fri Sep 25 12:16:38 2015 -0700
Committer: Li Lu <gt...@apache.org>
Committed: Tue Jan 19 17:41:58 2016 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../records/timelineservice/TimelineMetric.java |  10 +
 .../reader/TimelineReaderManager.java           |  25 +-
 .../reader/TimelineReaderServer.java            |   1 +
 .../reader/TimelineReaderWebServices.java       | 243 ++++++++++--
 .../timelineservice/reader/package-info.java    |  23 ++
 .../storage/TimelineSchemaCreator.java          |   3 +
 .../reader/TestTimelineReaderWebServices.java   |  52 ++-
 .../TestTimelineReaderWebServicesFlowRun.java   | 365 +++++++++++++++++++
 .../src/test/resources/log4j.properties         |  19 +
 10 files changed, 689 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4552598d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 004fec4..4846c5c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -109,6 +109,9 @@ Branch YARN-2928: Timeline Server Next Generation: Phase 1
     YARN-4074. [timeline reader] implement support for querying for flows
     and flow runs (sjlee via vrushali)
 
+    YARN-4075. [reader REST API] implement support for querying for flows
+    and flow runs (Varun Saxena via vrushali)
+
   IMPROVEMENTS
 
     YARN-3276. Code cleanup for timeline service API records. (Junping Du via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4552598d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetric.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetric.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetric.java
index f44c1a2..e3870da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetric.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetric.java
@@ -154,4 +154,14 @@ public class TimelineMetric {
     }
     return true;
   }
+
+  @Override
+  public String toString() {
+    String str = "{id:" + id + ", type:" + type;
+    if (!values.isEmpty()) {
+      str += ", values:" + values;
+    }
+    str += "}";
+    return str;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4552598d/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 7fafd82..27a50d5 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
@@ -25,9 +25,10 @@ import java.util.Set;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 
@@ -43,6 +44,22 @@ public class TimelineReaderManager extends AbstractService {
   }
 
   /**
+   * Gets cluster ID from config yarn.resourcemanager.cluster-id
+   * if not supplied by client.
+   * @param clusterId
+   * @param conf
+   * @return clusterId
+   */
+  private static String getClusterID(String clusterId, Configuration conf) {
+    if (clusterId == null || clusterId.isEmpty()) {
+      return conf.get(
+          YarnConfiguration.RM_CLUSTER_ID,
+              YarnConfiguration.DEFAULT_RM_CLUSTER_ID);
+    }
+    return clusterId;
+  }
+
+  /**
    * Get a set of entities matching given predicates. The meaning of each
    * argument has been documented with {@link TimelineReader#getEntities}.
    *
@@ -56,7 +73,8 @@ public class TimelineReaderManager extends AbstractService {
       Map<String, Object> infoFilters, Map<String, String> configFilters,
       Set<String>  metricFilters, Set<String> eventFilters,
       EnumSet<Field> fieldsToRetrieve) throws IOException {
-    return reader.getEntities(userId, clusterId, flowId, flowRunId, appId,
+    String cluster = getClusterID(clusterId, getConfig());
+    return reader.getEntities(userId, cluster, flowId, flowRunId, appId,
         entityType, limit, createdTimeBegin, createdTimeEnd, modifiedTimeBegin,
         modifiedTimeEnd, relatesTo, isRelatedTo, infoFilters, configFilters,
         metricFilters, eventFilters, fieldsToRetrieve);
@@ -71,7 +89,8 @@ public class TimelineReaderManager extends AbstractService {
   public TimelineEntity getEntity(String userId, String clusterId,
       String flowId, Long flowRunId, String appId, String entityType,
       String entityId, EnumSet<Field> fields) throws IOException {
-    return reader.getEntity(userId, clusterId, flowId, flowRunId, appId,
+    String cluster = getClusterID(clusterId, getConfig());
+    return reader.getEntity(userId, cluster, flowId, flowRunId, appId,
         entityType, entityId, fields);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4552598d/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 319cfb0..afe1536 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
@@ -77,6 +77,7 @@ public class TimelineReaderServer extends CompositeService {
     TimelineReader readerStore = ReflectionUtils.newInstance(conf.getClass(
         YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
         FileSystemTimelineReaderImpl.class, TimelineReader.class), conf);
+    LOG.info("Using store " + readerStore.getClass().getName());
     readerStore.init(conf);
     return readerStore;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4552598d/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 0b5fde0..f619c7d 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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.timelineservice.reader;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -45,6 +46,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineAbout;
 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.storage.TimelineReader.Field;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
@@ -169,6 +171,11 @@ public class TimelineReaderWebServices {
     return str == null ? null : str.trim();
   }
 
+  private static String parseUser(UserGroupInformation callerUGI, String user) {
+    return (callerUGI != null && (user == null || user.isEmpty()) ?
+        callerUGI.getUserName().trim() : parseStr(user));
+  }
+
   private static UserGroupInformation getUser(HttpServletRequest req) {
     String remoteUser = req.getRemoteUser();
     UserGroupInformation callerUGI = null;
@@ -183,6 +190,17 @@ public class TimelineReaderWebServices {
         ctxt.getAttribute(TimelineReaderServer.TIMELINE_READER_MANAGER_ATTR);
   }
 
+  private static void handleException(Exception e) throws BadRequestException,
+      WebApplicationException {
+    if (e instanceof IllegalArgumentException) {
+      throw new BadRequestException("Requested Invalid Field.");
+    } else {
+      LOG.error("Error while processing REST request", e);
+      throw new WebApplicationException(e,
+          Response.Status.INTERNAL_SERVER_ERROR);
+    }
+  }
+
   /**
    * Return the description of the timeline reader web services.
    */
@@ -196,25 +214,58 @@ public class TimelineReaderWebServices {
   }
 
   /**
+   * Return a set of entities that match the given parameters. Cluster ID is not
+   * provided by client so default cluster ID has to be taken.
+   */
+  @GET
+  @Path("/entities/{appid}/{entitytype}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Set<TimelineEntity> getEntities(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("appid") String appId,
+      @PathParam("entitytype") String entityType,
+      @QueryParam("userid") String userId,
+      @QueryParam("flowid") String flowId,
+      @QueryParam("flowrunid") String flowRunId,
+      @QueryParam("limit") String limit,
+      @QueryParam("createdtimestart") String createdTimeStart,
+      @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("modifiedtimestart") String modifiedTimeStart,
+      @QueryParam("modifiedtimeend") String modifiedTimeEnd,
+      @QueryParam("relatesto") String relatesTo,
+      @QueryParam("isrelatedto") String isRelatedTo,
+      @QueryParam("infofilters") String infofilters,
+      @QueryParam("conffilters") String conffilters,
+      @QueryParam("metricfilters") String metricfilters,
+      @QueryParam("eventfilters") String eventfilters,
+       @QueryParam("fields") String fields) {
+    return getEntities(req, res, null, appId, entityType, userId, flowId,
+        flowRunId, limit, createdTimeStart, createdTimeEnd, modifiedTimeStart,
+        modifiedTimeEnd, relatesTo, isRelatedTo, infofilters, conffilters,
+        metricfilters, eventfilters, fields);
+  }
+
+  /**
    * Return a set of entities that match the given parameters.
    */
   @GET
-  @Path("/entities/{clusterId}/{appId}/{entityType}")
+  @Path("/entities/{clusterid}/{appid}/{entitytype}/")
   @Produces(MediaType.APPLICATION_JSON)
   public Set<TimelineEntity> getEntities(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
-      @PathParam("clusterId") String clusterId,
-      @PathParam("appId") String appId,
-      @PathParam("entityType") String entityType,
-      @QueryParam("userId") String userId,
-      @QueryParam("flowId") String flowId,
-      @QueryParam("flowRunId") String flowRunId,
+      @PathParam("clusterid") String clusterId,
+      @PathParam("appid") String appId,
+      @PathParam("entitytype") String entityType,
+      @QueryParam("userid") String userId,
+      @QueryParam("flowid") String flowId,
+      @QueryParam("flowrunid") String flowRunId,
       @QueryParam("limit") String limit,
-      @QueryParam("createdTimeStart") String createdTimeStart,
-      @QueryParam("createdTimeEnd") String createdTimeEnd,
-      @QueryParam("modifiedTimeStart") String modifiedTimeStart,
-      @QueryParam("modifiedTimeEnd") String modifiedTimeEnd,
+      @QueryParam("createdtimestart") String createdTimeStart,
+      @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("modifiedtimestart") String modifiedTimeStart,
+      @QueryParam("modifiedtimeend") String modifiedTimeEnd,
       @QueryParam("relatesto") String relatesTo,
       @QueryParam("isrelatedto") String isRelatedTo,
       @QueryParam("infofilters") String infofilters,
@@ -225,11 +276,10 @@ public class TimelineReaderWebServices {
     init(res);
     TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
     UserGroupInformation callerUGI = getUser(req);
+    Set<TimelineEntity> entities = null;
     try {
-      return timelineReaderManager.getEntities(
-          callerUGI != null && (userId == null || userId.isEmpty()) ?
-          callerUGI.getUserName().trim() : parseStr(userId),
-          parseStr(clusterId), parseStr(flowId),
+      entities = timelineReaderManager.getEntities(
+          parseUser(callerUGI, userId), parseStr(clusterId), parseStr(flowId),
           parseLongStr(flowRunId), parseStr(appId), parseStr(entityType),
           parseLongStr(limit), parseLongStr(createdTimeStart),
           parseLongStr(createdTimeEnd), parseLongStr(modifiedTimeStart),
@@ -245,31 +295,52 @@ public class TimelineReaderWebServices {
       throw new BadRequestException(
           "createdTime or modifiedTime start/end or limit or flowId is not" +
           " a numeric value.");
-    } catch (IllegalArgumentException e) {
-      throw new BadRequestException("Requested Invalid Field.");
     } catch (Exception e) {
-      LOG.error("Error getting entities", e);
-      throw new WebApplicationException(e,
-          Response.Status.INTERNAL_SERVER_ERROR);
+      handleException(e);
+    }
+    if (entities == null) {
+      entities = Collections.emptySet();
     }
+    return entities;
+  }
+
+  /**
+   * Return a single entity of the given entity type and Id. Cluster ID is not
+   * provided by client so default cluster ID has to be taken.
+   */
+  @GET
+  @Path("/entity/{appid}/{entitytype}/{entityid}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public TimelineEntity getEntity(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("appid") String appId,
+      @PathParam("entitytype") String entityType,
+      @PathParam("entityid") String entityId,
+      @QueryParam("userid") String userId,
+      @QueryParam("flowid") String flowId,
+      @QueryParam("flowrunid") String flowRunId,
+      @QueryParam("fields") String fields) {
+    return getEntity(req, res, null, appId, entityType, entityId, userId,
+        flowId, flowRunId, fields);
   }
 
   /**
    * Return a single entity of the given entity type and Id.
    */
   @GET
-  @Path("/entity/{clusterId}/{appId}/{entityType}/{entityId}/")
+  @Path("/entity/{clusterid}/{appid}/{entitytype}/{entityid}/")
   @Produces(MediaType.APPLICATION_JSON)
   public TimelineEntity getEntity(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
-      @PathParam("clusterId") String clusterId,
-      @PathParam("appId") String appId,
-      @PathParam("entityType") String entityType,
-      @PathParam("entityId") String entityId,
-      @QueryParam("userId") String userId,
-      @QueryParam("flowId") String flowId,
-      @QueryParam("flowRunId") String flowRunId,
+      @PathParam("clusterid") String clusterId,
+      @PathParam("appid") String appId,
+      @PathParam("entitytype") String entityType,
+      @PathParam("entityid") String entityId,
+      @QueryParam("userid") String userId,
+      @QueryParam("flowid") String flowId,
+      @QueryParam("flowrunid") String flowRunId,
       @QueryParam("fields") String fields) {
     init(res);
     TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
@@ -277,19 +348,13 @@ public class TimelineReaderWebServices {
     TimelineEntity entity = null;
     try {
       entity = timelineReaderManager.getEntity(
-          callerUGI != null && (userId == null || userId.isEmpty()) ?
-          callerUGI.getUserName().trim() : parseStr(userId),
-          parseStr(clusterId), parseStr(flowId), parseLongStr(flowRunId),
-          parseStr(appId), parseStr(entityType), parseStr(entityId),
-          parseFieldsStr(fields, COMMA_DELIMITER));
+          parseUser(callerUGI, userId), parseStr(clusterId), parseStr(flowId),
+          parseLongStr(flowRunId), parseStr(appId), parseStr(entityType),
+          parseStr(entityId), parseFieldsStr(fields, COMMA_DELIMITER));
     } catch (NumberFormatException e) {
-      throw new BadRequestException("flowRunId is not a numeric value.");
-    } catch (IllegalArgumentException e) {
-      throw new BadRequestException("Requested Invalid Field.");
+      throw new BadRequestException("flowrunid is not a numeric value.");
     } catch (Exception e) {
-      LOG.error("Error getting entity", e);
-      throw new WebApplicationException(e,
-          Response.Status.INTERNAL_SERVER_ERROR);
+      handleException(e);
     }
     if (entity == null) {
       throw new NotFoundException("Timeline entity {id: " + parseStr(entityId) +
@@ -297,4 +362,104 @@ public class TimelineReaderWebServices {
     }
     return entity;
   }
+
+  /**
+   * Return a single flow run for the given cluster, flow id and run id.
+   * Cluster ID is not provided by client so default cluster ID has to be taken.
+   */
+  @GET
+  @Path("/flowrun/{flowid}/{flowrunid}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public TimelineEntity getFlowRun(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("flowid") String flowId,
+      @PathParam("flowrunid") String flowRunId,
+      @QueryParam("userid") String userId,
+      @QueryParam("fields") String fields) {
+    return getFlowRun(req, res, null, flowId, flowRunId, userId, fields);
+  }
+
+  /**
+   * Return a single flow run for the given cluster, flow id and run id.
+   */
+  @GET
+  @Path("/flowrun/{clusterid}/{flowid}/{flowrunid}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public TimelineEntity getFlowRun(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("clusterid") String clusterId,
+      @PathParam("flowid") String flowId,
+      @PathParam("flowrunid") String flowRunId,
+      @QueryParam("userid") String userId,
+      @QueryParam("fields") String fields) {
+    init(res);
+    TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
+    UserGroupInformation callerUGI = getUser(req);
+    TimelineEntity entity = null;
+    try {
+      entity = timelineReaderManager.getEntity(
+          parseUser(callerUGI, userId), parseStr(clusterId),
+          parseStr(flowId), parseLongStr(flowRunId), null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null,
+          parseFieldsStr(fields, COMMA_DELIMITER));
+    } catch (NumberFormatException e) {
+      throw new BadRequestException("flowRunId is not a numeric value.");
+    } catch (Exception e) {
+      handleException(e);
+    }
+    if (entity == null) {
+      throw new NotFoundException("Flow run {flow id: " + parseStr(flowId) +
+          ", run id: " + parseLongStr(flowRunId) + " } is not found");
+    }
+    return entity;
+  }
+
+  /**
+   * Return a list of flows for a given cluster id. Cluster ID is not
+   * provided by client so default cluster ID has to be taken.
+   */
+  @GET
+  @Path("/flows/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Set<TimelineEntity> getFlows(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @QueryParam("limit") String limit,
+      @QueryParam("fields") String fields) {
+    return getFlows(req, res, null, limit, fields);
+  }
+
+  /**
+   * Return a list of flows for a given cluster id.
+   */
+  @GET
+  @Path("/flows/{clusterid}/")
+  @Produces(MediaType.APPLICATION_JSON)
+  public Set<TimelineEntity> getFlows(
+      @Context HttpServletRequest req,
+      @Context HttpServletResponse res,
+      @PathParam("clusterid") String clusterId,
+      @QueryParam("limit") String limit,
+      @QueryParam("fields") String fields) {
+    init(res);
+    TimelineReaderManager timelineReaderManager = getTimelineReaderManager();
+    Set<TimelineEntity> entities = null;
+    try {
+      entities = timelineReaderManager.getEntities(
+          null, parseStr(clusterId), null, null, null,
+          TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), parseLongStr(limit),
+          null, null, null, null, null, null, null, null, null, null,
+          parseFieldsStr(fields, COMMA_DELIMITER));
+    } catch (NumberFormatException e) {
+      throw new BadRequestException("limit is not a numeric value.");
+    } catch (Exception e) {
+      handleException(e);
+    }
+    if (entities == null) {
+      entities = Collections.emptySet();
+    }
+    return entities;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4552598d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/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/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/package-info.java
new file mode 100644
index 0000000..51247bd
--- /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/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.server.timelineservice.reader;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4552598d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
index cbcff4c..46bc2e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/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/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java
@@ -45,6 +45,8 @@ 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 com.google.common.annotations.VisibleForTesting;
+
 /**
  * This creates the schema for a hbase based backend for storing application
  * timeline information.
@@ -201,6 +203,7 @@ public class TimelineSchemaCreator {
     return commandLine;
   }
 
+  @VisibleForTesting
   public static void createAllTables(Configuration hbaseConf,
       boolean skipExisting) throws IOException {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4552598d/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 0f7c22f..45bce2f 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
@@ -187,8 +187,8 @@ public class TestTimelineReaderWebServices {
     Client client = createClient();
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/entity/cluster1/app1/app/id_1?userId=user1&" +
-          "flowId=flow1&flowRunId=1");
+          "timeline/entity/cluster1/app1/app/id_1?userid=user1&" +
+          "flowid=flow1&flowrunid=1");
       ClientResponse resp = getResponse(client, uri);
       TimelineEntity entity = resp.getEntity(TimelineEntity.class);
       assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
@@ -259,6 +259,32 @@ public class TestTimelineReaderWebServices {
   }
 
   @Test
+  public void testQueryWithoutCluster() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entity/app1/app/id_1");
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entity);
+      assertEquals("id_1", entity.getId());
+      assertEquals("app", entity.getType());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entities/app1/app");
+      resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(4, entities.size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
   public void testGetEntities() throws Exception {
     Client client = createClient();
     try {
@@ -318,8 +344,8 @@ public class TestTimelineReaderWebServices {
     Client client = createClient();
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/entities/cluster1/app1/app?createdTimeStart=1425016502030&"
-          + "createdTimeEnd=1425016502060");
+          "timeline/entities/cluster1/app1/app?createdtimestart=1425016502030&"
+          + "createdtimeend=1425016502060");
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -330,7 +356,7 @@ public class TestTimelineReaderWebServices {
           entities.contains(newEntity("app", "id_4")));
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "entities/cluster1/app1/app?createdTimeEnd=1425016502010");
+          "entities/cluster1/app1/app?createdtimeend=1425016502010");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
       assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
@@ -340,7 +366,7 @@ public class TestTimelineReaderWebServices {
           entities.contains(newEntity("app", "id_4")));
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "entities/cluster1/app1/app?createdTimeStart=1425016502010");
+          "entities/cluster1/app1/app?createdtimestart=1425016502010");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
       assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
@@ -358,8 +384,8 @@ public class TestTimelineReaderWebServices {
     Client client = createClient();
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/entities/cluster1/app1/app?modifiedTimeStart=1425016502090"
-          + "&modifiedTimeEnd=1425016503020");
+          "timeline/entities/cluster1/app1/app?modifiedtimestart=1425016502090"
+          + "&modifiedtimeend=1425016503020");
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -372,7 +398,7 @@ public class TestTimelineReaderWebServices {
           entities.contains(newEntity("app", "id_4")));
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "entities/cluster1/app1/app?modifiedTimeEnd=1425016502090");
+          "entities/cluster1/app1/app?modifiedtimeend=1425016502090");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
       assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
@@ -384,7 +410,7 @@ public class TestTimelineReaderWebServices {
           entities.contains(newEntity("app", "id_3")));
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "entities/cluster1/app1/app?modifiedTimeStart=1425016503005");
+          "entities/cluster1/app1/app?modifiedtimestart=1425016503005");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
       assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
@@ -527,7 +553,7 @@ public class TestTimelineReaderWebServices {
           "timeline/entities/cluster1/app1/app?metricfilters=metric7&" +
           "isrelatedto=type1:tid1_1;tid1_2,type2:tid2_1%60&relatesto=" +
           "flow:flow1&eventfilters=event_2,event_4&infofilters=info2:3.5" +
-          "&createdTimeStart=1425016502030&createdTimeEnd=1425016502060");
+          "&createdtimestart=1425016502030&createdtimeend=1425016502060");
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -544,11 +570,11 @@ public class TestTimelineReaderWebServices {
     Client client = createClient();
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/entities/cluster1/app1/app?flowRunId=a23b");
+          "timeline/entities/cluster1/app1/app?flowrunid=a23b");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "entity/cluster1/app1/app/id_1?flowRunId=2ab15");
+          "entity/cluster1/app1/app/id_1?flowrunid=2ab15");
       verifyHttpResponse(client, uri, Status.BAD_REQUEST);
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4552598d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesFlowRun.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/TestTimelineReaderWebServicesFlowRun.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesFlowRun.java
new file mode 100644
index 0000000..ae71e2c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesFlowRun.java
@@ -0,0 +1,365 @@
+/**
+ * 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.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+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.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.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.HBaseTimelineWriterImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator;
+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;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+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 TestTimelineReaderWebServicesFlowRun {
+  private int serverPort;
+  private TimelineReaderServer server;
+  private static HBaseTestingUtility util;
+  private static long ts = System.currentTimeMillis();
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    util = new HBaseTestingUtility();
+    Configuration conf = util.getConfiguration();
+    conf.setInt("hfile.format.version", 3);
+    util.startMiniCluster();
+    TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
+    loadData();
+  }
+
+  private static void loadData() throws Exception {
+    String cluster = "cluster1";
+    String user = "user1";
+    String flow = "flow_name";
+    String flowVersion = "CF7022C10F1354";
+    Long runid = 1002345678919L;
+    Long runid1 = 1002345678920L;
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entity = new TimelineEntity();
+    String id = "flowRunMetrics_test";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    Long cTime = 1425016501000L;
+    entity.setCreatedTime(cTime);
+
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    TimelineMetric m1 = new TimelineMetric();
+    m1.setId("MAP_SLOT_MILLIS");
+    Map<Long, Number> metricValues = new HashMap<Long, Number>();
+    metricValues.put(ts - 100000, 2);
+    metricValues.put(ts - 80000, 40);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+
+    m1 = new TimelineMetric();
+    m1.setId("HDFS_BYTES_READ");
+    metricValues = new HashMap<Long, Number>();
+    metricValues.put(ts - 100000, 31);
+    metricValues.put(ts - 80000, 57);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    entity.addMetrics(metrics);
+
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    Long expTs = 1436512802000L;
+    event.setTimestamp(expTs);
+    String expKey = "foo_event";
+    Object expVal = "test";
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+    te.addEntity(entity);
+
+    // write another application with same metric to this flow
+    TimelineEntities te1 = new TimelineEntities();
+    TimelineEntity entity1 = new TimelineEntity();
+    id = "flowRunMetrics_test";
+    type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity1.setId(id);
+    entity1.setType(type);
+    cTime = 1425016501000L;
+    entity1.setCreatedTime(cTime);
+    // add metrics
+    metrics.clear();
+    TimelineMetric m2 = new TimelineMetric();
+    m2.setId("MAP_SLOT_MILLIS");
+    metricValues = new HashMap<Long, Number>();
+    metricValues.put(ts - 100000, 5L);
+    metricValues.put(ts - 80000, 101L);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues);
+    metrics.add(m2);
+    entity1.addMetrics(metrics);
+    te1.addEntity(entity1);
+
+    String flow2 = "flow_name2";
+    String flowVersion2 = "CF7022C10F1454";
+    Long runid2 = 2102356789046L;
+    TimelineEntities te3 = new TimelineEntities();
+    TimelineEntity entity3 = new TimelineEntity();
+    id = "flowRunMetrics_test1";
+    entity3.setId(id);
+    entity3.setType(type);
+    cTime = 1425016501030L;
+    entity3.setCreatedTime(cTime);
+    TimelineEvent event2 = new TimelineEvent();
+    event2.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    event2.setTimestamp(1436512802030L);
+    event2.addInfo("foo_event", "test");
+    entity3.addEvent(event2);
+    te3.addEntity(entity3);
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      String appName = "application_11111111111111_1111";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      appName = "application_11111111111111_2222";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+      hbi.write(cluster, user, flow, flowVersion, runid1, appName, te);
+      appName = "application_11111111111111_2223";
+      hbi.write(cluster, user, flow2, flowVersion2, runid2, appName, te3);
+      hbi.flush();
+    } finally {
+      hbi.close();
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    util.shutdownMiniCluster();
+  }
+
+  @Before
+  public void init() throws Exception {
+    try {
+      Configuration config = util.getConfiguration();
+      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 = new String();
+      if (resp != null) {
+        msg = resp.getClientResponseStatus().toString();
+      }
+      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 TimelineMetric newMetric(String id, long ts, Number value) {
+    TimelineMetric metric = new TimelineMetric();
+    metric.setId(id);
+    metric.addValue(ts, value);
+    return metric;
+  }
+
+  private static boolean verifyMetricValues(Map<Long, Number> m1,
+      Map<Long, Number> m2) {
+    for (Map.Entry<Long, Number> entry : m1.entrySet()) {
+      if (!m2.containsKey(entry.getKey())) {
+        return false;
+      }
+      if (m2.get(entry.getKey()).equals(entry.getValue())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static boolean verifyMetrics(
+      TimelineMetric m, TimelineMetric... metrics) {
+    for (TimelineMetric metric : metrics) {
+      if (!metric.equals(m)) {
+        continue;
+      }
+      if (!verifyMetricValues(metric.getValues(), m.getValues())) {
+        continue;
+      }
+      return true;
+    }
+    return false;
+  }
+
+  @Test
+  public void testGetFlowRun() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowrun/cluster1/flow_name/1002345678919?userid=user1");
+      ClientResponse resp = getResponse(client, uri);
+      FlowRunEntity entity = resp.getEntity(FlowRunEntity.class);
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entity);
+      assertEquals("user1@flow_name/1002345678919", entity.getId());
+      assertEquals(2, entity.getMetrics().size());
+      TimelineMetric m1 = newMetric("HDFS_BYTES_READ", ts - 80000, 57L);
+      TimelineMetric m2 = newMetric("MAP_SLOT_MILLIS", ts - 80000, 141L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m1, m2));
+      }
+
+      // Query without specifying cluster ID.
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flowrun/flow_name/1002345678919?userid=user1");
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(FlowRunEntity.class);
+      assertNotNull(entity);
+      assertEquals("user1@flow_name/1002345678919", entity.getId());
+      assertEquals(2, entity.getMetrics().size());
+      m1 = newMetric("HDFS_BYTES_READ", ts - 80000, 57L);
+      m2 = newMetric("MAP_SLOT_MILLIS", ts - 80000, 141L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m1, m2));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlows() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows/cluster1");
+      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));
+      }
+
+      // 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());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+              "timeline/flows/cluster1?limit=1");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @After
+  public void stop() throws Exception {
+    if (server != null) {
+      server.stop();
+      server = null;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4552598d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/resources/log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/resources/log4j.properties
new file mode 100644
index 0000000..81a3f6a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/resources/log4j.properties
@@ -0,0 +1,19 @@
+#   Licensed 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.
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n


[03/50] [abbrv] hadoop git commit: YARN-3049. [Storage Implementation] Implement storage reader interface to fetch raw data from HBase backend (Zhijie Shen via sjlee)

Posted by gt...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/0bed3fb3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.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/TestHBaseTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
index fd5643d..ab02779 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineWriterImpl.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -38,11 +39,15 @@ 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.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
 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.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.storage.apptoflow.AppToFlowTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineWriterUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumn;
@@ -71,6 +76,8 @@ public class TestHBaseTimelineWriterImpl {
   private static void createSchema() throws IOException {
     new EntityTable()
         .createTable(util.getHBaseAdmin(), util.getConfiguration());
+    new AppToFlowTable()
+        .createTable(util.getHBaseAdmin(), util.getConfiguration());
   }
 
   @Test
@@ -138,10 +145,15 @@ public class TestHBaseTimelineWriterImpl {
     te.addEntity(entity);
 
     HBaseTimelineWriterImpl hbi = null;
+    HBaseTimelineReaderImpl hbr = null;
     try {
       Configuration c1 = util.getConfiguration();
       hbi = new HBaseTimelineWriterImpl(c1);
       hbi.init(c1);
+      hbi.start();
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
       String cluster = "cluster1";
       String user = "user1";
       String flow = "some_flow_name";
@@ -255,9 +267,22 @@ public class TestHBaseTimelineWriterImpl {
       assertEquals(1, rowCount);
       assertEquals(17, colCount);
 
+      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
+          entity.getType(), entity.getId(), EnumSet.of(TimelineReader.Field.ALL));
+      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
+          appName, entity.getType(), null, null, null, null, null, null, null,
+          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
+      assertNotNull(e1);
+      assertEquals(1, es1.size());
     } finally {
-      hbi.stop();
-      hbi.close();
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+      if (hbr != null) {
+        hbr.stop();
+        hbr.close();
+      }
     }
 
     // Somewhat of a hack, not a separate test in order not to have to deal with
@@ -283,7 +308,7 @@ public class TestHBaseTimelineWriterImpl {
 
   private void testAdditionalEntity() throws IOException {
     TimelineEvent event = new TimelineEvent();
-    String eventId = "foo_event_id";
+    String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE;
     event.setId(eventId);
     Long expTs = 1436512802000L;
     event.setTimestamp(expTs);
@@ -291,19 +316,23 @@ public class TestHBaseTimelineWriterImpl {
     Object expVal = "test";
     event.addInfo(expKey, expVal);
 
-    final TimelineEntity entity = new TimelineEntity();
-    entity.setId("attempt_1329348432655_0001_m_000008_18");
-    entity.setType("FOO_ATTEMPT");
+    final TimelineEntity entity = new ApplicationEntity();
+    entity.setId(ApplicationId.newInstance(0, 1).toString());
     entity.addEvent(event);
 
     TimelineEntities entities = new TimelineEntities();
     entities.addEntity(entity);
 
     HBaseTimelineWriterImpl hbi = null;
+    HBaseTimelineReaderImpl hbr = null;
     try {
       Configuration c1 = util.getConfiguration();
       hbi = new HBaseTimelineWriterImpl(c1);
       hbi.init(c1);
+      hbi.start();
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
       String cluster = "cluster2";
       String user = "user2";
       String flow = "other_flow_name";
@@ -352,9 +381,31 @@ public class TestHBaseTimelineWriterImpl {
       }
       assertEquals(1, rowCount);
 
+      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
+          entity.getType(), entity.getId(), EnumSet.of(TimelineReader.Field.ALL));
+      TimelineEntity e2 = hbr.getEntity(user, cluster, null, null, appName,
+          entity.getType(), entity.getId(), EnumSet.of(TimelineReader.Field.ALL));
+      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
+          appName, entity.getType(), null, null, null, null, null, null, null,
+          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
+      Set<TimelineEntity> es2 = hbr.getEntities(user, cluster, null, null,
+          appName, entity.getType(), null, null, null, null, null, null, null,
+          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
+      assertNotNull(e1);
+      assertNotNull(e2);
+      assertEquals(e1, e2);
+      assertEquals(1, es1.size());
+      assertEquals(1, es2.size());
+      assertEquals(es1, es2);
     } finally {
-      hbi.stop();
-      hbi.close();
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+      if (hbr != null) {
+        hbr.stop();
+        hbr.close();
+      }
     }
   }
 
@@ -375,10 +426,15 @@ public class TestHBaseTimelineWriterImpl {
     entities.addEntity(entity);
 
     HBaseTimelineWriterImpl hbi = null;
+    HBaseTimelineReaderImpl hbr = null;
     try {
       Configuration c1 = util.getConfiguration();
       hbi = new HBaseTimelineWriterImpl(c1);
       hbi.init(c1);
+      hbi.start();
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
       String cluster = "cluster_emptyeventkey";
       String user = "user_emptyeventkey";
       String flow = "other_flow_name";
@@ -430,13 +486,21 @@ public class TestHBaseTimelineWriterImpl {
       }
       assertEquals(1, rowCount);
 
+      TimelineEntity e1 = hbr.getEntity(user, cluster, flow, runid, appName,
+          entity.getType(), entity.getId(), EnumSet.of(TimelineReader.Field.ALL));
+      Set<TimelineEntity> es1 = hbr.getEntities(user, cluster, flow, runid,
+          appName, entity.getType(), null, null, null, null, null, null, null,
+          null, null, null, null, EnumSet.of(TimelineReader.Field.ALL));
+      assertNotNull(e1);
+      assertEquals(1, es1.size());
     } finally {
       hbi.stop();
       hbi.close();
+      hbr.stop();;
+      hbr.close();
     }
   }
 
-
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     util.shutdownMiniCluster();