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 vr...@apache.org on 2016/06/21 23:48:47 UTC

[01/50] [abbrv] hadoop git commit: YARN-3863. Support complex filters in TimelineReader (Varun Saxena via sjlee) [Forced Update!]

Repository: hadoop
Updated Branches:
  refs/heads/YARN-2928 b9b9068ed -> 92faa0152 (forced update)


http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 9793ce6..3b8036d 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
@@ -154,6 +154,14 @@ class TestFlowDataGenerator {
     metrics.add(m2);
 
     entity.addMetrics(metrics);
+    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;
   }
 
@@ -178,6 +186,14 @@ class TestFlowDataGenerator {
     m1.setValues(metricValues);
     metrics.add(m1);
     entity.addMetrics(metrics);
+    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;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 f04dd48..a724db2 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
@@ -47,8 +47,10 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
 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.TimelineFilterList.Operator;
 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;
@@ -307,7 +309,7 @@ public class TestHBaseStorageFlowRun {
       assertEquals(141L, Bytes.toLong(values.get(q)));
 
       // check metric2
-      assertEquals(2, values.size());
+      assertEquals(3, values.size());
       q = ColumnHelper.getColumnQualifier(
           FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric2);
       assertTrue(values.containsKey(q));
@@ -318,11 +320,10 @@ public class TestHBaseStorageFlowRun {
 
   @Test
   public void testWriteFlowRunMetricsPrefix() throws Exception {
-    String cluster = "testWriteFlowRunMetricsOneFlow_cluster1";
-    String user = "testWriteFlowRunMetricsOneFlow_user1";
-    String flow = "testing_flowRun_metrics_flow_name";
+    String cluster = "testWriteFlowRunMetricsPrefix_cluster1";
+    String user = "testWriteFlowRunMetricsPrefix_user1";
+    String flow = "testWriteFlowRunMetricsPrefix_flow_name";
     String flowVersion = "CF7022C10F1354";
-    long runid = 1002345678919L;
 
     TimelineEntities te = new TimelineEntities();
     TimelineEntity entityApp1 = TestFlowDataGenerator
@@ -335,33 +336,30 @@ public class TestHBaseStorageFlowRun {
       hbi = new HBaseTimelineWriterImpl(c1);
       hbi.init(c1);
       String appName = "application_11111111111111_1111";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(cluster, user, flow, flowVersion, 1002345678919L, appName, te);
       // write another application with same metric to this flow
       te = new TimelineEntities();
       TimelineEntity entityApp2 = TestFlowDataGenerator
           .getEntityMetricsApp2(System.currentTimeMillis());
       te.addEntity(entityApp2);
       appName = "application_11111111111111_2222";
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
+      hbi.write(cluster, user, flow, flowVersion, 1002345678918L, 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)));
+      TimelineFilterList metricsToRetrieve = new TimelineFilterList(
+          Operator.OR, new TimelinePrefixFilter(TimelineCompareOp.EQUAL,
+              metric1.substring(0, metric1.indexOf("_") + 1)));
       TimelineEntity entity = hbr.getEntity(
-          new TimelineReaderContext(cluster, user, flow, runid, null,
+          new TimelineReaderContext(cluster, user, flow, 1002345678919L, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
           new TimelineDataToRetrieve(null, metricsToRetrieve, null));
       assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
@@ -377,7 +375,7 @@ public class TestHBaseStorageFlowRun {
         }
         switch (id) {
         case metric1:
-          assertEquals(141L, value);
+          assertEquals(40L, value);
           break;
         default:
           fail("unrecognized metric: " + id);
@@ -385,31 +383,16 @@ public class TestHBaseStorageFlowRun {
       }
 
       Set<TimelineEntity> entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, user, flow, runid, null,
+          new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
           new TimelineEntityFilters(),
           new TimelineDataToRetrieve(null, metricsToRetrieve, null));
-      assertEquals(1, entities.size());
+      assertEquals(2, entities.size());
+      int metricCnt = 0;
       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);
-          }
-        }
+        metricCnt += timelineEntity.getMetrics().size();
       }
+      assertEquals(2, metricCnt);
     } finally {
       hbr.close();
     }
@@ -417,9 +400,9 @@ public class TestHBaseStorageFlowRun {
 
   @Test
   public void testWriteFlowRunsMetricFields() throws Exception {
-    String cluster = "testWriteFlowRunMetricsOneFlow_cluster1";
-    String user = "testWriteFlowRunMetricsOneFlow_user1";
-    String flow = "testing_flowRun_metrics_flow_name";
+    String cluster = "testWriteFlowRunsMetricFields_cluster1";
+    String user = "testWriteFlowRunsMetricFields_user1";
+    String flow = "testWriteFlowRunsMetricFields_flow_name";
     String flowVersion = "CF7022C10F1354";
     long runid = 1002345678919L;
 
@@ -592,6 +575,214 @@ public class TestHBaseStorageFlowRun {
     }
   }
 
+  @Test
+  public void testFilterFlowRunsByCreatedTime() throws Exception {
+    String cluster = "cluster2";
+    String user = "user2";
+    String flow = "flow_name2";
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1(
+        System.currentTimeMillis());
+    entityApp1.setCreatedTime(1425016501000L);
+    te.addEntity(entityApp1);
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678919L,
+          "application_11111111111111_1111", te);
+      // write another application with same metric to this flow
+      te = new TimelineEntities();
+      TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2(
+          System.currentTimeMillis());
+      entityApp2.setCreatedTime(1425016502000L);
+      te.addEntity(entityApp2);
+      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678918L,
+          "application_11111111111111_2222", te);
+      hbi.flush();
+    } finally {
+      hbi.close();
+    }
+
+    // use the timeline reader to verify data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+
+      Set<TimelineEntity> entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow,
+          null, null, TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, 1425016501000L, 1425016502001L, null,
+          null, null, null, null, null), new TimelineDataToRetrieve());
+      assertEquals(2, entities.size());
+      for (TimelineEntity entity : entities) {
+        if (!entity.getId().equals("user2@flow_name2/1002345678918") &&
+            !entity.getId().equals("user2@flow_name2/1002345678919")) {
+          fail("Entities with flow runs 1002345678918 and 1002345678919" +
+              "should be present.");
+        }
+      }
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, 1425016501050L, null, null, null,
+          null, null, null, null), new TimelineDataToRetrieve());
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        if (!entity.getId().equals("user2@flow_name2/1002345678918")) {
+          fail("Entity with flow run 1002345678918 should be present.");
+        }
+      }
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, null, 1425016501050L, null, null,
+          null, null, null, null), new TimelineDataToRetrieve());
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        if (!entity.getId().equals("user2@flow_name2/1002345678919")) {
+          fail("Entity with flow run 1002345678919 should be present.");
+        }
+      }
+    } finally {
+      hbr.close();
+    }
+  }
+
+  @Test
+  public void testMetricFilters() throws Exception {
+    String cluster = "cluster1";
+    String user = "user1";
+    String flow = "flow_name1";
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1(
+        System.currentTimeMillis());
+    te.addEntity(entityApp1);
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678919L,
+          "application_11111111111111_1111", te);
+      // write another application with same metric to this flow
+      te = new TimelineEntities();
+      TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2(
+          System.currentTimeMillis());
+      te.addEntity(entityApp2);
+      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678918L,
+          "application_11111111111111_2222", te);
+      hbi.flush();
+    } finally {
+      hbi.close();
+    }
+
+    // use the timeline reader to verify data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+
+      TimelineFilterList list1 = new TimelineFilterList();
+      list1.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.GREATER_OR_EQUAL, metric1, 101));
+      TimelineFilterList list2 = new TimelineFilterList();
+      list2.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.LESS_THAN, metric1, 43));
+      list2.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.EQUAL, metric2, 57));
+      TimelineFilterList metricFilterList =
+          new TimelineFilterList(Operator.OR, list1, list2);
+      Set<TimelineEntity> entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null,
+          null, TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, null, null, null, null, null, null,
+          metricFilterList, null),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+      assertEquals(2, entities.size());
+      int metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+      }
+      assertEquals(3, metricCnt);
+
+      TimelineFilterList metricFilterList1 = new TimelineFilterList(
+          new TimelineCompareFilter(
+          TimelineCompareOp.LESS_OR_EQUAL, metric1, 127),
+          new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, metric2, 30));
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, null, null, null, null, null, null,
+          metricFilterList1, null),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+      assertEquals(1, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+      }
+      assertEquals(2, metricCnt);
+
+      TimelineFilterList metricFilterList2 = new TimelineFilterList(
+          new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, metric1, 32),
+          new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, metric2, 57));
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, null, null, null, null, null, null,
+          metricFilterList2, null),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+      assertEquals(0, entities.size());
+
+      TimelineFilterList metricFilterList3 = new TimelineFilterList(
+          new TimelineCompareFilter(TimelineCompareOp.EQUAL, "s_metric", 32));
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, null, null, null, null, null, null,
+          metricFilterList3, null),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+      assertEquals(0, entities.size());
+
+      TimelineFilterList list3 = new TimelineFilterList();
+      list3.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.GREATER_OR_EQUAL, metric1, 101));
+      TimelineFilterList list4 = new TimelineFilterList();
+      list4.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.LESS_THAN, metric1, 43));
+      list4.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.EQUAL, metric2, 57));
+      TimelineFilterList metricFilterList4 =
+          new TimelineFilterList(Operator.OR, list3, list4);
+      TimelineFilterList metricsToRetrieve = new TimelineFilterList(Operator.OR,
+          new TimelinePrefixFilter(TimelineCompareOp.EQUAL,
+          metric2.substring(0, metric2.indexOf("_") + 1)));
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, null, null, null, null, null, null,
+          metricFilterList4, null),
+          new TimelineDataToRetrieve(null, metricsToRetrieve,
+          EnumSet.of(Field.ALL)));
+      assertEquals(2, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+      }
+      assertEquals(1, metricCnt);
+    } finally {
+      hbr.close();
+    }
+  }
+
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     util.shutdownMiniCluster();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
index ace218b..2738e6a 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/TestHBaseStorageFlowRunCompaction.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/TestHBaseStorageFlowRunCompaction.java
@@ -148,7 +148,7 @@ public class TestHBaseStorageFlowRunCompaction {
     }
 
     // check flow run for one flow many apps
-    checkFlowRunTable(cluster, user, flow, runid, c1, 3);
+    checkFlowRunTable(cluster, user, flow, runid, c1, 4);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java b/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java
new file mode 100644
index 0000000..36d0d7b
--- /dev/null
+++ b/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java
@@ -0,0 +1,62 @@
+/**
+ * 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 existence of a
+ * value.
+ */
+@Private
+@Unstable
+public class TimelineExistsFilter extends TimelineFilter {
+
+  private final TimelineCompareOp compareOp;
+  private final String value;
+
+  public TimelineExistsFilter(TimelineCompareOp op, String value) {
+    this.value = value;
+    if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
+      throw new IllegalArgumentException("CompareOp for exists filter should " +
+          "be EQUAL or NOT_EQUAL");
+    }
+    this.compareOp = op;
+  }
+
+  @Override
+  public TimelineFilterType getFilterType() {
+    return TimelineFilterType.EXISTS;
+  }
+
+  public String getValue() {
+    return value;
+  }
+
+  public TimelineCompareOp getCompareOp() {
+    return compareOp;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("%s (%s %s)",
+        this.getClass().getSimpleName(), this.compareOp.name(), this.value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java b/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java
new file mode 100644
index 0000000..58f0ee9
--- /dev/null
+++ b/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.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.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
+ * being equal or not to the values in back-end store.
+ */
+@Private
+@Unstable
+public class TimelineKeyValueFilter extends TimelineCompareFilter {
+  public TimelineKeyValueFilter(TimelineCompareOp op, String key, Object val,
+      boolean keyMustExistFlag) {
+    super(op, key, val, keyMustExistFlag);
+    if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
+      throw new IllegalArgumentException("TimelineCompareOp for equality"
+          + " filter should be EQUAL or NOT_EQUAL");
+    }
+  }
+
+  public TimelineKeyValueFilter(TimelineCompareOp op, String key, Object val) {
+    this(op, key, val, true);
+  }
+
+  @Override
+  public TimelineFilterType getFilterType() {
+    return TimelineFilterType.KEY_VALUE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java b/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java
new file mode 100644
index 0000000..0d34d47
--- /dev/null
+++ b/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java
@@ -0,0 +1,71 @@
+/**
+ * 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.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Filter class which represents filter to be applied based on multiple values
+ * for a key and these values being equal or not equal to values in back-end
+ * store.
+ */
+@Private
+@Unstable
+public class TimelineKeyValuesFilter extends TimelineFilter {
+  private final TimelineCompareOp compareOp;
+  private final String key;
+  private final Set<Object> values;
+  public TimelineKeyValuesFilter(TimelineCompareOp op, String key,
+      Set<Object> values) {
+    if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
+      throw new IllegalArgumentException("TimelineCompareOp for multi value "
+          + "equality filter should be EQUAL or NOT_EQUAL");
+    }
+    this.compareOp = op;
+    this.key = key;
+    this.values = values;
+  }
+
+  @Override
+  public TimelineFilterType getFilterType() {
+    return TimelineFilterType.KEY_VALUES;
+  }
+
+  public String getKey() {
+    return key;
+  }
+
+  public Set<Object> getValues() {
+    return values;
+  }
+
+  public TimelineCompareOp getCompareOp() {
+    return compareOp;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("%s (%s, %s:%s)",
+        this.getClass().getSimpleName(), this.compareOp.name(),
+        this.key, (values == null) ? "" : values.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java b/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java
new file mode 100644
index 0000000..4099e92
--- /dev/null
+++ b/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java
@@ -0,0 +1,71 @@
+/**
+ * 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.server.timelineservice.reader.filter.TimelineFilter.TimelineFilterType;
+
+/**
+ * Used to define which filter to match.
+ */
+enum TimelineEntityFiltersType {
+  CONFIG {
+    boolean isValidFilter(TimelineFilterType filterType) {
+      return filterType == TimelineFilterType.LIST ||
+          filterType == TimelineFilterType.KEY_VALUE;
+    }
+  },
+  INFO {
+    boolean isValidFilter(TimelineFilterType filterType) {
+      return filterType == TimelineFilterType.LIST ||
+          filterType == TimelineFilterType.KEY_VALUE;
+    }
+  },
+  METRIC {
+    boolean isValidFilter(TimelineFilterType filterType) {
+      return filterType == TimelineFilterType.LIST ||
+          filterType == TimelineFilterType.COMPARE;
+    }
+  },
+  EVENT {
+    boolean isValidFilter(TimelineFilterType filterType) {
+      return filterType == TimelineFilterType.LIST ||
+          filterType == TimelineFilterType.EXISTS;
+    }
+  },
+  IS_RELATED_TO {
+    boolean isValidFilter(TimelineFilterType filterType) {
+      return filterType == TimelineFilterType.LIST ||
+          filterType == TimelineFilterType.KEY_VALUES;
+    }
+  },
+  RELATES_TO {
+    boolean isValidFilter(TimelineFilterType filterType) {
+      return filterType == TimelineFilterType.LIST ||
+          filterType == TimelineFilterType.KEY_VALUES;
+    }
+  };
+
+  /**
+   * Checks whether filter type is valid for the filter being matched.
+   *
+   * @param filterType filter type.
+   * @return true, if its a valid filter, false otherwise.
+   */
+  abstract boolean isValidFilter(TimelineFilterType filterType);
+}
\ No newline at end of file


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


[33/50] [abbrv] hadoop git commit: YARN-5015. entire time series is returned for YARN container system metrics (CPU and memory) (Varun Saxena via sjlee)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc78a937/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 6ec59ea..fcab78c 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
@@ -217,7 +217,10 @@ public class TimelineReaderWebServices {
    *     flowrun id and app id which are extracted from UID and then used to
    *     query backend(Mandatory path param).
    * @param entityType Type of entities(Mandatory path param).
-   * @param limit Number of entities to return(Optional query param).
+   * @param limit If specified, defines the number of entities to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param createdTimeStart If specified, matched entities should not be
    *     created before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched entities should not be created
@@ -254,6 +257,13 @@ public class TimelineReaderWebServices {
    *     {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type, id and created time is returned
    *     (Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances of the given entity type
@@ -283,7 +293,8 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -308,7 +319,7 @@ public class TimelineReaderWebServices {
           limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
           infofilters, conffilters, metricfilters, eventfilters),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
     } catch (Exception e) {
       handleException(e, url, startTime,
           "createdTime start/end or limit or flowrunid");
@@ -342,7 +353,10 @@ public class TimelineReaderWebServices {
    *     query param).
    * @param flowRunId Run id which should match for the entities(Optional query
    *     param).
-   * @param limit Number of entities to return(Optional query param).
+   * @param limit If specified, defines the number of entities to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param createdTimeStart If specified, matched entities should not be
    *     created before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched entities should not be created
@@ -379,6 +393,13 @@ public class TimelineReaderWebServices {
    *     {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type, id, created time is returned
    *     (Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances of the given entity type
@@ -413,11 +434,12 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     return getEntities(req, res, null, appId, entityType, userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit);
   }
 
   /**
@@ -441,7 +463,10 @@ public class TimelineReaderWebServices {
    *     query param).
    * @param flowRunId Run id which should match for the entities(Optional query
    *     param).
-   * @param limit Number of entities to return(Optional query param).
+   * @param limit If specified, defines the number of entities to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param createdTimeStart If specified, matched entities should not be
    *     created before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched entities should not be created
@@ -478,6 +503,13 @@ public class TimelineReaderWebServices {
    *     {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type, id, created time is returned
    *     (Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances of the given entity type
@@ -513,7 +545,8 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -533,7 +566,7 @@ public class TimelineReaderWebServices {
           limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
           infofilters, conffilters, metricfilters, eventfilters),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
     } catch (Exception e) {
       handleException(e, url, startTime,
           "createdTime start/end or limit or flowrunid");
@@ -568,6 +601,13 @@ public class TimelineReaderWebServices {
    *     {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type, id, created time is returned
    *     (Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>TimelineEntity</cite> instance is returned.<br>
@@ -588,7 +628,8 @@ public class TimelineReaderWebServices {
       @PathParam("uid") String uId,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -608,7 +649,7 @@ public class TimelineReaderWebServices {
       }
       entity = timelineReaderManager.getEntity(context,
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -655,6 +696,13 @@ public class TimelineReaderWebServices {
    *     {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type, id, created time is returned
    *     (Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>TimelineEntity</cite> instance is returned.<br>
@@ -680,9 +728,11 @@ public class TimelineReaderWebServices {
       @QueryParam("flowrunid") String flowRunId,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     return getEntity(req, res, null, appId, entityType, entityId, userId,
-        flowName, flowRunId, confsToRetrieve, metricsToRetrieve, fields);
+        flowName, flowRunId, confsToRetrieve, metricsToRetrieve, fields,
+        metricsLimit);
   }
 
   /**
@@ -717,6 +767,13 @@ public class TimelineReaderWebServices {
    *     {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type, id and created time is returned
    *     (Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>TimelineEntity</cite> instance is returned.<br>
@@ -743,7 +800,8 @@ public class TimelineReaderWebServices {
       @QueryParam("flowrunid") String flowRunId,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -760,7 +818,7 @@ public class TimelineReaderWebServices {
           TimelineReaderWebServicesUtils.createTimelineReaderContext(
           clusterId, userId, flowName, flowRunId, appId, entityType, entityId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -827,7 +885,7 @@ public class TimelineReaderWebServices {
       context.setEntityType(TimelineEntityType.YARN_FLOW_RUN.toString());
       entity = timelineReaderManager.getEntity(context,
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, metricsToRetrieve, null));
+          null, metricsToRetrieve, null, null));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -936,7 +994,7 @@ public class TimelineReaderWebServices {
           clusterId, userId, flowName, flowRunId, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, metricsToRetrieve, null));
+          null, metricsToRetrieve, null, null));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -963,7 +1021,10 @@ public class TimelineReaderWebServices {
    * @param uId a delimited string containing clusterid, userid, and flow name
    *     which are extracted from UID and then used to query backend(Mandatory
    *     path param).
-   * @param limit Number of flow runs to return(Optional query param).
+   * @param limit If specified, defines the number of flow runs to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param createdTimeStart If specified, matched flow runs should not be
    *     created before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched flow runs should not be created
@@ -972,10 +1033,11 @@ public class TimelineReaderWebServices {
    *     and send back in response. These metrics will be retrieved
    *     irrespective of whether metrics are specified in fields to retrieve or
    *     not.
-   * @param fields Specifies which fields to retrieve, see {@link Field}.
-   *     All fields will be retrieved if fields=ALL. Fields other than METRICS
-   *     have no meaning for this REST endpoint. If not specified, all fields
-   *     other than metrics are returned(Optional query param).
+   * @param fields Specifies which fields to retrieve, see {@link Field}. All
+   *     fields will be retrieved if fields=ALL. Amongst all the fields, only
+   *     METRICS makes sense for flow runs hence only ALL or METRICS are
+   *     supported as fields for fetching flow runs. Other fields will lead to
+   *     HTTP 400 (Bad Request) response. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     set of <cite>FlowRunEntity</cite> instances for the given flow are
@@ -1021,7 +1083,7 @@ public class TimelineReaderWebServices {
           limit, createdTimeStart, createdTimeEnd, null, null, null,
           null, null, null),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, metricsToRetrieve, fields));
+          null, metricsToRetrieve, fields, null));
     } catch (Exception e) {
       handleException(e, url, startTime, "createdTime start/end or limit");
     }
@@ -1044,7 +1106,10 @@ public class TimelineReaderWebServices {
    *     Mandatory path param)
    * @param flowName Flow name to which the flow runs to be queried belongs to(
    *     Mandatory path param).
-   * @param limit Number of flow runs to return(Optional query param).
+   * @param limit If specified, defines the number of flow runs to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param createdTimeStart If specified, matched flow runs should not be
    *     created before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched flow runs should not be created
@@ -1053,10 +1118,11 @@ public class TimelineReaderWebServices {
    *     and send back in response. These metrics will be retrieved
    *     irrespective of whether metrics are specified in fields to retrieve or
    *     not.
-   * @param fields Specifies which fields to retrieve, see {@link Field}.
-   *     All fields will be retrieved if fields=ALL. Fields other than METRICS
-   *     have no meaning for this REST endpoint. If not specified, all fields
-   *     other than metrics are returned(Optional query param).
+   * @param fields Specifies which fields to retrieve, see {@link Field}. All
+   *     fields will be retrieved if fields=ALL. Amongst all the fields, only
+   *     METRICS makes sense for flow runs hence only ALL or METRICS are
+   *     supported as fields for fetching flow runs. Other fields will lead to
+   *     HTTP 400 (Bad Request) response. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     set of <cite>FlowRunEntity</cite> instances for the given flow are
@@ -1095,7 +1161,10 @@ public class TimelineReaderWebServices {
    *     Mandatory path param)
    * @param flowName Flow name to which the flow runs to be queried belongs to(
    *     Mandatory path param).
-   * @param limit Number of flow runs to return(Optional query param).
+   * @param limit If specified, defines the number of flow runs to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param createdTimeStart If specified, matched flow runs should not be
    *     created before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched flow runs should not be created
@@ -1104,10 +1173,11 @@ public class TimelineReaderWebServices {
    *     and send back in response. These metrics will be retrieved
    *     irrespective of whether metrics are specified in fields to retrieve or
    *     not.
-   * @param fields Specifies which fields to retrieve, see {@link Field}.
-   *     All fields will be retrieved if fields=ALL. Fields other than METRICS
-   *     have no meaning for this REST endpoint. If not specified, all fields
-   *     other than metrics are returned(Optional query param).
+   * @param fields Specifies which fields to retrieve, see {@link Field}. All
+   *     fields will be retrieved if fields=ALL. Amongst all the fields, only
+   *     METRICS makes sense for flow runs hence only ALL or METRICS are
+   *     supported as fields for fetching flow runs. Other fields will lead to
+   *     HTTP 400 (Bad Request) response. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     set of <cite>FlowRunEntity</cite> instances for the given flow are
@@ -1152,7 +1222,7 @@ public class TimelineReaderWebServices {
           limit, createdTimeStart, createdTimeEnd, null, null, null,
           null, null, null),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, metricsToRetrieve, fields));
+          null, metricsToRetrieve, fields, null));
     } catch (Exception e) {
       handleException(e, url, startTime, "createdTime start/end or limit");
     }
@@ -1171,7 +1241,10 @@ public class TimelineReaderWebServices {
    *
    * @param req Servlet request.
    * @param res Servlet response.
-   * @param limit Number of flows to return(Optional query param).
+   * @param limit If specified, defines the number of flows to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param 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(Optional
@@ -1214,7 +1287,10 @@ public class TimelineReaderWebServices {
    * @param res Servlet response.
    * @param clusterId Cluster id to which the flows to be queried belong to(
    *     Mandatory path param).
-   * @param limit Number of flows to return(Optional query param).
+   * @param limit If specified, defines the number of flows to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param 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(Optional
@@ -1271,7 +1347,7 @@ public class TimelineReaderWebServices {
           clusterId, null, null, null, null,
           TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
           entityFilters, TimelineReaderWebServicesUtils.
-          createTimelineDataToRetrieve(null, null, null));
+          createTimelineDataToRetrieve(null, null, null, null));
     } catch (Exception e) {
       handleException(e, url, startTime, "limit");
     }
@@ -1305,6 +1381,13 @@ public class TimelineReaderWebServices {
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
    *     app id and app created time is returned(Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>TimelineEntity</cite> instance is returned.<br>
@@ -1325,7 +1408,8 @@ public class TimelineReaderWebServices {
       @PathParam("uid") String uId,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -1346,7 +1430,7 @@ public class TimelineReaderWebServices {
       context.setEntityType(TimelineEntityType.YARN_APPLICATION.toString());
       entity = timelineReaderManager.getEntity(context,
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -1388,6 +1472,13 @@ public class TimelineReaderWebServices {
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
    *     app id and app created time is returned(Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>TimelineEntity</cite> instance is returned.<br>
@@ -1411,9 +1502,10 @@ public class TimelineReaderWebServices {
       @QueryParam("userid") String userId,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     return getApp(req, res, null, appId, flowName, flowRunId, userId,
-        confsToRetrieve, metricsToRetrieve, fields);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit);
   }
 
   /**
@@ -1444,6 +1536,13 @@ public class TimelineReaderWebServices {
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
    *     app id and app created time is returned(Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>TimelineEntity</cite> instance is returned.<br>
@@ -1468,7 +1567,8 @@ public class TimelineReaderWebServices {
       @QueryParam("userid") String userId,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -1486,7 +1586,7 @@ public class TimelineReaderWebServices {
           clusterId, userId, flowName, flowRunId, appId,
           TimelineEntityType.YARN_APPLICATION.toString(), null),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -1512,7 +1612,10 @@ public class TimelineReaderWebServices {
    * @param uId a delimited string containing clusterid, userid, flow name and
    *     flowrun id which are extracted from UID and then used to query backend.
    *     (Mandatory path param).
-   * @param limit Number of apps to return(Optional query param).
+   * @param limit If specified, defines the number of apps to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param createdTimeStart If specified, matched apps should not be created
    *     before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched apps should not be created
@@ -1549,6 +1652,13 @@ public class TimelineReaderWebServices {
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
    *     app id and app created time is returned(Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -1577,7 +1687,8 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -1601,7 +1712,7 @@ public class TimelineReaderWebServices {
           limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
           infofilters, conffilters, metricfilters, eventfilters),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          confsToRetrieve, metricsToRetrieve, fields));
+          confsToRetrieve, metricsToRetrieve, fields, metricsLimit));
     } catch (Exception e) {
       handleException(e, url, startTime,
           "createdTime start/end or limit or flowrunid");
@@ -1628,7 +1739,10 @@ public class TimelineReaderWebServices {
    *     param).
    * @param flowRunId Run id which should match for the apps(Mandatory path
    *     param).
-   * @param limit Number of apps to return(Optional query param).
+   * @param limit If specified, defines the number of apps to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param createdTimeStart If specified, matched apps should not be created
    *     before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched apps should not be created
@@ -1665,6 +1779,13 @@ public class TimelineReaderWebServices {
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
    *     app id and app created time is returned(Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -1695,12 +1816,13 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     return getEntities(req, res, null, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit);
   }
 
   /**
@@ -1717,7 +1839,10 @@ public class TimelineReaderWebServices {
    *     param).
    * @param flowRunId Run id which should match for the apps(Mandatory path
    *     param).
-   * @param limit Number of apps to return(Optional query param).
+   * @param limit If specified, defines the number of apps to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param createdTimeStart If specified, matched apps should not be created
    *     before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched apps should not be created
@@ -1754,6 +1879,13 @@ public class TimelineReaderWebServices {
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
    *     app id and app created time is returned(Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -1786,12 +1918,13 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     return getEntities(req, res, clusterId, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit);
   }
 
   /**
@@ -1805,7 +1938,10 @@ public class TimelineReaderWebServices {
    * @param userId User id which should match for the apps(Mandatory path param)
    * @param flowName Flow name which should match for the apps(Mandatory path
    *     param).
-   * @param limit Number of apps to return(Optional query param).
+   * @param limit If specified, defines the number of apps to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param createdTimeStart If specified, matched apps should not be created
    *     before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched apps should not be created
@@ -1842,6 +1978,13 @@ public class TimelineReaderWebServices {
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
    *     app id and app created time is returned(Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -1871,19 +2014,19 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     return getEntities(req, res, null, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         null, limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
         infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit);
   }
 
   /**
    * Return a list of apps for a given user, cluster id and flow name. If number
    * of matching apps are more than the limit, most recent apps till the limit
-   * is reached, will be returned. If number of matching apps are more than the
-   * limit, most recent apps till the limit is reached, will be returned.
+   * is reached, will be returned.
    *
    * @param req Servlet request.
    * @param res Servlet response.
@@ -1892,7 +2035,10 @@ public class TimelineReaderWebServices {
    * @param userId User id which should match for the apps(Mandatory path param)
    * @param flowName Flow name which should match for the apps(Mandatory path
    *     param).
-   * @param limit Number of apps to return(Optional query param).
+   * @param limit If specified, defines the number of apps to return. The
+   *     maximum possible value for limit can be {@link Long#MAX_VALUE}. If it
+   *     is not specified or has a value less than 0, then limit will be
+   *     considered as 100. (Optional query param).
    * @param createdTimeStart If specified, matched apps should not be created
    *     before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched apps should not be created
@@ -1929,6 +2075,13 @@ public class TimelineReaderWebServices {
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
    *     app id and app created time is returned(Optional query param).
+   * @param metricsLimit If specified, defines the number of metrics to return.
+   *     Considered only if fields contains METRICS/ALL or metricsToRetrieve is
+   *     specified. Ignored otherwise. The maximum possible value for
+   *     metricsLimit can be {@link Integer#MAX_VALUE}. If it is not specified
+   *     or has a value less than 1, and metrics have to be retrieved, then
+   *     metricsLimit will be considered as 1 i.e. latest single value of
+   *     metric(s) will be returned. (Optional query param).
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing
    *     a set of <cite>TimelineEntity</cite> instances representing apps is
@@ -1959,11 +2112,12 @@ public class TimelineReaderWebServices {
       @QueryParam("eventfilters") String eventfilters,
       @QueryParam("confstoretrieve") String confsToRetrieve,
       @QueryParam("metricstoretrieve") String metricsToRetrieve,
-      @QueryParam("fields") String fields) {
+      @QueryParam("fields") String fields,
+      @QueryParam("metricslimit") String metricsLimit) {
     return getEntities(req, res, clusterId, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         null, limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
         infofilters, conffilters, metricfilters, eventfilters,
-        confsToRetrieve, metricsToRetrieve, fields);
+        confsToRetrieve, metricsToRetrieve, fields, metricsLimit);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc78a937/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
index 2e667d6..7fc8cb8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
@@ -87,14 +87,16 @@ final class TimelineReaderWebServicesUtils {
    * @param confs confs to retrieve.
    * @param metrics metrics to retrieve.
    * @param fields fields to retrieve.
+   * @param metricsLimit upper limit on number of metrics to return.
    * @return a {@link TimelineDataToRetrieve} object.
    * @throws TimelineParseException if any problem occurs during parsing.
    */
   static TimelineDataToRetrieve createTimelineDataToRetrieve(String confs,
-      String metrics, String fields) throws TimelineParseException {
+      String metrics, String fields, String metricsLimit)
+      throws TimelineParseException {
     return new TimelineDataToRetrieve(parseDataToRetrieve(confs),
-        parseDataToRetrieve(metrics), parseFieldsStr(
-            fields, TimelineParseConstants.COMMA_DELIMITER));
+        parseDataToRetrieve(metrics), parseFieldsStr(fields,
+        TimelineParseConstants.COMMA_DELIMITER), parseIntStr(metricsLimit));
   }
 
   /**
@@ -191,6 +193,15 @@ final class TimelineReaderWebServicesUtils {
   }
 
   /**
+   * Interpret passed string as a integer.
+   * @param str Passed string.
+   * @return integer representation if string is not null, null otherwise.
+   */
+  static Integer parseIntStr(String str) {
+    return str == null ? null : Integer.parseInt(str.trim());
+  }
+
+  /**
    * Trims the passed string if its not null.
    * @param str Passed string.
    * @return trimmed string if string is not null, null otherwise.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc78a937/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
index 71c90fb..8e806bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
@@ -312,7 +312,7 @@ class ApplicationEntityReader extends GenericEntityReader {
         ApplicationRowKey.getRowKey(context.getClusterId(), context.getUserId(),
             context.getFlowName(), context.getFlowRunId(), context.getAppId());
     Get get = new Get(rowKey);
-    get.setMaxVersions(Integer.MAX_VALUE);
+    get.setMaxVersions(getDataToRetrieve().getMetricsLimit());
     if (filterList != null && !filterList.getFilters().isEmpty()) {
       get.setFilter(filterList);
     }
@@ -382,6 +382,7 @@ class ApplicationEntityReader extends GenericEntityReader {
       newList.addFilter(filterList);
     }
     scan.setFilter(newList);
+    scan.setMaxVersions(getDataToRetrieve().getMetricsLimit());
     return getTable().getResultScanner(hbaseConf, conn, scan);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc78a937/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
index b2de2e5..e1695ef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
 
 import java.io.IOException;
+import java.util.EnumSet;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
@@ -48,6 +49,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnF
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
+import org.apache.hadoop.yarn.webapp.BadRequestException;
 
 import com.google.common.base.Preconditions;
 
@@ -91,6 +93,15 @@ class FlowRunEntityReader extends TimelineEntityReader {
       Preconditions.checkNotNull(getContext().getFlowRunId(),
           "flowRunId shouldn't be null");
     }
+    EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
+    if (!isSingleEntityRead() && fieldsToRetrieve != null) {
+      for (Field field : fieldsToRetrieve) {
+        if (field != Field.ALL && field != Field.METRICS) {
+          throw new BadRequestException("Invalid field " + field +
+              " specified while querying flow runs.");
+        }
+      }
+    }
   }
 
   @Override
@@ -209,6 +220,7 @@ class FlowRunEntityReader extends TimelineEntityReader {
       newList.addFilter(filterList);
     }
     scan.setFilter(newList);
+    scan.setMaxVersions(Integer.MAX_VALUE);
     return getTable().getResultScanner(hbaseConf, conn, scan);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc78a937/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
index 878695c..22583b5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/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
@@ -489,7 +489,7 @@ class GenericEntityReader extends TimelineEntityReader {
             context.getFlowName(), context.getFlowRunId(), context.getAppId(),
             context.getEntityType(), context.getEntityId());
     Get get = new Get(rowKey);
-    get.setMaxVersions(Integer.MAX_VALUE);
+    get.setMaxVersions(getDataToRetrieve().getMetricsLimit());
     if (filterList != null && !filterList.getFilters().isEmpty()) {
       get.setFilter(filterList);
     }
@@ -506,7 +506,7 @@ class GenericEntityReader extends TimelineEntityReader {
     scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
         context.getClusterId(), context.getUserId(), context.getFlowName(),
         context.getFlowRunId(), context.getAppId(), context.getEntityType()));
-    scan.setMaxVersions(Integer.MAX_VALUE);
+    scan.setMaxVersions(getDataToRetrieve().getMetricsLimit());
     if (filterList != null && !filterList.getFilters().isEmpty()) {
       scan.setFilter(filterList);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc78a937/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
index be27643..852834e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/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
@@ -338,8 +338,9 @@ public abstract class TimelineEntityReader {
       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);
+      TimelineMetric.Type metricType = metricResult.getValue().size() > 1 ?
+          TimelineMetric.Type.TIME_SERIES : TimelineMetric.Type.SINGLE_VALUE;
+      metric.setType(metricType);
       metric.addValues(metricResult.getValue());
       entity.addMetric(metric);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc78a937/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 ad2122c..2af7817 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
@@ -274,7 +274,7 @@ public class TestFileSystemTimelineReaderImpl {
     TimelineEntity result = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", "id_1"),
-        new TimelineDataToRetrieve(null, null, null));
+        new TimelineDataToRetrieve(null, null, null, null));
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
@@ -289,7 +289,7 @@ public class TestFileSystemTimelineReaderImpl {
     TimelineEntity result = reader.getEntity(
         new TimelineReaderContext("cluster1", null, null, null, "app1", "app",
         "id_1"),
-        new TimelineDataToRetrieve(null, null, null));
+        new TimelineDataToRetrieve(null, null, null, null));
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
@@ -306,7 +306,7 @@ public class TestFileSystemTimelineReaderImpl {
     TimelineEntity result = reader.getEntity(
         new TimelineReaderContext("cluster1", null, null, null, "app2",
         "app", "id_5"),
-        new TimelineDataToRetrieve(null, null, null));
+        new TimelineDataToRetrieve(null, null, null, null));
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_5")).toString(),
         result.getIdentifier().toString());
@@ -320,7 +320,7 @@ public class TestFileSystemTimelineReaderImpl {
         new TimelineReaderContext("cluster1","user1", "flow1", 1L, "app1",
         "app", "id_1"),
         new TimelineDataToRetrieve(null, null,
-        EnumSet.of(Field.INFO, Field.CONFIGS, Field.METRICS)));
+        EnumSet.of(Field.INFO, Field.CONFIGS, Field.METRICS), null));
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
@@ -338,7 +338,7 @@ public class TestFileSystemTimelineReaderImpl {
     TimelineEntity result = reader.getEntity(
         new TimelineReaderContext("cluster1","user1", "flow1", 1L, "app1",
         "app", "id_1"),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
@@ -354,7 +354,7 @@ public class TestFileSystemTimelineReaderImpl {
     Set<TimelineEntity> result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null), new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     // All 4 entities will be returned
     Assert.assertEquals(4, result.size());
   }


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


[08/50] [abbrv] hadoop git commit: YARN-3816. [Aggregation] App-level aggregation and accumulation for YARN system metrics (Li Lu via sjlee)

Posted by vr...@apache.org.
YARN-3816. [Aggregation] App-level aggregation and accumulation for YARN system metrics (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/78ffdf08
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/78ffdf08
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/78ffdf08

Branch: refs/heads/YARN-2928
Commit: 78ffdf087a787aef6a105a98094816ac7f8a4889
Parents: a5bf4fa
Author: Sangjin Lee <sj...@apache.org>
Authored: Fri Apr 22 10:24:40 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:06 2016 -0700

----------------------------------------------------------------------
 .../records/timelineservice/TimelineMetric.java | 140 ++++++++++--
 .../TimelineMetricCalculator.java               | 115 ++++++++++
 .../TimelineMetricOperation.java                | 167 +++++++++++++++
 .../timelineservice/TestTimelineMetric.java     | 100 +++++++++
 .../TestTimelineServiceRecords.java             |   6 +-
 .../timelineservice/NMTimelinePublisher.java    |   4 +
 .../collector/AppLevelTimelineCollector.java    |  72 +++++++
 .../collector/TimelineCollector.java            | 213 ++++++++++++++++++-
 .../storage/TimelineAggregationTrack.java       |   2 +-
 .../collector/TestTimelineCollector.java        | 127 +++++++++++
 .../TestFileSystemTimelineWriterImpl.java       |  43 +++-
 .../storage/TestHBaseTimelineStorage.java       |  35 ++-
 12 files changed, 998 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/78ffdf08/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 2f60515..f0c6849 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
@@ -19,12 +19,13 @@ package org.apache.hadoop.yarn.api.records.timelineservice;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
-import java.util.Comparator;
+import java.util.Collections;
 import java.util.Map;
 import java.util.TreeMap;
 
@@ -48,13 +49,13 @@ public class TimelineMetric {
 
   private Type type;
   private String id;
-  private Comparator<Long> reverseComparator = new Comparator<Long>() {
-    @Override
-    public int compare(Long l1, Long l2) {
-      return l2.compareTo(l1);
-    }
-  };
-  private TreeMap<Long, Number> values = new TreeMap<>(reverseComparator);
+  // By default, not to do any aggregation operations. This field will NOT be
+  // persisted (like a "transient" member).
+  private TimelineMetricOperation realtimeAggregationOp
+      = TimelineMetricOperation.NOP;
+
+  private TreeMap<Long, Number> values
+      = new TreeMap<>(Collections.reverseOrder());
 
   public TimelineMetric() {
     this(Type.SINGLE_VALUE);
@@ -83,6 +84,26 @@ public class TimelineMetric {
     this.id = metricId;
   }
 
+  /**
+   * Get the real time aggregation operation of this metric.
+   *
+   * @return Real time aggregation operation
+   */
+  public TimelineMetricOperation getRealtimeAggregationOp() {
+    return realtimeAggregationOp;
+  }
+
+  /**
+   * Set the real time aggregation operation of this metric.
+   *
+   * @param op A timeline metric operation that the metric should perform on
+   *           real time aggregations
+   */
+  public void setRealtimeAggregationOp(
+      final TimelineMetricOperation op) {
+    this.realtimeAggregationOp = op;
+  }
+
   // required by JAXB
   @InterfaceAudience.Private
   @XmlElement(name = "values")
@@ -98,8 +119,8 @@ public class TimelineMetric {
     if (type == Type.SINGLE_VALUE) {
       overwrite(vals);
     } else {
-      if (values != null) {
-        this.values = new TreeMap<Long, Number>(reverseComparator);
+      if (vals != null) {
+        this.values = new TreeMap<>(Collections.reverseOrder());
         this.values.putAll(vals);
       } else {
         this.values = null;
@@ -166,11 +187,100 @@ public class TimelineMetric {
 
   @Override
   public String toString() {
-    String str = "{id:" + id + ", type:" + type;
-    if (!values.isEmpty()) {
-      str += ", values:" + values;
+    return "{id: " + id + ", type: " + type +
+        ", realtimeAggregationOp: " +
+        realtimeAggregationOp + "; " + values.toString() +
+        "}";
+  }
+
+  /**
+   * Get the latest timeline metric as single value type.
+   *
+   * @param metric Incoming timeline metric
+   * @return The latest metric in the incoming metric
+   */
+  public static TimelineMetric getLatestSingleValueMetric(
+      TimelineMetric metric) {
+    if (metric.getType() == Type.SINGLE_VALUE) {
+      return metric;
+    } else {
+      TimelineMetric singleValueMetric = new TimelineMetric(Type.SINGLE_VALUE);
+      Long firstKey = metric.values.firstKey();
+      if (firstKey != null) {
+        Number firstValue = metric.values.get(firstKey);
+        singleValueMetric.addValue(firstKey, firstValue);
+      }
+      return singleValueMetric;
     }
-    str += "}";
-    return str;
   }
+
+  /**
+   * Get single data timestamp of the metric.
+   *
+   * @return the single data timestamp
+   */
+  public long getSingleDataTimestamp() {
+    if (this.type == Type.SINGLE_VALUE) {
+      if (values.size() == 0) {
+        throw new YarnRuntimeException("Values for this timeline metric is " +
+            "empty.");
+      } else {
+        return values.firstKey();
+      }
+    } else {
+      throw new YarnRuntimeException("Type for this timeline metric is not " +
+          "SINGLE_VALUE.");
+    }
+  }
+
+  /**
+   * Get single data value of the metric.
+   *
+   * @return the single data value
+   */
+  public Number getSingleDataValue() {
+    if (this.type == Type.SINGLE_VALUE) {
+      if (values.size() == 0) {
+        return null;
+      } else {
+        return values.get(values.firstKey());
+      }
+    } else {
+      throw new YarnRuntimeException("Type for this timeline metric is not " +
+          "SINGLE_VALUE.");
+    }
+  }
+
+  /**
+   * Aggregate an incoming metric to the base aggregated metric with the given
+   * operation state in a stateless fashion. The assumption here is
+   * baseAggregatedMetric and latestMetric should be single value data if not
+   * null.
+   *
+   * @param incomingMetric Incoming timeline metric to aggregate
+   * @param baseAggregatedMetric Base timeline metric
+   * @return Result metric after aggregation
+   */
+  public static TimelineMetric aggregateTo(TimelineMetric incomingMetric,
+      TimelineMetric baseAggregatedMetric) {
+    return aggregateTo(incomingMetric, baseAggregatedMetric, null);
+  }
+
+  /**
+   * Aggregate an incoming metric to the base aggregated metric with the given
+   * operation state. The assumption here is baseAggregatedMetric and
+   * latestMetric should be single value data if not null.
+   *
+   * @param incomingMetric Incoming timeline metric to aggregate
+   * @param baseAggregatedMetric Base timeline metric
+   * @param state Operation state
+   * @return Result metric after aggregation
+   */
+  public static TimelineMetric aggregateTo(TimelineMetric incomingMetric,
+      TimelineMetric baseAggregatedMetric, Map<Object, Object> state) {
+    TimelineMetricOperation operation
+        = incomingMetric.getRealtimeAggregationOp();
+    return operation.aggregate(incomingMetric, baseAggregatedMetric, state);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78ffdf08/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricCalculator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricCalculator.java
new file mode 100644
index 0000000..4c9045f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricCalculator.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.api.records.timelineservice;
+
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+
+/**
+ * A calculator for timeline metrics.
+ */
+public final class TimelineMetricCalculator {
+
+  private TimelineMetricCalculator() {
+    // do nothing.
+  }
+
+  /**
+   * Compare two not-null numbers.
+   * @param n1 Number n1
+   * @param n2 Number n2
+   * @return 0 if n1 equals n2, a negative int if n1 is less than n2, a
+   * positive int otherwise.
+   */
+  public static int compare(Number n1, Number n2) {
+    if (n1 == null || n2 == null) {
+      throw new YarnRuntimeException(
+          "Number to be compared shouldn't be null.");
+    }
+
+    if (n1 instanceof Integer || n1 instanceof Long) {
+      if (n1.longValue() == n2.longValue()) {
+        return 0;
+      } else {
+        return (n1.longValue() < n2.longValue()) ? -1 : 1;
+      }
+    }
+
+    if (n1 instanceof Float || n1 instanceof Double) {
+      if (n1.doubleValue() == n2.doubleValue()) {
+        return 0;
+      } else {
+        return (n1.doubleValue() < n2.doubleValue()) ? -1 : 1;
+      }
+    }
+
+    // TODO throw warnings/exceptions for other types of number.
+    throw new YarnRuntimeException("Unsupported types for number comparison: "
+        + n1.getClass().getName() + ", " + n2.getClass().getName());
+  }
+
+  /**
+   * Subtract operation between two Numbers.
+   * @param n1 Number n1
+   * @param n2 Number n2
+   * @return Number represent to (n1 - n2).
+   */
+  public static Number sub(Number n1, Number n2) {
+    if (n1 == null) {
+      throw new YarnRuntimeException(
+          "Number to be subtracted shouldn't be null.");
+    } else if (n2 == null) {
+      return n1;
+    }
+
+    if (n1 instanceof Integer || n1 instanceof Long) {
+      return n1.longValue() - n2.longValue();
+    }
+
+    if (n1 instanceof Float || n1 instanceof Double) {
+      return n1.doubleValue() - n2.doubleValue();
+    }
+
+    // TODO throw warnings/exceptions for other types of number.
+    return null;
+  }
+
+  /**
+   * Sum up two Numbers.
+   * @param n1 Number n1
+   * @param n2 Number n2
+   * @return Number represent to (n1 + n2).
+   */
+  public static Number sum(Number n1, Number n2) {
+    if (n1 == null) {
+      return n2;
+    } else if (n2 == null) {
+      return n1;
+    }
+
+    if (n1 instanceof Integer || n1 instanceof Long) {
+      return n1.longValue() + n2.longValue();
+    }
+
+    if (n1 instanceof Float || n1 instanceof Double) {
+      return n1.doubleValue() + n2.doubleValue();
+    }
+
+    // TODO throw warnings/exceptions for other types of number.
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78ffdf08/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricOperation.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricOperation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricOperation.java
new file mode 100644
index 0000000..58e5c38
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timelineservice/TimelineMetricOperation.java
@@ -0,0 +1,167 @@
+/*
+ * 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.Map;
+
+/**
+ * Aggregation operations.
+ */
+public enum TimelineMetricOperation {
+  NOP("NOP") {
+    /**
+     * Do nothing on the base metric.
+     *
+     * @param incoming Metric a
+     * @param base Metric b
+     * @param state Operation state (not used)
+     * @return Metric b
+     */
+    @Override
+    public TimelineMetric exec(TimelineMetric incoming,
+        TimelineMetric base, Map<Object, Object> state) {
+      return base;
+    }
+  },
+  MAX("MAX") {
+    /**
+     * Keep the greater value of incoming and base. Stateless operation.
+     *
+     * @param incoming Metric a
+     * @param base Metric b
+     * @param state Operation state (not used)
+     * @return the greater value of a and b
+     */
+    @Override
+    public TimelineMetric exec(TimelineMetric incoming,
+        TimelineMetric base, Map<Object, Object> state) {
+      if (base == null) {
+        return incoming;
+      }
+      Number incomingValue = incoming.getSingleDataValue();
+      Number aggregateValue = base.getSingleDataValue();
+      if (aggregateValue == null) {
+        aggregateValue = Long.MIN_VALUE;
+      }
+      if (TimelineMetricCalculator.compare(incomingValue, aggregateValue) > 0) {
+        base.addValue(incoming.getSingleDataTimestamp(), incomingValue);
+      }
+      return base;
+    }
+  },
+  REPLACE("REPLACE") {
+    /**
+     * Replace the base metric with the incoming value. Stateless operation.
+     *
+     * @param incoming Metric a
+     * @param base Metric b
+     * @param state Operation state (not used)
+     * @return Metric a
+     */
+    @Override
+    public TimelineMetric exec(TimelineMetric incoming,
+        TimelineMetric base,
+        Map<Object, Object> state) {
+      return incoming;
+    }
+  },
+  SUM("SUM") {
+    /**
+     * Return the sum of the incoming metric and the base metric if the
+     * operation is stateless. For stateful operations, also subtract the
+     * value of the timeline metric mapped to the PREV_METRIC_STATE_KEY
+     * in the state object.
+     *
+     * @param incoming Metric a
+     * @param base Metric b
+     * @param state Operation state (PREV_METRIC_STATE_KEY's value as Metric p)
+     * @return A metric with value a + b - p
+     */
+    @Override
+    public TimelineMetric exec(TimelineMetric incoming, TimelineMetric base,
+        Map<Object, Object> state) {
+      if (base == null) {
+        return incoming;
+      }
+      Number incomingValue = incoming.getSingleDataValue();
+      Number aggregateValue = base.getSingleDataValue();
+      Number result
+          = TimelineMetricCalculator.sum(incomingValue, aggregateValue);
+
+      // If there are previous value in the state, we will take it off from the
+      // sum
+      if (state != null) {
+        Object prevMetric = state.get(PREV_METRIC_STATE_KEY);
+        if (prevMetric instanceof TimelineMetric) {
+          result = TimelineMetricCalculator.sub(result,
+              ((TimelineMetric) prevMetric).getSingleDataValue());
+        }
+      }
+      base.addValue(incoming.getSingleDataTimestamp(), result);
+      return base;
+    }
+  },
+  AVG("AVERAGE") {
+    /**
+     * Return the average value of the incoming metric and the base metric,
+     * with a given state. Not supported yet.
+     *
+     * @param incoming Metric a
+     * @param base Metric b
+     * @param state Operation state
+     * @return Not finished yet
+     */
+    @Override
+    public TimelineMetric exec(TimelineMetric incoming, TimelineMetric base,
+        Map<Object, Object> state) {
+      // Not supported yet
+      throw new UnsupportedOperationException(
+          "Unsupported aggregation operation: AVERAGE");
+    }
+  };
+
+  public static final String PREV_METRIC_STATE_KEY = "PREV_METRIC";
+
+  /**
+   * Perform the aggregation operation.
+   *
+   * @param incoming Incoming metric
+   * @param aggregate Base aggregation metric
+   * @param state Operation state
+   * @return Result metric for this aggregation operation
+   */
+  public TimelineMetric aggregate(TimelineMetric incoming,
+      TimelineMetric aggregate, Map<Object, Object> state) {
+    return exec(incoming, aggregate, state);
+  }
+
+  private final String opName;
+
+  TimelineMetricOperation(String opString) {
+    opName = opString;
+  }
+
+  @Override
+  public String toString() {
+    return this.opName;
+  }
+
+  abstract TimelineMetric exec(TimelineMetric incoming, TimelineMetric base,
+      Map<Object, Object> state);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78ffdf08/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineMetric.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineMetric.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineMetric.java
new file mode 100644
index 0000000..3244bc3
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/test/java/org/apache/hadoop/yarn/api/records/timelineservice/TestTimelineMetric.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.api.records.timelineservice;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric.Type;
+
+import org.junit.Test;
+
+public class TestTimelineMetric {
+
+  @Test
+  public void testTimelineMetricAggregation() {
+    long ts = System.currentTimeMillis();
+    // single_value metric add against null metric
+    TimelineMetric m1 = getSingleValueMetric("MEGA_BYTES_MILLIS",
+        TimelineMetricOperation.SUM, ts, 10000L);
+    TimelineMetric aggregatedMetric = TimelineMetric.aggregateTo(m1, null);
+    assertEquals(10000L, aggregatedMetric.getSingleDataValue());
+
+    TimelineMetric m2 = getSingleValueMetric("MEGA_BYTES_MILLIS",
+        TimelineMetricOperation.SUM, ts, 20000L);
+    aggregatedMetric = TimelineMetric.aggregateTo(m2, aggregatedMetric);
+    assertEquals(30000L, aggregatedMetric.getSingleDataValue());
+
+    // stateful sum test
+    Map<Object, Object> state = new HashMap<>();
+    state.put(TimelineMetricOperation.PREV_METRIC_STATE_KEY, m2);
+    TimelineMetric m2New = getSingleValueMetric("MEGA_BYTES_MILLIS",
+        TimelineMetricOperation.SUM, ts, 10000L);
+    aggregatedMetric = TimelineMetric.aggregateTo(m2New, aggregatedMetric,
+        state);
+    assertEquals(20000L, aggregatedMetric.getSingleDataValue());
+
+    // single_value metric max against single_value metric
+    TimelineMetric m3 = getSingleValueMetric("TRANSFER_RATE",
+        TimelineMetricOperation.MAX, ts, 150L);
+    TimelineMetric aggregatedMax = TimelineMetric.aggregateTo(m3, null);
+    assertEquals(150L, aggregatedMax.getSingleDataValue());
+
+    TimelineMetric m4 = getSingleValueMetric("TRANSFER_RATE",
+        TimelineMetricOperation.MAX, ts, 170L);
+    aggregatedMax = TimelineMetric.aggregateTo(m4, aggregatedMax);
+    assertEquals(170L, aggregatedMax.getSingleDataValue());
+
+    // single_value metric avg against single_value metric
+    TimelineMetric m5 = getSingleValueMetric("TRANSFER_RATE",
+        TimelineMetricOperation.AVG, ts, 150L);
+    try {
+      TimelineMetric.aggregateTo(m5, null);
+      fail("Taking average among metrics is not supported! ");
+    } catch (UnsupportedOperationException e) {
+      // Expected
+    }
+
+  }
+
+  private static TimelineMetric getSingleValueMetric(String id,
+      TimelineMetricOperation op, long timestamp, long value) {
+    TimelineMetric m = new TimelineMetric();
+    m.setId(id);
+    m.setType(Type.SINGLE_VALUE);
+    m.setRealtimeAggregationOp(op);
+    Map<Long, Number> metricValues = new HashMap<Long, Number>();
+    metricValues.put(timestamp, value);
+    m.setValues(metricValues);
+    return m;
+  }
+
+  private static TimelineMetric getTimeSeriesMetric(String id,
+      TimelineMetricOperation op, Map<Long, Number> metricValues) {
+    TimelineMetric m = new TimelineMetric();
+    m.setId(id);
+    m.setType(Type.TIME_SERIES);
+    m.setRealtimeAggregationOp(op);
+    m.setValues(metricValues);
+    return m;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78ffdf08/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 51ec762..592bfa3 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
@@ -64,13 +64,13 @@ public class TestTimelineServiceRecords {
         metric1.getValues().entrySet().iterator();
     Map.Entry<Long, Number> entry = itr.next();
     Assert.assertEquals(new Long(3L), entry.getKey());
-    Assert.assertEquals(new Double(3.0D), entry.getValue());
+    Assert.assertEquals(3.0D, entry.getValue());
     entry = itr.next();
     Assert.assertEquals(new Long(2L), entry.getKey());
-    Assert.assertEquals(new Integer(2), entry.getValue());
+    Assert.assertEquals(2, entry.getValue());
     entry = itr.next();
     Assert.assertEquals(new Long(1L), entry.getKey());
-    Assert.assertEquals(new Float(1.0F), entry.getValue());
+    Assert.assertEquals(1.0F, entry.getValue());
     Assert.assertFalse(itr.hasNext());
     entity.addMetric(metric1);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78ffdf08/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 4d3dafd..39a6181 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
@@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 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.TimelineMetricOperation;
 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;
@@ -119,12 +120,15 @@ public class NMTimelinePublisher extends CompositeService {
       if (pmemUsage != ResourceCalculatorProcessTree.UNAVAILABLE) {
         TimelineMetric memoryMetric = new TimelineMetric();
         memoryMetric.setId(ContainerMetric.MEMORY.toString());
+        memoryMetric.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
         memoryMetric.addValue(currentTimeMillis, pmemUsage);
         entity.addMetric(memoryMetric);
       }
       if (cpuUsagePercentPerCore != ResourceCalculatorProcessTree.UNAVAILABLE) {
         TimelineMetric cpuMetric = new TimelineMetric();
         cpuMetric.setId(ContainerMetric.CPU.toString());
+        // TODO: support average
+        cpuMetric.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
         cpuMetric.addValue(currentTimeMillis,
             Math.round(cpuUsagePercentPerCore));
         entity.addMetric(cpuMetric);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78ffdf08/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 4fe445a..eb05262 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/AppLevelTimelineCollector.java
@@ -18,15 +18,26 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.collector;
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 import com.google.common.base.Preconditions;
 
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
 /**
  * Service that handles writes to the timeline service and writes them to the
  * backing storage for a given YARN application.
@@ -36,8 +47,16 @@ import com.google.common.base.Preconditions;
 @Private
 @Unstable
 public class AppLevelTimelineCollector extends TimelineCollector {
+  private static final Log LOG = LogFactory.getLog(TimelineCollector.class);
+
+  private final static int AGGREGATION_EXECUTOR_NUM_THREADS = 1;
+  private final static int AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS = 15;
+  private static Set<String> entityTypesSkipAggregation
+      = initializeSkipSet();
+
   private final ApplicationId appId;
   private final TimelineCollectorContext context;
+  private ScheduledThreadPoolExecutor appAggregationExecutor;
 
   public AppLevelTimelineCollector(ApplicationId appId) {
     super(AppLevelTimelineCollector.class.getName() + " - " + appId.toString());
@@ -46,6 +65,14 @@ public class AppLevelTimelineCollector extends TimelineCollector {
     context = new TimelineCollectorContext();
   }
 
+  private static Set<String> initializeSkipSet() {
+    Set<String> result = new HashSet<>();
+    result.add(TimelineEntityType.YARN_APPLICATION.toString());
+    result.add(TimelineEntityType.YARN_FLOW_RUN.toString());
+    result.add(TimelineEntityType.YARN_FLOW_ACTIVITY.toString());
+    return result;
+  }
+
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
     context.setClusterId(conf.get(YarnConfiguration.RM_CLUSTER_ID,
@@ -60,11 +87,25 @@ public class AppLevelTimelineCollector extends TimelineCollector {
 
   @Override
   protected void serviceStart() throws Exception {
+    // Launch the aggregation thread
+    appAggregationExecutor = new ScheduledThreadPoolExecutor(
+        AppLevelTimelineCollector.AGGREGATION_EXECUTOR_NUM_THREADS,
+        new ThreadFactoryBuilder()
+            .setNameFormat("TimelineCollector Aggregation thread #%d")
+            .build());
+    appAggregationExecutor.scheduleAtFixedRate(new AppLevelAggregator(), 0,
+        AppLevelTimelineCollector.AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS,
+        TimeUnit.SECONDS);
     super.serviceStart();
   }
 
   @Override
   protected void serviceStop() throws Exception {
+    appAggregationExecutor.shutdown();
+    if (!appAggregationExecutor.awaitTermination(10, TimeUnit.SECONDS)) {
+      LOG.info("App-level aggregator shutdown timed out, shutdown now. ");
+      appAggregationExecutor.shutdownNow();
+    }
     super.serviceStop();
   }
 
@@ -73,4 +114,35 @@ public class AppLevelTimelineCollector extends TimelineCollector {
     return context;
   }
 
+  @Override
+  protected Set<String> getEntityTypesSkipAggregation() {
+    return entityTypesSkipAggregation;
+  }
+
+  private class AppLevelAggregator implements Runnable {
+
+    @Override
+    public void run() {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("App-level real-time aggregating");
+      }
+      try {
+        TimelineCollectorContext currContext = getTimelineEntityContext();
+        TimelineEntity resultEntity = TimelineCollector.aggregateWithoutGroupId(
+            getAggregationGroups(), currContext.getAppId(),
+            TimelineEntityType.YARN_APPLICATION.toString());
+        TimelineEntities entities = new TimelineEntities();
+        entities.addEntity(resultEntity);
+        getWriter().write(currContext.getClusterId(), currContext.getUserId(),
+            currContext.getFlowName(), currContext.getFlowVersion(),
+            currContext.getFlowRunId(), currContext.getAppId(), entities);
+      } catch (Exception e) {
+        LOG.error("Error aggregating timeline metrics", e);
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("App-level real-time aggregation complete");
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78ffdf08/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
index 15187d1..8cd645c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
@@ -19,6 +19,12 @@
 package org.apache.hadoop.yarn.server.timelineservice.collector;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -27,7 +33,10 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineWriteResponse;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 
@@ -41,9 +50,15 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 @Private
 @Unstable
 public abstract class TimelineCollector extends CompositeService {
+
   private static final Log LOG = LogFactory.getLog(TimelineCollector.class);
+  public static final String SEPARATOR = "_";
 
   private TimelineWriter writer;
+  private ConcurrentMap<String, AggregationStatusTable> aggregationGroups
+      = new ConcurrentHashMap<>();
+  private static Set<String> entityTypesSkipAggregation
+      = new HashSet<>();
 
   public TimelineCollector(String name) {
     super(name);
@@ -68,6 +83,28 @@ public abstract class TimelineCollector extends CompositeService {
     this.writer = w;
   }
 
+  protected TimelineWriter getWriter() {
+    return writer;
+  }
+
+  protected Map<String, AggregationStatusTable> getAggregationGroups() {
+    return aggregationGroups;
+  }
+
+  /**
+   * Method to decide the set of timeline entity types the collector should
+   * skip on aggregations. Subclasses may want to override this method to
+   * customize their own behaviors.
+   *
+   * @return A set of strings consists of all types the collector should skip.
+   */
+  protected Set<String> getEntityTypesSkipAggregation() {
+    return entityTypesSkipAggregation;
+  }
+
+  public abstract TimelineCollectorContext getTimelineEntityContext();
+
+
   /**
    * Handles entity writes. These writes are synchronous and are written to the
    * backing storage without buffering/batching. If any entity already exists,
@@ -90,8 +127,12 @@ public abstract class TimelineCollector extends CompositeService {
       LOG.debug("putEntities(entities=" + entities + ", callerUgi="
           + callerUgi + ")");
     }
-
     TimelineCollectorContext context = getTimelineEntityContext();
+
+    // Update application metrics for aggregation
+    updateAggregateStatus(entities, aggregationGroups,
+        getEntityTypesSkipAggregation());
+
     return writer.write(context.getClusterId(), context.getUserId(),
         context.getFlowName(), context.getFlowVersion(), context.getFlowRunId(),
         context.getAppId(), entities);
@@ -117,6 +158,174 @@ public abstract class TimelineCollector extends CompositeService {
     }
   }
 
-  public abstract TimelineCollectorContext getTimelineEntityContext();
+  /**
+   * Aggregate all metrics in given timeline entities with no predefined states.
+   *
+   * @param entities Entities to aggregate
+   * @param resultEntityId Id of the result entity
+   * @param resultEntityType Type of the result entity
+   * @param needsGroupIdInResult Marks if we want the aggregation group id in
+   *                             each aggregated metrics.
+   * @return A timeline entity that contains all aggregated TimelineMetric.
+   */
+  public static TimelineEntity aggregateEntities(
+      TimelineEntities entities, String resultEntityId,
+      String resultEntityType, boolean needsGroupIdInResult) {
+    ConcurrentMap<String, AggregationStatusTable> aggregationGroups
+        = new ConcurrentHashMap<>();
+    updateAggregateStatus(entities, aggregationGroups, null);
+    if (needsGroupIdInResult) {
+      return aggregate(aggregationGroups, resultEntityId, resultEntityType);
+    } else {
+      return aggregateWithoutGroupId(
+          aggregationGroups, resultEntityId, resultEntityType);
+    }
+  }
 
+  /**
+   * Update the aggregation status table for a timeline collector.
+   *
+   * @param entities Entities to update
+   * @param aggregationGroups Aggregation status table
+   * @param typesToSkip Entity types that we can safely assume to skip updating
+   */
+  static void updateAggregateStatus(
+      TimelineEntities entities,
+      ConcurrentMap<String, AggregationStatusTable> aggregationGroups,
+      Set<String> typesToSkip) {
+    for (TimelineEntity e : entities.getEntities()) {
+      if ((typesToSkip != null && typesToSkip.contains(e.getType()))
+          || e.getMetrics().isEmpty()) {
+        continue;
+      }
+      AggregationStatusTable aggrTable = aggregationGroups.get(e.getType());
+      if (aggrTable == null) {
+        AggregationStatusTable table = new AggregationStatusTable();
+        aggrTable = aggregationGroups.putIfAbsent(e.getType(),
+            table);
+        if (aggrTable == null) {
+          aggrTable = table;
+        }
+      }
+      aggrTable.update(e);
+    }
+  }
+
+  /**
+   * Aggregate internal status and generate timeline entities for the
+   * aggregation results.
+   *
+   * @param aggregationGroups Aggregation status table
+   * @param resultEntityId Id of the result entity
+   * @param resultEntityType Type of the result entity
+   * @return A timeline entity that contains all aggregated TimelineMetric.
+   */
+  static TimelineEntity aggregate(
+      Map<String, AggregationStatusTable> aggregationGroups,
+      String resultEntityId, String resultEntityType) {
+    TimelineEntity result = new TimelineEntity();
+    result.setId(resultEntityId);
+    result.setType(resultEntityType);
+    for (Map.Entry<String, AggregationStatusTable> entry
+        : aggregationGroups.entrySet()) {
+      entry.getValue().aggregateAllTo(result, entry.getKey());
+    }
+    return result;
+  }
+
+  /**
+   * Aggregate internal status and generate timeline entities for the
+   * aggregation results. The result metrics will not have aggregation group
+   * information.
+   *
+   * @param aggregationGroups Aggregation status table
+   * @param resultEntityId Id of the result entity
+   * @param resultEntityType Type of the result entity
+   * @return A timeline entity that contains all aggregated TimelineMetric.
+   */
+  static TimelineEntity aggregateWithoutGroupId(
+      Map<String, AggregationStatusTable> aggregationGroups,
+      String resultEntityId, String resultEntityType) {
+    TimelineEntity result = new TimelineEntity();
+    result.setId(resultEntityId);
+    result.setType(resultEntityType);
+    for (Map.Entry<String, AggregationStatusTable> entry
+        : aggregationGroups.entrySet()) {
+      entry.getValue().aggregateAllTo(result, "");
+    }
+    return result;
+  }
+
+  // Note: In memory aggregation is performed in an eventually consistent
+  // fashion.
+  private static class AggregationStatusTable {
+    // On aggregation, for each metric, aggregate all per-entity accumulated
+    // metrics. We only use the id and type for TimelineMetrics in the key set
+    // of this table.
+    private ConcurrentMap<TimelineMetric, Map<String, TimelineMetric>>
+        aggregateTable;
+
+    public AggregationStatusTable() {
+      aggregateTable = new ConcurrentHashMap<>();
+    }
+
+    public void update(TimelineEntity incoming) {
+      String entityId = incoming.getId();
+      for (TimelineMetric m : incoming.getMetrics()) {
+        // Skip if the metric does not need aggregation
+        if (m.getRealtimeAggregationOp() == TimelineMetricOperation.NOP) {
+          continue;
+        }
+        // Update aggregateTable
+        Map<String, TimelineMetric> aggrRow = aggregateTable.get(m);
+        if (aggrRow == null) {
+          Map<String, TimelineMetric> tempRow = new ConcurrentHashMap<>();
+          aggrRow = aggregateTable.putIfAbsent(m, tempRow);
+          if (aggrRow == null) {
+            aggrRow = tempRow;
+          }
+        }
+        aggrRow.put(entityId, m);
+      }
+    }
+
+    public TimelineEntity aggregateTo(TimelineMetric metric, TimelineEntity e,
+        String aggregationGroupId) {
+      if (metric.getRealtimeAggregationOp() == TimelineMetricOperation.NOP) {
+        return e;
+      }
+      Map<String, TimelineMetric> aggrRow = aggregateTable.get(metric);
+      if (aggrRow != null) {
+        TimelineMetric aggrMetric = new TimelineMetric();
+        if (aggregationGroupId.length() > 0) {
+          aggrMetric.setId(metric.getId() + SEPARATOR + aggregationGroupId);
+        } else {
+          aggrMetric.setId(metric.getId());
+        }
+        aggrMetric.setRealtimeAggregationOp(TimelineMetricOperation.NOP);
+        Map<Object, Object> status = new HashMap<>();
+        for (TimelineMetric m : aggrRow.values()) {
+          TimelineMetric.aggregateTo(m, aggrMetric, status);
+          // getRealtimeAggregationOp returns an enum so we can directly
+          // compare with "!=".
+          if (m.getRealtimeAggregationOp()
+              != aggrMetric.getRealtimeAggregationOp()) {
+            aggrMetric.setRealtimeAggregationOp(m.getRealtimeAggregationOp());
+          }
+        }
+        Set<TimelineMetric> metrics = e.getMetrics();
+        metrics.remove(aggrMetric);
+        metrics.add(aggrMetric);
+      }
+      return e;
+    }
+
+    public TimelineEntity aggregateAllTo(TimelineEntity e,
+        String aggregationGroupId) {
+      for (TimelineMetric m : aggregateTable.keySet()) {
+        aggregateTo(m, e, aggregationGroupId);
+      }
+      return e;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78ffdf08/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineAggregationTrack.java
----------------------------------------------------------------------
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/TimelineAggregationTrack.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineAggregationTrack.java
index f0b1e47..6a1e086 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineAggregationTrack.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineAggregationTrack.java
@@ -24,5 +24,5 @@ package org.apache.hadoop.yarn.server.timelineservice.storage;
  *
  */
 public enum TimelineAggregationTrack {
-  FLOW, USER, QUEUE
+  APP, FLOW, USER, QUEUE
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78ffdf08/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
new file mode 100644
index 0000000..5b4dc50
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.collector;
+
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class TestTimelineCollector {
+
+  private TimelineEntities generateTestEntities(int groups, int entities) {
+    TimelineEntities te = new TimelineEntities();
+    for (int j = 0; j < groups; j++) {
+      for (int i = 0; i < entities; i++) {
+        TimelineEntity entity = new TimelineEntity();
+        String containerId = "container_1000178881110_2002_" + i;
+        entity.setId(containerId);
+        String entityType = "TEST_" + j;
+        entity.setType(entityType);
+        long cTime = 1425016501000L;
+        entity.setCreatedTime(cTime);
+
+        // add metrics
+        Set<TimelineMetric> metrics = new HashSet<>();
+        TimelineMetric m1 = new TimelineMetric();
+        m1.setId("HDFS_BYTES_WRITE");
+        m1.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
+        long ts = System.currentTimeMillis();
+        m1.addValue(ts - 20000, 100L);
+        metrics.add(m1);
+
+        TimelineMetric m2 = new TimelineMetric();
+        m2.setId("VCORES_USED");
+        m2.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
+        m2.addValue(ts - 20000, 3L);
+        metrics.add(m2);
+
+        // m3 should not show up in the aggregation
+        TimelineMetric m3 = new TimelineMetric();
+        m3.setId("UNRELATED_VALUES");
+        m3.addValue(ts - 20000, 3L);
+        metrics.add(m3);
+
+        TimelineMetric m4 = new TimelineMetric();
+        m4.setId("TXN_FINISH_TIME");
+        m4.setRealtimeAggregationOp(TimelineMetricOperation.MAX);
+        m4.addValue(ts - 20000, i);
+        metrics.add(m4);
+
+        entity.addMetrics(metrics);
+        te.addEntity(entity);
+      }
+    }
+
+    return te;
+  }
+
+  @Test
+  public void testAggregation() throws Exception {
+    // Test aggregation with multiple groups.
+    int groups = 3;
+    int n = 50;
+    TimelineEntities testEntities = generateTestEntities(groups, n);
+    TimelineEntity resultEntity = TimelineCollector.aggregateEntities(
+        testEntities, "test_result", "TEST_AGGR", true);
+    assertEquals(resultEntity.getMetrics().size(), groups * 3);
+
+    for (int i = 0; i < groups; i++) {
+      Set<TimelineMetric> metrics = resultEntity.getMetrics();
+      for (TimelineMetric m : metrics) {
+        if (m.getId().startsWith("HDFS_BYTES_WRITE")) {
+          assertEquals(100 * n, m.getSingleDataValue().intValue());
+        } else if (m.getId().startsWith("VCORES_USED")) {
+          assertEquals(3 * n, m.getSingleDataValue().intValue());
+        } else if (m.getId().startsWith("TXN_FINISH_TIME")) {
+          assertEquals(n - 1, m.getSingleDataValue());
+        } else {
+          fail("Unrecognized metric! " + m.getId());
+        }
+      }
+    }
+
+    // Test aggregation with a single group.
+    TimelineEntities testEntities1 = generateTestEntities(1, n);
+    TimelineEntity resultEntity1 = TimelineCollector.aggregateEntities(
+        testEntities1, "test_result", "TEST_AGGR", false);
+    assertEquals(resultEntity1.getMetrics().size(), 3);
+
+    Set<TimelineMetric> metrics = resultEntity1.getMetrics();
+    for (TimelineMetric m : metrics) {
+      if (m.getId().equals("HDFS_BYTES_WRITE")) {
+        assertEquals(100 * n, m.getSingleDataValue().intValue());
+      } else if (m.getId().equals("VCORES_USED")) {
+        assertEquals(3 * n, m.getSingleDataValue().intValue());
+      } else if (m.getId().equals("TXN_FINISH_TIME")) {
+        assertEquals(n - 1, m.getSingleDataValue());
+      } else {
+        fail("Unrecognized metric! " + m.getId());
+      }
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78ffdf08/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java
index 5ce7d3b..2f79daa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineWriterImpl.java
@@ -25,11 +25,15 @@ import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.junit.Test;
@@ -51,6 +55,26 @@ public class TestFileSystemTimelineWriterImpl {
     entity.setCreatedTime(1425016501000L);
     te.addEntity(entity);
 
+    TimelineMetric metric = new TimelineMetric();
+    String metricId = "CPU";
+    metric.setId(metricId);
+    metric.setType(TimelineMetric.Type.SINGLE_VALUE);
+    metric.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
+    metric.addValue(1425016501000L, 1234567L);
+
+    TimelineEntity entity2 = new TimelineEntity();
+    String id2 = "metric";
+    String type2 = "app";
+    entity2.setId(id2);
+    entity2.setType(type2);
+    entity2.setCreatedTime(1425016503000L);
+    entity2.addMetric(metric);
+    te.addEntity(entity2);
+
+    Map<String, TimelineMetric> aggregatedMetrics =
+        new HashMap<String, TimelineMetric>();
+    aggregatedMetrics.put(metricId, metric);
+
     FileSystemTimelineWriterImpl fsi = null;
     try {
       fsi = new FileSystemTimelineWriterImpl();
@@ -68,11 +92,27 @@ public class TestFileSystemTimelineWriterImpl {
       assertTrue(f.exists() && !f.isDirectory());
       List<String> data = Files.readAllLines(path, StandardCharsets.UTF_8);
       // ensure there's only one entity + 1 new line
-      assertTrue(data.size() == 2);
+      assertTrue("data size is:" + data.size(), data.size() == 2);
       String d = data.get(0);
       // confirm the contents same as what was written
       assertEquals(d, TimelineUtils.dumpTimelineRecordtoJSON(entity));
 
+      // verify aggregated metrics
+      String fileName2 = fsi.getOutputRoot() +
+          "/entities/cluster_id/user_id/flow_name/flow_version/12345678/app_id/"
+          + type2 + "/" + id2 +
+          FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
+      Path path2 = Paths.get(fileName2);
+      File file = new File(fileName2);
+      assertTrue(file.exists() && !file.isDirectory());
+      List<String> data2 = Files.readAllLines(path2, StandardCharsets.UTF_8);
+      // ensure there's only one entity + 1 new line
+      assertTrue("data size is:" + data.size(), data2.size() == 2);
+      String metricToString = data2.get(0);
+      // confirm the contents same as what was written
+      assertEquals(metricToString,
+          TimelineUtils.dumpTimelineRecordtoJSON(entity2));
+
       // delete the directory
       File outputDir = new File(fsi.getOutputRoot());
       FileUtils.deleteDirectory(outputDir);
@@ -84,4 +124,5 @@ public class TestFileSystemTimelineWriterImpl {
       }
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78ffdf08/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 6b57ec4..8ab54bc 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
@@ -42,6 +42,7 @@ 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.TimelineMetricOperation;
 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;
@@ -539,6 +540,26 @@ public class TestHBaseTimelineStorage {
     metrics.add(m1);
     entity.addMetrics(metrics);
 
+    // add aggregated metrics
+    TimelineEntity aggEntity = new TimelineEntity();
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    aggEntity.setId(appId);
+    aggEntity.setType(type);
+    long cTime2 = 1425016502000L;
+    long mTime2 = 1425026902000L;
+    aggEntity.setCreatedTime(cTime2);
+
+    TimelineMetric aggMetric = new TimelineMetric();
+    aggMetric.setId("MEM_USAGE");
+    Map<Long, Number> aggMetricValues = new HashMap<Long, Number>();
+    ts = System.currentTimeMillis();
+    aggMetricValues.put(ts - 120000, 102400000);
+    aggMetric.setType(Type.SINGLE_VALUE);
+    aggMetric.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
+    aggMetric.setValues(aggMetricValues);
+    Set<TimelineMetric> aggMetrics = new HashSet<>();
+    aggMetrics.add(aggMetric);
+    entity.addMetrics(aggMetrics);
     te.addEntity(entity);
 
     HBaseTimelineWriterImpl hbi = null;
@@ -564,7 +585,7 @@ public class TestHBaseTimelineStorage {
       Result result = new ApplicationTable().getResult(c1, conn, get);
 
       assertTrue(result != null);
-      assertEquals(15, result.size());
+      assertEquals(16, result.size());
 
       // check the row key
       byte[] row1 = result.getRow();
@@ -652,10 +673,17 @@ public class TestHBaseTimelineStorage {
       assertEquals(conf, conf2);
 
       Set<TimelineMetric> metrics2 = e1.getMetrics();
-      assertEquals(metrics, metrics2);
+      assertEquals(2, metrics2.size());
       for (TimelineMetric metric2 : metrics2) {
         Map<Long, Number> metricValues2 = metric2.getValues();
-        matchMetrics(metricValues, metricValues2);
+        assertTrue(metric2.getId().equals("MAP_SLOT_MILLIS") ||
+            metric2.getId().equals("MEM_USAGE"));
+        if (metric2.getId().equals("MAP_SLOT_MILLIS")) {
+          matchMetrics(metricValues, metricValues2);
+        }
+        if (metric2.getId().equals("MEM_USAGE")) {
+          matchMetrics(aggMetricValues, metricValues2);
+        }
       }
     } finally {
       if (hbi != null) {
@@ -724,7 +752,6 @@ public class TestHBaseTimelineStorage {
     m1.setValues(metricValues);
     metrics.add(m1);
     entity.addMetrics(metrics);
-
     te.addEntity(entity);
 
     HBaseTimelineWriterImpl hbi = null;


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


[37/50] [abbrv] hadoop git commit: YARN-5109. timestamps are stored unencoded causing parse errors (Varun Saxena via sjlee)

Posted by vr...@apache.org.
YARN-5109. timestamps are stored unencoded causing parse errors (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/4ad33a07
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4ad33a07
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4ad33a07

Branch: refs/heads/YARN-2928
Commit: 4ad33a07c8e22e8ec5b6b02c9a272c0d48b1d11b
Parents: d10932a
Author: Sangjin Lee <sj...@apache.org>
Authored: Thu May 26 21:39:16 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:12 2016 -0700

----------------------------------------------------------------------
 .../storage/TestHBaseTimelineStorage.java       | 145 ++++++---
 .../flow/TestHBaseStorageFlowActivity.java      |   8 +-
 .../reader/filter/TimelineFilterUtils.java      |  20 +-
 .../storage/HBaseTimelineWriterImpl.java        |  67 +++--
 .../application/ApplicationColumnPrefix.java    |  65 ++--
 .../storage/application/ApplicationRowKey.java  |  50 +---
 .../application/ApplicationRowKeyConverter.java | 130 ++++++++
 .../storage/apptoflow/AppToFlowRowKey.java      |  20 +-
 .../apptoflow/AppToFlowRowKeyConverter.java     |  96 ++++++
 .../storage/common/AppIdKeyConverter.java       | 101 +++++++
 .../storage/common/ColumnHelper.java            | 175 +++++------
 .../storage/common/ColumnPrefix.java            |  43 +--
 .../storage/common/EventColumnName.java         |  48 +++
 .../common/EventColumnNameConverter.java        | 105 +++++++
 .../storage/common/KeyConverter.java            |  41 +++
 .../storage/common/LongKeyConverter.java        |  68 +++++
 .../storage/common/Separator.java               | 198 ++++++++++++-
 .../storage/common/StringKeyConverter.java      |  59 ++++
 .../storage/common/TimelineStorageUtils.java    | 199 ++-----------
 .../storage/entity/EntityColumnPrefix.java      |  48 +--
 .../storage/entity/EntityRowKey.java            |  67 +----
 .../storage/entity/EntityRowKeyConverter.java   | 143 +++++++++
 .../storage/flow/FlowActivityColumnPrefix.java  |  38 +--
 .../storage/flow/FlowActivityRowKey.java        |  41 +--
 .../flow/FlowActivityRowKeyConverter.java       | 115 ++++++++
 .../storage/flow/FlowRunColumnPrefix.java       |  82 ++----
 .../storage/flow/FlowRunRowKey.java             |  41 +--
 .../storage/flow/FlowRunRowKeyConverter.java    | 120 ++++++++
 .../storage/flow/FlowScanner.java               |   9 +-
 .../reader/FlowActivityEntityReader.java        |  12 +-
 .../storage/reader/TimelineEntityReader.java    |   4 +-
 .../storage/common/TestKeyConverters.java       | 293 +++++++++++++++++++
 .../storage/common/TestSeparator.java           |  82 +++++-
 .../common/TestTimelineStorageUtils.java        |  56 ----
 34 files changed, 1988 insertions(+), 801 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/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-hbase-tests/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-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index 68135a0..bcf2d2c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -20,6 +20,7 @@ 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.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -42,7 +43,6 @@ 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.TimelineMetricOperation;
 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;
@@ -50,25 +50,28 @@ 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.api.records.timelineservice.TimelineMetricOperation;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
 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.reader.filter.TimelineKeyValueFilter;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
 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.EventColumnName;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
 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.StringKeyConverter;
 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;
@@ -482,7 +485,6 @@ public class TestHBaseTimelineStorage {
     }
   }
 
-
   @Test
   public void testWriteNullApplicationToHBase() throws Exception {
     TimelineEntities te = new TimelineEntities();
@@ -494,7 +496,7 @@ public class TestHBaseTimelineStorage {
 
     // add the info map in Timeline Entity
     Map<String, Object> infoMap = new HashMap<String, Object>();
-    infoMap.put("infoMapKey1", "infoMapValue1");
+    infoMap.put("in fo M apK  ey1", "infoMapValue1");
     infoMap.put("infoMapKey2", 10);
     entity.addInfo(infoMap);
 
@@ -517,6 +519,7 @@ public class TestHBaseTimelineStorage {
       // retrieve the row
       Scan scan = new Scan();
       scan.setStartRow(Bytes.toBytes(cluster));
+      scan.setStopRow(Bytes.toBytes(cluster + "1"));
       Connection conn = ConnectionFactory.createConnection(c1);
       ResultScanner resultScanner = new ApplicationTable()
           .getResultScanner(c1, conn, scan);
@@ -626,7 +629,7 @@ public class TestHBaseTimelineStorage {
       hbi.start();
       String cluster = "cluster_test_write_app";
       String user = "user1";
-      String flow = "some_flow_name";
+      String flow = "s!ome_f\tlow  _n am!e";
       String flowVersion = "AB7822C10F1111";
       long runid = 1002345678919L;
       hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
@@ -670,7 +673,8 @@ public class TestHBaseTimelineStorage {
       assertEquals(cTime, cTime1);
 
       Map<String, Object> infoColumns =
-          ApplicationColumnPrefix.INFO.readResults(result);
+          ApplicationColumnPrefix.INFO.readResults(result,
+              StringKeyConverter.getInstance());
       assertEquals(infoMap, infoColumns);
 
       // Remember isRelatedTo is of type Map<String, Set<String>>
@@ -706,11 +710,13 @@ public class TestHBaseTimelineStorage {
 
       // Configuration
       Map<String, Object> configColumns =
-          ApplicationColumnPrefix.CONFIG.readResults(result);
+          ApplicationColumnPrefix.CONFIG.readResults(result,
+              StringKeyConverter.getInstance());
       assertEquals(conf, configColumns);
 
       NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
-          ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result);
+          ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(
+              result, StringKeyConverter.getInstance());
 
       NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
       matchMetrics(metricValues, metricMap);
@@ -868,7 +874,8 @@ public class TestHBaseTimelineStorage {
           assertEquals(cTime1, cTime);
 
           Map<String, Object> infoColumns =
-              EntityColumnPrefix.INFO.readResults(result);
+              EntityColumnPrefix.INFO.readResults(result,
+                  StringKeyConverter.getInstance());
           assertEquals(infoMap, infoColumns);
 
           // Remember isRelatedTo is of type Map<String, Set<String>>
@@ -906,11 +913,12 @@ public class TestHBaseTimelineStorage {
 
           // Configuration
           Map<String, Object> configColumns =
-              EntityColumnPrefix.CONFIG.readResults(result);
+              EntityColumnPrefix.CONFIG.readResults(result, StringKeyConverter.getInstance());
           assertEquals(conf, configColumns);
 
           NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
-              EntityColumnPrefix.METRIC.readResultsWithTimestamps(result);
+              EntityColumnPrefix.METRIC.readResultsWithTimestamps(
+                  result, StringKeyConverter.getInstance());
 
           NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
           matchMetrics(metricValues, metricMap);
@@ -963,7 +971,7 @@ public class TestHBaseTimelineStorage {
   }
 
   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) {
 
     EntityRowKey key = EntityRowKey.parseRowKey(rowKey);
 
@@ -978,7 +986,7 @@ public class TestHBaseTimelineStorage {
   }
 
   private boolean isApplicationRowKeyCorrect(byte[] rowKey, String cluster,
-      String user, String flow, long runid, String appName) {
+      String user, String flow, Long runid, String appName) {
 
     ApplicationRowKey key = ApplicationRowKey.parseRowKey(rowKey);
 
@@ -995,7 +1003,7 @@ public class TestHBaseTimelineStorage {
     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";
@@ -1038,20 +1046,18 @@ public class TestHBaseTimelineStorage {
       assertTrue(isApplicationRowKeyCorrect(row1, cluster, user, flow, runid,
           appName));
 
-      Map<?, Object> eventsResult =
-          ApplicationColumnPrefix.EVENT.
-              readResultsHavingCompoundColumnQualifiers(result);
+      Map<EventColumnName, Object> eventsResult =
+          ApplicationColumnPrefix.EVENT.readResults(result,
+              EventColumnNameConverter.getInstance());
       // there should be only one event
       assertEquals(1, eventsResult.size());
-      for (Map.Entry<?, Object> e : eventsResult.entrySet()) {
+      for (Map.Entry<EventColumnName, Object> e : eventsResult.entrySet()) {
+        EventColumnName eventColumnName = e.getKey();
         // 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(
-            TimelineStorageUtils.invertLong(expTs), Bytes.toLong(karr[1]));
-        assertEquals(expKey, Bytes.toString(karr[2]));
+        assertEquals(eventId, eventColumnName.getId());
+        assertEquals(expTs, eventColumnName.getTimestamp());
+        assertEquals(expKey, eventColumnName.getInfoKey());
         Object value = e.getValue();
         // there should be only one timestamp and value
         assertEquals(expVal, value.toString());
@@ -1076,7 +1082,7 @@ public class TestHBaseTimelineStorage {
       assertEquals(1, events.size());
       for (TimelineEvent e : events) {
         assertEquals(eventId, e.getId());
-        assertEquals(expTs, e.getTimestamp());
+        assertEquals(expTs, Long.valueOf(e.getTimestamp()));
         Map<String,Object> info = e.getInfo();
         assertEquals(1, info.size());
         for (Map.Entry<String, Object> infoEntry : info.entrySet()) {
@@ -1095,9 +1101,9 @@ public class TestHBaseTimelineStorage {
   @Test
   public void testEventsWithEmptyInfo() throws IOException {
     TimelineEvent event = new TimelineEvent();
-    String eventId = "foo_event_id";
+    String eventId = "foo_ev e  nt_id";
     event.setId(eventId);
-    long expTs = 1436512802000L;
+    Long expTs = 1436512802000L;
     event.setTimestamp(expTs);
 
     final TimelineEntity entity = new TimelineEntity();
@@ -1142,21 +1148,19 @@ public class TestHBaseTimelineStorage {
           assertTrue(isRowKeyCorrect(row1, cluster, user, flow, runid, appName,
               entity));
 
-          Map<?, Object> eventsResult =
-              EntityColumnPrefix.EVENT.
-                  readResultsHavingCompoundColumnQualifiers(result);
+          Map<EventColumnName, Object> eventsResult =
+              EntityColumnPrefix.EVENT.readResults(result,
+                  EventColumnNameConverter.getInstance());
           // there should be only one event
           assertEquals(1, eventsResult.size());
-          for (Map.Entry<?, Object> e : eventsResult.entrySet()) {
+          for (Map.Entry<EventColumnName, Object> e : eventsResult.entrySet()) {
+            EventColumnName eventColumnName = e.getKey();
             // 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(TimelineStorageUtils.invertLong(expTs),
-                Bytes.toLong(karr[1]));
+            assertEquals(eventId, eventColumnName.getId());
+            assertEquals(expTs,eventColumnName.getTimestamp());
             // key must be empty
-            assertEquals(0, karr[2].length);
+            assertNull(eventColumnName.getInfoKey());
             Object value = e.getValue();
             // value should be empty
             assertEquals("", value.toString());
@@ -1184,7 +1188,7 @@ public class TestHBaseTimelineStorage {
       assertEquals(1, events.size());
       for (TimelineEvent e : events) {
         assertEquals(eventId, e.getId());
-        assertEquals(expTs, e.getTimestamp());
+        assertEquals(expTs, Long.valueOf(e.getTimestamp()));
         Map<String,Object> info = e.getInfo();
         assertTrue(info == null || info.isEmpty());
       }
@@ -1195,6 +1199,67 @@ public class TestHBaseTimelineStorage {
   }
 
   @Test
+  public void testEventsEscapeTs() throws IOException {
+    TimelineEvent event = new TimelineEvent();
+    String eventId = ApplicationMetricsConstants.CREATED_EVENT_TYPE;
+    event.setId(eventId);
+    long expTs = 1463567041056L;
+    event.setTimestamp(expTs);
+    String expKey = "f==o o_e ve\tnt";
+    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;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      String cluster = "clus!ter_\ttest_ev  ents";
+      String user = "user2";
+      String flow = "other_flow_name";
+      String flowVersion = "1111F01C2287BA";
+      long runid = 1009876543218L;
+      String appName = "application_123465899910_2001";
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
+      hbi.stop();
+
+      // read the timeline entity using the reader this time
+      TimelineEntity e1 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, appName,
+          entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+      assertNotNull(e1);
+      // 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();
+      }
+    }
+  }
+
+  @Test
   public void testNonIntegralMetricValues() throws IOException {
     TimelineEntities teApp = new TimelineEntities();
     ApplicationEntity entityApp = new ApplicationEntity();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
index 6b23b6c..072332d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
@@ -170,7 +170,7 @@ public class TestHBaseStorageFlowActivity {
     assertEquals(cluster, flowActivityRowKey.getClusterId());
     assertEquals(user, flowActivityRowKey.getUserId());
     assertEquals(flow, flowActivityRowKey.getFlowName());
-    long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(minStartTs);
+    Long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(minStartTs);
     assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
     assertEquals(1, values.size());
     checkFlowActivityRunId(runid, flowVersion, values);
@@ -194,7 +194,7 @@ public class TestHBaseStorageFlowActivity {
         assertEquals(cluster, flowActivity.getCluster());
         assertEquals(user, flowActivity.getUser());
         assertEquals(flow, flowActivity.getFlowName());
-        assertEquals(dayTs, flowActivity.getDate().getTime());
+        assertEquals(dayTs, Long.valueOf(flowActivity.getDate().getTime()));
         Set<FlowRunEntity> flowRuns = flowActivity.getFlowRuns();
         assertEquals(1, flowRuns.size());
       }
@@ -294,7 +294,7 @@ public class TestHBaseStorageFlowActivity {
       assertEquals(cluster, flowActivityRowKey.getClusterId());
       assertEquals(user, flowActivityRowKey.getUserId());
       assertEquals(flow, flowActivityRowKey.getFlowName());
-      long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(appCreatedTime);
+      Long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(appCreatedTime);
       assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
       assertEquals(1, values.size());
       checkFlowActivityRunId(runid, flowVersion, values);
@@ -429,7 +429,7 @@ public class TestHBaseStorageFlowActivity {
       assertEquals(cluster, flowActivityRowKey.getClusterId());
       assertEquals(user, flowActivityRowKey.getUserId());
       assertEquals(flow, flowActivityRowKey.getFlowName());
-      long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(appCreatedTime);
+      Long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(appCreatedTime);
       assertEquals(dayTs, flowActivityRowKey.getDayTimestamp());
 
       Map<byte[], byte[]> values = result

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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
index 8cae410..036746b 100644
--- 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
@@ -31,10 +31,14 @@ 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.application.ApplicationColumnPrefix;
 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.ColumnPrefix;
-
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 
@@ -205,6 +209,17 @@ public final class TimelineFilterUtils {
     return singleColValFilter;
   }
 
+  private static <T> byte[] createColQualifierPrefix(ColumnPrefix<T> colPrefix,
+      String column) {
+    if (colPrefix == ApplicationColumnPrefix.EVENT ||
+        colPrefix == EntityColumnPrefix.EVENT) {
+      return EventColumnNameConverter.getInstance().encode(
+          new EventColumnName(column, null, null));
+    } else {
+      return StringKeyConverter.getInstance().encode(column);
+    }
+  }
+
   /**
    * Create a filter list of qualifier filters based on passed set of columns.
    *
@@ -219,8 +234,7 @@ public final class TimelineFilterUtils {
     for (String column : columns) {
       // For columns which have compound column qualifiers (eg. events), we need
       // to include the required separator.
-      byte[] compoundColQual =
-          colPrefix.getCompoundColQualBytes(column, (byte[])null);
+      byte[] compoundColQual = createColQualifierPrefix(colPrefix, column);
       list.addFilter(new QualifierFilter(CompareOp.EQUAL,
           new BinaryPrefixComparator(
               colPrefix.getColumnPrefixBytes(compoundColQual))));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 fe4671f..f8b5a65 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
@@ -29,7 +29,6 @@ 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.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;
@@ -37,7 +36,6 @@ 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;
@@ -46,7 +44,11 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlow
 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.EventColumnName;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongKeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
 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;
@@ -194,7 +196,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       long activityTimeStamp) throws IOException {
     byte[] rowKey = FlowActivityRowKey.getRowKey(clusterId, activityTimeStamp,
         userId, flowName);
-    byte[] qualifier = GenericObjectMapper.write(flowRunId);
+    byte[] qualifier = LongKeyConverter.getInstance().encode(flowRunId);
     FlowActivityColumnPrefix.RUN_ID.store(rowKey, flowActivityTable, qualifier,
         null, flowVersion,
         AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
@@ -278,7 +280,8 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   private void storeFlowMetrics(byte[] rowKey, Set<TimelineMetric> metrics,
       Attribute... attributes) throws IOException {
     for (TimelineMetric metric : metrics) {
-      String metricColumnQualifier = metric.getId();
+      byte[] metricColumnQualifier =
+          StringKeyConverter.getInstance().encode(metric.getId());
       Map<Long, Number> timeseries = metric.getValues();
       for (Map.Entry<Long, Number> timeseriesEntry : timeseries.entrySet()) {
         Long timestamp = timeseriesEntry.getKey();
@@ -316,8 +319,9 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       // id3?id4?id5
       String compoundValue =
           Separator.VALUES.joinEncoded(connectedEntity.getValue());
-      columnPrefix.store(rowKey, table, connectedEntity.getKey(), null,
-          compoundValue);
+      columnPrefix.store(rowKey, table,
+          StringKeyConverter.getInstance().encode(connectedEntity.getKey()),
+          null, compoundValue);
     }
   }
 
@@ -337,7 +341,8 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       if (info != null) {
         for (Map.Entry<String, Object> entry : info.entrySet()) {
           ApplicationColumnPrefix.INFO.store(rowKey, applicationTable,
-              entry.getKey(), null, entry.getValue());
+              StringKeyConverter.getInstance().encode(entry.getKey()), null,
+              entry.getValue());
         }
       }
     } else {
@@ -349,8 +354,9 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       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());
+          EntityColumnPrefix.INFO.store(rowKey, entityTable,
+              StringKeyConverter.getInstance().encode(entry.getKey()), null,
+              entry.getValue());
         }
       }
     }
@@ -365,11 +371,13 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       return;
     }
     for (Map.Entry<String, String> entry : config.entrySet()) {
+      byte[] configKey =
+          StringKeyConverter.getInstance().encode(entry.getKey());
       if (isApplication) {
         ApplicationColumnPrefix.CONFIG.store(rowKey, applicationTable,
-            entry.getKey(), null, entry.getValue());
+            configKey, null, entry.getValue());
       } else {
-        EntityColumnPrefix.CONFIG.store(rowKey, entityTable, entry.getKey(),
+        EntityColumnPrefix.CONFIG.store(rowKey, entityTable, configKey,
             null, entry.getValue());
       }
     }
@@ -383,7 +391,8 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       boolean isApplication) throws IOException {
     if (metrics != null) {
       for (TimelineMetric metric : metrics) {
-        String metricColumnQualifier = metric.getId();
+        byte[] metricColumnQualifier =
+            StringKeyConverter.getInstance().encode(metric.getId());
         Map<Long, Number> timeseries = metric.getValues();
         for (Map.Entry<Long, Number> timeseriesEntry : timeseries.entrySet()) {
           Long timestamp = timeseriesEntry.getKey();
@@ -416,41 +425,31 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
                   "! Using the current timestamp");
               eventTimestamp = System.currentTimeMillis();
             }
-            byte[] eventTs =
-                Bytes.toBytes(TimelineStorageUtils.invertLong(eventTimestamp));
+            EventColumnNameConverter converter =
+                EventColumnNameConverter.getInstance();
             Map<String, Object> eventInfo = event.getInfo();
             if ((eventInfo == null) || (eventInfo.size() == 0)) {
+              byte[] columnQualifierBytes = converter.encode(
+                  new EventColumnName(eventId, eventTimestamp, null));
               if (isApplication) {
-                byte[] compoundColumnQualifierBytes =
-                    ApplicationColumnPrefix.EVENT.
-                        getCompoundColQualBytes(eventId, eventTs, null);
                 ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
-                    compoundColumnQualifierBytes, null,
-                    TimelineStorageUtils.EMPTY_BYTES);
+                    columnQualifierBytes, null, Separator.EMPTY_BYTES);
               } else {
-                byte[] compoundColumnQualifierBytes =
-                    EntityColumnPrefix.EVENT.
-                        getCompoundColQualBytes(eventId, eventTs, null);
                 EntityColumnPrefix.EVENT.store(rowKey, entityTable,
-                    compoundColumnQualifierBytes, null,
-                    TimelineStorageUtils.EMPTY_BYTES);
+                    columnQualifierBytes, null, Separator.EMPTY_BYTES);
               }
             } else {
               for (Map.Entry<String, Object> info : eventInfo.entrySet()) {
-                // eventId?infoKey
-                byte[] infoKey = Bytes.toBytes(info.getKey());
+                // eventId=infoKey
+                byte[] columnQualifierBytes = converter.encode(
+                    new EventColumnName(eventId, eventTimestamp,
+                        info.getKey()));
                 if (isApplication) {
-                  byte[] compoundColumnQualifierBytes =
-                      ApplicationColumnPrefix.EVENT.
-                          getCompoundColQualBytes(eventId, eventTs, infoKey);
                   ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
-                      compoundColumnQualifierBytes, null, info.getValue());
+                      columnQualifierBytes, null, info.getValue());
                 } else {
-                  byte[] compoundColumnQualifierBytes =
-                      EntityColumnPrefix.EVENT.
-                          getCompoundColQualBytes(eventId, eventTs, infoKey);
                   EntityColumnPrefix.EVENT.store(rowKey, entityTable,
-                      compoundColumnQualifierBytes, null, info.getValue());
+                      columnQualifierBytes, null, info.getValue());
                 }
               } // for info: eventInfo
             }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 1dfc4db..0febc67 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
@@ -27,9 +27,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.ColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
@@ -56,7 +57,7 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
   /**
    * Lifecycle events for an application.
    */
-  EVENT(ApplicationColumnFamily.INFO, "e", true),
+  EVENT(ApplicationColumnFamily.INFO, "e"),
 
   /**
    * Config column stores configuration with config key as the column name.
@@ -78,7 +79,6 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
    */
   private final String columnPrefix;
   private final byte[] columnPrefixBytes;
-  private final boolean compoundColQual;
 
   /**
    * Private constructor, meant to be used by the enum definition.
@@ -88,18 +88,7 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
    */
   private ApplicationColumnPrefix(ColumnFamily<ApplicationTable> columnFamily,
       String columnPrefix) {
-    this(columnFamily, columnPrefix, false, GenericConverter.getInstance());
-  }
-
-  private ApplicationColumnPrefix(ColumnFamily<ApplicationTable> columnFamily,
-      String columnPrefix, boolean compoundColQual) {
-    this(columnFamily, columnPrefix, compoundColQual,
-        GenericConverter.getInstance());
-  }
-
-  private ApplicationColumnPrefix(ColumnFamily<ApplicationTable> columnFamily,
-      String columnPrefix, ValueConverter converter) {
-    this(columnFamily, columnPrefix, false, converter);
+    this(columnFamily, columnPrefix, GenericConverter.getInstance());
   }
 
   /**
@@ -111,7 +100,7 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
    * this column prefix.
    */
   private ApplicationColumnPrefix(ColumnFamily<ApplicationTable> columnFamily,
-      String columnPrefix, boolean compoundColQual, ValueConverter converter) {
+      String columnPrefix, ValueConverter converter) {
     column = new ColumnHelper<ApplicationTable>(columnFamily, converter);
     this.columnFamily = columnFamily;
     this.columnPrefix = columnPrefix;
@@ -122,7 +111,6 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
       this.columnPrefixBytes =
           Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
     }
-    this.compoundColQual = compoundColQual;
   }
 
   /**
@@ -149,15 +137,6 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
     return columnFamily.getBytes();
   }
 
-  @Override
-  public byte[] getCompoundColQualBytes(String qualifier,
-      byte[]...components) {
-    if (!compoundColQual) {
-      return ColumnHelper.getColumnQualifier(null, qualifier);
-    }
-    return ColumnHelper.getCompoundColumnQualifierBytes(qualifier, components);
-  }
-
   /*
    * (non-Javadoc)
    *
@@ -232,25 +211,12 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
    *
    * @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);
-  }
-
-  /**
-   * @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 if any problem occurs while reading results.
+   * #readResults(org.apache.hadoop.hbase.client.Result,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
    */
-  public Map<?, Object> readResultsHavingCompoundColumnQualifiers(Result result)
-      throws IOException {
-    return column.readResultsHavingCompoundColumnQualifiers(result,
-        columnPrefixBytes);
+  public <K> Map<K, Object> readResults(Result result,
+      KeyConverter<K> keyConverter) throws IOException {
+    return column.readResults(result, columnPrefixBytes, keyConverter);
   }
 
   /*
@@ -258,11 +224,14 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
    *
    * @see
    * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
-   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result)
+   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
    */
-  public <V> NavigableMap<String, NavigableMap<Long, V>>
-      readResultsWithTimestamps(Result result) throws IOException {
-    return column.readResultsWithTimestamps(result, columnPrefixBytes);
+  public <K, V> NavigableMap<K, NavigableMap<Long, V>>
+      readResultsWithTimestamps(Result result, KeyConverter<K> keyConverter)
+      throws IOException {
+    return column.readResultsWithTimestamps(result, columnPrefixBytes,
+        keyConverter);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 ad2aa7a..e476b21 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
@@ -15,11 +15,8 @@
  * 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;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+package org.apache.hadoop.yarn.server.timelineservice.storage.application;
 
 /**
  * Represents a rowkey for the application table.
@@ -28,11 +25,11 @@ public class ApplicationRowKey {
   private final String clusterId;
   private final String userId;
   private final String flowName;
-  private final long flowRunId;
+  private final Long flowRunId;
   private final String appId;
 
   public ApplicationRowKey(String clusterId, String userId, String flowName,
-      long flowRunId, String appId) {
+      Long flowRunId, String appId) {
     this.clusterId = clusterId;
     this.userId = userId;
     this.flowName = flowName;
@@ -52,7 +49,7 @@ public class ApplicationRowKey {
     return flowName;
   }
 
-  public long getFlowRunId() {
+  public Long getFlowRunId() {
     return flowRunId;
   }
 
@@ -71,9 +68,8 @@ public class ApplicationRowKey {
    */
   public static byte[] getRowKeyPrefix(String clusterId, String userId,
       String flowName) {
-    byte[] first = Bytes.toBytes(
-        Separator.QUALIFIERS.joinEncoded(clusterId, userId, flowName));
-    return Separator.QUALIFIERS.join(first, new byte[0]);
+    return ApplicationRowKeyConverter.getInstance().encode(
+        new ApplicationRowKey(clusterId, userId, flowName, null, null));
   }
 
   /**
@@ -88,10 +84,8 @@ public class ApplicationRowKey {
    */
   public static byte[] getRowKeyPrefix(String clusterId, String userId,
       String flowName, Long flowRunId) {
-    byte[] first = Bytes.toBytes(
-        Separator.QUALIFIERS.joinEncoded(clusterId, userId, flowName));
-    byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(flowRunId));
-    return Separator.QUALIFIERS.join(first, second, new byte[0]);
+    return ApplicationRowKeyConverter.getInstance().encode(
+        new ApplicationRowKey(clusterId, userId, flowName, flowRunId, null));
   }
 
   /**
@@ -107,14 +101,8 @@ public class ApplicationRowKey {
    */
   public static byte[] getRowKey(String clusterId, String userId,
       String flowName, Long flowRunId, String appId) {
-    byte[] first =
-        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId, 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));
-    byte[] third = TimelineStorageUtils.encodeAppId(appId);
-    return Separator.QUALIFIERS.join(first, second, third);
+    return ApplicationRowKeyConverter.getInstance().encode(
+        new ApplicationRowKey(clusterId, userId, flowName, flowRunId, appId));
   }
 
   /**
@@ -124,22 +112,6 @@ public class ApplicationRowKey {
    * @return An <cite>ApplicationRowKey</cite> object.
    */
   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 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, flowName, flowRunId, appId);
+    return ApplicationRowKeyConverter.getInstance().decode(rowKey);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyConverter.java
----------------------------------------------------------------------
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/ApplicationRowKeyConverter.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/ApplicationRowKeyConverter.java
new file mode 100644
index 0000000..3b054a5
--- /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/ApplicationRowKeyConverter.java
@@ -0,0 +1,130 @@
+/**
+ * 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.AppIdKeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+
+/**
+ * Encodes and decodes row key for application table.
+ * The row key is of the form : clusterId!userName!flowName!flowRunId!appId.
+ * flowRunId is a long, appId is encoded and decoded using
+ * {@link AppIdKeyConverter} and rest are strings.
+ */
+public final class ApplicationRowKeyConverter implements
+    KeyConverter<ApplicationRowKey> {
+  private static final ApplicationRowKeyConverter INSTANCE =
+      new ApplicationRowKeyConverter();
+
+  public static ApplicationRowKeyConverter getInstance() {
+    return INSTANCE;
+  }
+
+  private ApplicationRowKeyConverter() {
+  }
+
+  // Application row key is of the form
+  // clusterId!userName!flowName!flowRunId!appId with each segment separated
+  // by !. The sizes below indicate sizes of each one of these segements in
+  // sequence. clusterId, userName and flowName are strings. flowrunId is a long
+  // hence 8 bytes in size. app id is represented as 12 bytes with cluster
+  // timestamp part of appid being 8 bytes(long) and seq id being 4 bytes(int).
+  // Strings are variable in size (i.e. end whenever separator is encountered).
+  // This is used while decoding and helps in determining where to split.
+  private static final int[] SEGMENT_SIZES = {
+      Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+      Bytes.SIZEOF_LONG, AppIdKeyConverter.getKeySize() };
+
+  /*
+   * (non-Javadoc)
+   *
+   * Encodes ApplicationRowKey object into a byte array with each
+   * component/field in ApplicationRowKey separated by Separator#QUALIFIERS.
+   * This leads to an application table row key of the form
+   * clusterId!userName!flowName!flowRunId!appId
+   * If flowRunId in passed ApplicationRowKey object is null (and the fields
+   * preceding it i.e. clusterId, userId and flowName are not null), this
+   * returns a row key prefix of the form clusterId!userName!flowName! and if
+   * appId in ApplicationRowKey is null (other 4 components are not null), this
+   * returns a row key prefix of the form clusterId!userName!flowName!flowRunId!
+   * flowRunId is inverted while encoding as it helps maintain a descending
+   * order for row keys in application table.
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #encode(java.lang.Object)
+   */
+  @Override
+  public byte[] encode(ApplicationRowKey rowKey) {
+    byte[] cluster = Separator.encode(rowKey.getClusterId(),
+        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
+    byte[] user = Separator.encode(rowKey.getUserId(),
+        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
+    byte[] flow = Separator.encode(rowKey.getFlowName(),
+        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
+    byte[] first = Separator.QUALIFIERS.join(cluster, user, flow);
+    // Note that flowRunId is a long, so we can't encode them all at the same
+    // time.
+    if (rowKey.getFlowRunId() == null) {
+      return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES);
+    }
+    byte[] second = Bytes.toBytes(
+        TimelineStorageUtils.invertLong(rowKey.getFlowRunId()));
+    if (rowKey.getAppId() == null || rowKey.getAppId().isEmpty()) {
+      return Separator.QUALIFIERS.join(first, second, Separator.EMPTY_BYTES);
+    }
+    byte[] third = AppIdKeyConverter.getInstance().encode(rowKey.getAppId());
+    return Separator.QUALIFIERS.join(first, second, third);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * Decodes an application row key of the form
+   * clusterId!userName!flowName!flowRunId!appId represented in byte format and
+   * converts it into an ApplicationRowKey object.flowRunId is inverted while
+   * decoding as it was inverted while encoding.
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #decode(byte[])
+   */
+  @Override
+  public ApplicationRowKey decode(byte[] rowKey) {
+    byte[][] rowKeyComponents =
+        Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
+    if (rowKeyComponents.length != 5) {
+      throw new IllegalArgumentException("the row key is not valid for " +
+          "an application");
+    }
+    String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    String userId = Separator.decode(Bytes.toString(rowKeyComponents[1]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    String flowName = Separator.decode(Bytes.toString(rowKeyComponents[2]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    Long flowRunId =
+        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
+    String appId = AppIdKeyConverter.getInstance().decode(rowKeyComponents[4]);
+    return new ApplicationRowKey(clusterId, userId, flowName, flowRunId, appId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 3085bb1..6a38e32 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKey.java
@@ -17,10 +17,6 @@
  */
 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.
  */
@@ -50,9 +46,8 @@ public class AppToFlowRowKey {
    * @return byte array with the row key
    */
   public static byte[] getRowKey(String clusterId, String appId) {
-    byte[] first = Bytes.toBytes(clusterId);
-    byte[] second = TimelineStorageUtils.encodeAppId(appId);
-    return Separator.QUALIFIERS.join(first, second);
+    return AppToFlowRowKeyConverter.getInstance().encode(
+        new AppToFlowRowKey(clusterId, appId));
   }
 
   /**
@@ -62,15 +57,6 @@ public class AppToFlowRowKey {
    * @return an <cite>AppToFlowRowKey</cite> 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 = TimelineStorageUtils.decodeAppId(rowKeyComponents[1]);
-    return new AppToFlowRowKey(clusterId, appId);
+    return AppToFlowRowKeyConverter.getInstance().decode(rowKey);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKeyConverter.java
----------------------------------------------------------------------
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/AppToFlowRowKeyConverter.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/AppToFlowRowKeyConverter.java
new file mode 100644
index 0000000..0f0b879d
--- /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/AppToFlowRowKeyConverter.java
@@ -0,0 +1,96 @@
+/**
+ * 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.AppIdKeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+
+/**
+ * Encodes and decodes row key for app_flow table.
+ * The row key is of the form : clusterId!appId.
+ * clusterId is a string and appId is encoded/decoded using
+ * {@link AppIdKeyConverter}.
+ */
+public final class AppToFlowRowKeyConverter
+    implements KeyConverter<AppToFlowRowKey> {
+  private static final AppToFlowRowKeyConverter INSTANCE =
+      new AppToFlowRowKeyConverter();
+
+  public static AppToFlowRowKeyConverter getInstance() {
+    return INSTANCE;
+  }
+
+  private AppToFlowRowKeyConverter() {
+  }
+
+  // App to flow row key is of the form clusterId!appId with the 2 segments
+  // separated by !. The sizes below indicate sizes of both of these segments
+  // in sequence. clusterId is a string. appId is represented as 12 bytes with
+  // cluster Timestamp part of appid being 8 bytes(long) and seq id being 4
+  // bytes(int).
+  // Strings are variable in size (i.e. end whenever separator is encountered).
+  // This is used while decoding and helps in determining where to split.
+  private static final int[] SEGMENT_SIZES = {
+      Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT };
+
+  /*
+   * (non-Javadoc)
+   *
+   * Encodes AppToFlowRowKey object into a byte array with each component/field
+   * in AppToFlowRowKey separated by Separator#QUALIFIERS. This leads to an
+   * app to flow table row key of the form clusterId!appId
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #encode(java.lang.Object)
+   */
+  @Override
+  public byte[] encode(AppToFlowRowKey rowKey) {
+    byte[] first = Separator.encode(rowKey.getClusterId(),
+        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
+    byte[] second = AppIdKeyConverter.getInstance().encode(rowKey.getAppId());
+    return Separator.QUALIFIERS.join(first, second);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * Decodes an app to flow row key of the form clusterId!appId represented in
+   * byte format and converts it into an AppToFlowRowKey object.
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #decode(byte[])
+   */
+  @Override
+  public AppToFlowRowKey decode(byte[] rowKey) {
+    byte[][] rowKeyComponents =
+        Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
+    if (rowKeyComponents.length != 2) {
+      throw new IllegalArgumentException("the row key is not valid for " +
+          "the app-to-flow table");
+    }
+    String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    String appId = AppIdKeyConverter.getInstance().decode(rowKeyComponents[1]);
+    return new AppToFlowRowKey(clusterId, appId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
----------------------------------------------------------------------
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/AppIdKeyConverter.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/AppIdKeyConverter.java
new file mode 100644
index 0000000..a173b0f
--- /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/AppIdKeyConverter.java
@@ -0,0 +1,101 @@
+/**
+ * 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.hbase.util.Bytes;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+
+/**
+ * Encodes and decodes {@link ApplicationId} for row keys.
+ * App ID is stored in row key as 12 bytes, cluster timestamp section of app id
+ * (long - 8 bytes) followed by sequence id section of app id (int - 4 bytes).
+ */
+public final class AppIdKeyConverter implements KeyConverter<String> {
+  private static final AppIdKeyConverter INSTANCE = new AppIdKeyConverter();
+
+  public static AppIdKeyConverter getInstance() {
+    return INSTANCE;
+  }
+
+  private AppIdKeyConverter() {
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * 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 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).
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #encode(java.lang.Object)
+   */
+  @Override
+  public byte[] encode(String appIdStr) {
+    ApplicationId appId = ConverterUtils.toApplicationId(appIdStr);
+    byte[] appIdBytes = new byte[getKeySize()];
+    byte[] clusterTs = Bytes.toBytes(
+        TimelineStorageUtils.invertLong(appId.getClusterTimestamp()));
+    System.arraycopy(clusterTs, 0, appIdBytes, 0, Bytes.SIZEOF_LONG);
+    byte[] seqId = Bytes.toBytes(TimelineStorageUtils.invertInt(appId.getId()));
+    System.arraycopy(seqId, 0, appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT);
+    return appIdBytes;
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * 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
+   * ApplicationId#toString to generate string representation of app id.
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #decode(byte[])
+   */
+  @Override
+  public String decode(byte[] appIdBytes) {
+    if (appIdBytes.length != getKeySize()) {
+      throw new IllegalArgumentException("Invalid app id in byte format");
+    }
+    long clusterTs = TimelineStorageUtils.invertLong(
+        Bytes.toLong(appIdBytes, 0, Bytes.SIZEOF_LONG));
+    int seqId = TimelineStorageUtils.invertInt(
+        Bytes.toInt(appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT));
+    return ApplicationId.newInstance(clusterTs, seqId).toString();
+  }
+
+  /**
+   * Returns the size of app id after encoding.
+   *
+   * @return size of app id after encoding.
+   */
+  public static int getKeySize() {
+    return Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 759bf27..be55db5 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
@@ -166,19 +166,22 @@ public class ColumnHelper<T> {
    * @param result from which to reads data with timestamps
    * @param columnPrefixBytes optional prefix to limit columns. If null all
    *          columns are returned.
+   * @param <K> identifies the type of column name(indicated by type of key
+   *     converter).
    * @param <V> the type of the values. The values will be cast into that type.
+   * @param keyConverter used to convert column bytes to the appropriate key
+   *     type.
    * @return the cell values at each respective time in for form
    *         {@literal {idA={timestamp1->value1}, idA={timestamp2->value2},
    *         idB={timestamp3->value3}, idC={timestamp1->value4}}}
    * @throws IOException if any problem occurs while reading results.
    */
   @SuppressWarnings("unchecked")
-  public <V> NavigableMap<String, NavigableMap<Long, V>>
-      readResultsWithTimestamps(Result result, byte[] columnPrefixBytes)
-          throws IOException {
+  public <K, V> NavigableMap<K, NavigableMap<Long, V>>
+      readResultsWithTimestamps(Result result, byte[] columnPrefixBytes,
+          KeyConverter<K> keyConverter) throws IOException {
 
-    NavigableMap<String, NavigableMap<Long, V>> results =
-        new TreeMap<String, NavigableMap<Long, V>>();
+    NavigableMap<K, NavigableMap<Long, V>> results = new TreeMap<>();
 
     if (result != null) {
       NavigableMap<
@@ -192,13 +195,17 @@ public class ColumnHelper<T> {
       if (columnCellMap != null) {
         for (Entry<byte[], NavigableMap<Long, byte[]>> entry : columnCellMap
             .entrySet()) {
-          String columnName = null;
+          K converterColumnKey = null;
           if (columnPrefixBytes == null) {
             if (LOG.isDebugEnabled()) {
               LOG.debug("null prefix was specified; returning all columns");
             }
-            // Decode the spaces we encoded in the column name.
-            columnName = Separator.decode(entry.getKey(), Separator.SPACE);
+            try {
+              converterColumnKey = keyConverter.decode(entry.getKey());
+            } catch (IllegalArgumentException iae) {
+              LOG.error("Illegal column found, skipping this column.", iae);
+              continue;
+            }
           } else {
             // A non-null prefix means columns are actually of the form
             // prefix!columnNameRemainder
@@ -207,13 +214,18 @@ public class ColumnHelper<T> {
             byte[] actualColumnPrefixBytes = columnNameParts[0];
             if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes)
                 && columnNameParts.length == 2) {
-              // This is the prefix that we want
-              columnName = Separator.decode(columnNameParts[1]);
+              try {
+                // This is the prefix that we want
+                converterColumnKey = keyConverter.decode(columnNameParts[1]);
+              } catch (IllegalArgumentException iae) {
+                LOG.error("Illegal column found, skipping this column.", iae);
+                continue;
+              }
             }
           }
 
           // If this column has the prefix we want
-          if (columnName != null) {
+          if (converterColumnKey != null) {
             NavigableMap<Long, V> cellResults =
                 new TreeMap<Long, V>();
             NavigableMap<Long, byte[]> cells = entry.getValue();
@@ -226,7 +238,7 @@ public class ColumnHelper<T> {
                     value);
               }
             }
-            results.put(columnName, cellResults);
+            results.put(converterColumnKey, cellResults);
           }
         } // for entry : columnCellMap
       } // if columnCellMap != null
@@ -235,20 +247,24 @@ public class ColumnHelper<T> {
   }
 
   /**
+   * @param <K> identifies the type of column name(indicated by type of key
+   *     converter).
    * @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. 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.
+   * @param keyConverter used to convert column bytes to the appropriate key
+   *          type.
+   * @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 if any problem occurs while reading results.
    */
-  public Map<String, Object> readResults(Result result,
-      byte[] columnPrefixBytes) throws IOException {
-    Map<String, Object> results = new HashMap<String, Object>();
+  public <K> Map<K, Object> readResults(Result result,
+      byte[] columnPrefixBytes, KeyConverter<K> keyConverter)
+      throws IOException {
+    Map<K, Object> results = new HashMap<K, Object>();
 
     if (result != null) {
       Map<byte[], byte[]> columns = result.getFamilyMap(columnFamilyBytes);
@@ -256,84 +272,40 @@ public class ColumnHelper<T> {
         byte[] columnKey = entry.getKey();
         if (columnKey != null && columnKey.length > 0) {
 
-          String columnName = null;
+          K converterColumnKey = null;
           if (columnPrefixBytes == null) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("null prefix was specified; returning all columns");
+            try {
+              converterColumnKey = keyConverter.decode(columnKey);
+            } catch (IllegalArgumentException iae) {
+              LOG.error("Illegal column found, skipping this column.", iae);
+              continue;
             }
-            // Decode the spaces we encoded in the column name.
-            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(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]);
+            byte[][] columnNameParts = Separator.QUALIFIERS.split(columnKey, 2);
+            if (columnNameParts.length > 0) {
+              byte[] actualColumnPrefixBytes = columnNameParts[0];
+              // If this is the prefix that we want
+              if (Bytes.equals(columnPrefixBytes, actualColumnPrefixBytes)
+                  && columnNameParts.length == 2) {
+                try {
+                  converterColumnKey = keyConverter.decode(columnNameParts[1]);
+                } catch (IllegalArgumentException iae) {
+                  LOG.error("Illegal column found, skipping this column.", iae);
+                  continue;
+                }
+              }
             }
-          }
+          } // if-else
 
-          // If this column has the prefix we want
-          if (columnName != null) {
+          // If the columnPrefix is null (we want all columns), or the actual
+          // prefix matches the given prefix we want this column
+          if (converterColumnKey != null) {
             Object value = converter.decodeValue(entry.getValue());
-            results.put(columnName, value);
-          }
-        }
-      } // for entry
-    }
-    return results;
-  }
-
-  /**
-   * @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 if any problem occurs while reading results.
-   */
-  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 = 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);
-            }
+            // we return the columnQualifier in parts since we don't know
+            // which part is of which data type.
+            results.put(converterColumnKey, value);
           }
         }
       } // for entry
@@ -353,8 +325,9 @@ public class ColumnHelper<T> {
   public static byte[] getColumnQualifier(byte[] columnPrefixBytes,
       String qualifier) {
 
-    // We don't want column names to have spaces
-    byte[] encodedQualifier = Bytes.toBytes(Separator.SPACE.encode(qualifier));
+    // We don't want column names to have spaces / tabs.
+    byte[] encodedQualifier =
+        Separator.encode(qualifier, Separator.SPACE, Separator.TAB);
     if (columnPrefixBytes == null) {
       return encodedQualifier;
     }
@@ -367,22 +340,6 @@ public class ColumnHelper<T> {
   }
 
   /**
-   * Create a compound column qualifier by combining qualifier and components.
-   *
-   * @param qualifier Column QUalifier.
-   * @param components Other components.
-   * @return a byte array representing compound column qualifier.
-   */
-  public static byte[] getCompoundColumnQualifierBytes(String qualifier,
-      byte[]...components) {
-    byte[] colQualBytes = Bytes.toBytes(Separator.VALUES.encode(qualifier));
-    for (int i = 0; i < components.length; i++) {
-      colQualBytes = Separator.VALUES.join(colQualBytes, components[i]);
-    }
-    return colQualBytes;
-  }
-
-  /**
    * @param columnPrefixBytes The byte representation for the column prefix.
    *          Should not contain {@link Separator#QUALIFIERS}.
    * @param qualifier for the remainder of the column.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 e4b7f16..89aa013 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
@@ -91,37 +91,33 @@ public interface ColumnPrefix<T> {
   Object readResult(Result result, String qualifier) throws IOException;
 
   /**
-   * @param result from which to read columns
+   *
+   * @param <K> identifies the type of key converter.
+   * @param result from which to read columns.
+   * @param keyConverter used to convert column bytes to the appropriate key
+   *          type
    * @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 if there is any exception encountered while reading
-   *     results.
+   *           results.
    */
-  Map<String, Object> readResults(Result result) throws IOException;
+  <K> Map<K, Object> readResults(Result result, KeyConverter<K> keyConverter)
+      throws IOException;
 
   /**
-   * @param result from which to reads data with timestamps
+   * @param result from which to reads data with timestamps.
+   * @param <K> identifies the type of key converter.
    * @param <V> the type of the values. The values will be cast into that type.
+   * @param keyConverter used to convert column bytes to the appropriate key
+   *     type.
    * @return the cell values at each respective time in for form
    *         {@literal {idA={timestamp1->value1}, idA={timestamp2->value2},
    *         idB={timestamp3->value3}, idC={timestamp1->value4}}}
    * @throws IOException if there is any exception encountered while reading
    *     result.
    */
-  <V> NavigableMap<String, NavigableMap<Long, V>>
-      readResultsWithTimestamps(Result result) throws IOException;
-
-  /**
-   * @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 if any problem occurs while reading results.
-   */
-  Map<?, Object> readResultsHavingCompoundColumnQualifiers(Result result)
-      throws IOException;
+  <K, V> NavigableMap<K, NavigableMap<Long, V>> readResultsWithTimestamps(
+      Result result, KeyConverter<K> keyConverter) throws IOException;
 
   /**
    * @param qualifierPrefix Column qualifier or prefix of qualifier.
@@ -146,15 +142,4 @@ public interface ColumnPrefix<T> {
    * @return a {@link ValueConverter} implementation.
    */
   ValueConverter getValueConverter();
-
-  /**
-   * Get compound column qualifier bytes if the column qualifier is a compound
-   * qualifier. Returns the qualifier passed as bytes if the column is not a
-   * compound column qualifier.
-   *
-   * @param qualifier Column Qualifier.
-   * @param components Other components.
-   * @return byte array representing compound column qualifier.
-   */
-  byte[] getCompoundColQualBytes(String qualifier, byte[]...components);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java
----------------------------------------------------------------------
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/EventColumnName.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/EventColumnName.java
new file mode 100644
index 0000000..6018f86
--- /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/EventColumnName.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;
+
+/**
+ * Encapsulates information about Event column names for application and entity
+ * tables. Used while encoding/decoding event column names.
+ */
+public class EventColumnName {
+
+  private final String id;
+  private final Long timestamp;
+  private final String infoKey;
+
+  public EventColumnName(String id, Long timestamp, String infoKey) {
+    this.id = id;
+    this.timestamp = timestamp;
+    this.infoKey = infoKey;
+  }
+
+  public String getId() {
+    return id;
+  }
+
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  public String getInfoKey() {
+    return infoKey;
+  }
+
+}


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


[03/50] [abbrv] hadoop git commit: YARN-3863. Support complex filters in TimelineReader (Varun Saxena via sjlee)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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
index d8f73d4..6696ac5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/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
@@ -19,13 +19,8 @@ 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;
@@ -33,28 +28,22 @@ 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.TimelineDataToRetrieve;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.reader.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.ApplicationTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
 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;
@@ -71,7 +60,6 @@ import com.google.common.base.Preconditions;
  */
 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.
@@ -97,92 +85,322 @@ class GenericEntityReader extends TimelineEntityReader {
   }
 
   @Override
-  protected FilterList constructFilterListBasedOnFields() {
-    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
-    TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
-    // Fetch all the columns.
-    if (dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
-        (dataToRetrieve.getConfsToRetrieve() == null ||
-        dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) &&
-        (dataToRetrieve.getMetricsToRetrieve() == null ||
-        dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty())) {
-      return list;
+  protected FilterList constructFilterListBasedOnFilters() throws IOException {
+    // Filters here cannot be null for multiple entity reads as they are set in
+    // augmentParams if null.
+    FilterList listBasedOnFilters = new FilterList();
+    TimelineEntityFilters filters = getFilters();
+    // Create filter list based on created time range and add it to
+    // listBasedOnFilters.
+    long createdTimeBegin = filters.getCreatedTimeBegin();
+    long createdTimeEnd = filters.getCreatedTimeEnd();
+    if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) {
+      listBasedOnFilters.addFilter(
+          TimelineFilterUtils.createSingleColValueFiltersByRange(
+              EntityColumn.CREATED_TIME, createdTimeBegin, createdTimeEnd));
     }
-    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);
+    // Create filter list based on metric filters and add it to
+    // listBasedOnFilters.
+    TimelineFilterList metricFilters = filters.getMetricFilters();
+    if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) {
+      listBasedOnFilters.addFilter(
+          TimelineFilterUtils.createHBaseFilterList(
+              EntityColumnPrefix.METRIC, metricFilters));
+    }
+    // Create filter list based on config filters and add it to
+    // listBasedOnFilters.
+    TimelineFilterList configFilters = filters.getConfigFilters();
+    if (configFilters != null && !configFilters.getFilterList().isEmpty()) {
+      listBasedOnFilters.addFilter(
+          TimelineFilterUtils.createHBaseFilterList(
+              EntityColumnPrefix.CONFIG, configFilters));
+    }
+    // Create filter list based on info filters and add it to listBasedOnFilters
+    TimelineFilterList infoFilters = filters.getInfoFilters();
+    if (infoFilters != null && !infoFilters.getFilterList().isEmpty()) {
+      listBasedOnFilters.addFilter(
+          TimelineFilterUtils.createHBaseFilterList(
+              EntityColumnPrefix.INFO, infoFilters));
+    }
+    return listBasedOnFilters;
+  }
+
+  /**
+   * Check if we need to fetch only some of the event columns.
+   *
+   * @return true if we need to fetch some of the columns, false otherwise.
+   */
+  private static boolean fetchPartialEventCols(TimelineFilterList eventFilters,
+      EnumSet<Field> fieldsToRetrieve) {
+    return (eventFilters != null && !eventFilters.getFilterList().isEmpty() &&
+        !TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS));
+  }
+
+  /**
+   * Check if we need to fetch only some of the relates_to columns.
+   *
+   * @return true if we need to fetch some of the columns, false otherwise.
+   */
+  private static boolean fetchPartialRelatesToCols(TimelineFilterList relatesTo,
+      EnumSet<Field> fieldsToRetrieve) {
+    return (relatesTo != null && !relatesTo.getFilterList().isEmpty() &&
+        !TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO));
+  }
+
+  /**
+   * Check if we need to fetch only some of the is_related_to columns.
+   *
+   * @return true if we need to fetch some of the columns, false otherwise.
+   */
+  private static boolean fetchPartialIsRelatedToCols(
+      TimelineFilterList isRelatedTo, EnumSet<Field> fieldsToRetrieve) {
+    return (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty() &&
+        !TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO));
+  }
+
+  /**
+   * Check if we need to fetch only some of the columns based on event filters,
+   * relatesto and isrelatedto from info family.
+   *
+   * @return true, if we need to fetch only some of the columns, false if we
+   *     need to fetch all the columns under info column family.
+   */
+  protected boolean fetchPartialColsFromInfoFamily() {
+    EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
     TimelineEntityFilters filters = getFilters();
+    return fetchPartialEventCols(filters.getEventFilters(), fieldsToRetrieve) ||
+        fetchPartialRelatesToCols(filters.getRelatesTo(), fieldsToRetrieve) ||
+        fetchPartialIsRelatedToCols(filters.getIsRelatedTo(), fieldsToRetrieve);
+  }
+
+  /**
+   * Check if we need to create filter list based on fields. We need to create
+   * a filter list iff all fields need not be retrieved or we have some specific
+   * fields or metrics to retrieve. We also need to create a filter list if we
+   * have relationships(relatesTo/isRelatedTo) and event filters specified for
+   * the query.
+   *
+   * @return true if we need to create the filter list, false otherwise.
+   */
+  protected boolean needCreateFilterListBasedOnFields() {
+    TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
+    // Check if all fields are to be retrieved or not. If all fields have to
+    // be retrieved, also check if we have some metrics or configs to
+    // retrieve specified for the query because then a filter list will have
+    // to be created.
+    boolean flag = !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) ||
+        (dataToRetrieve.getConfsToRetrieve() != null &&
+        !dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) ||
+        (dataToRetrieve.getMetricsToRetrieve() != null &&
+        !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty());
+    // Filters need to be checked only if we are reading multiple entities. If
+    // condition above is false, we check if there are relationships(relatesTo/
+    // isRelatedTo) and event filters specified for the query.
+    if (!flag && !isSingleEntityRead()) {
+      TimelineEntityFilters filters = getFilters();
+      flag = (filters.getEventFilters() != null &&
+          !filters.getEventFilters().getFilterList().isEmpty()) ||
+          (filters.getIsRelatedTo() != null &&
+          !filters.getIsRelatedTo().getFilterList().isEmpty()) ||
+          (filters.getRelatesTo() != null &&
+          !filters.getRelatesTo().getFilterList().isEmpty());
+    }
+    return flag;
+  }
+
+  /**
+   * Add {@link QualifierFilter} filters to filter list for each column of
+   * entity table.
+   *
+   * @param list filter list to which qualifier filters have to be added.
+   */
+  protected void updateFixedColumns(FilterList list) {
+    for (EntityColumn column : EntityColumn.values()) {
+      list.addFilter(new QualifierFilter(CompareOp.EQUAL,
+          new BinaryComparator(column.getColumnQualifierBytes())));
+    }
+  }
+
+  /**
+   * Creates a filter list which indicates that only some of the column
+   * qualifiers in the info column family will be returned in result.
+   *
+   * @param isApplication If true, it means operations are to be performed for
+   *     application table, otherwise for entity table.
+   * @return filter list.
+   * @throws IOException if any problem occurs while creating filter list.
+   */
+  private FilterList createFilterListForColsOfInfoFamily()
+      throws IOException {
+    FilterList infoFamilyColsFilter = new FilterList(Operator.MUST_PASS_ONE);
+    // Add filters for each column in entity table.
+    updateFixedColumns(infoFamilyColsFilter);
+    EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
+    // If INFO field has to be retrieved, add a filter for fetching columns
+    // with INFO column prefix.
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.EQUAL, EntityColumnPrefix.INFO));
+    }
+    TimelineFilterList relatesTo = getFilters().getRelatesTo();
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
+      // If RELATES_TO field has to be retrieved, add a filter for fetching
+      // columns with RELATES_TO column prefix.
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.EQUAL, EntityColumnPrefix.RELATES_TO));
+    } else if (relatesTo != null && !relatesTo.getFilterList().isEmpty()) {
+      // Even if fields to retrieve does not contain RELATES_TO, we still
+      // need to have a filter to fetch some of the column qualifiers if
+      // relatesTo filters are specified. relatesTo filters will then be
+      // matched after fetching rows from HBase.
+      Set<String> relatesToCols =
+          TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo);
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createFiltersFromColumnQualifiers(
+              EntityColumnPrefix.RELATES_TO, relatesToCols));
+    }
+    TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo();
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
+      // If IS_RELATED_TO field has to be retrieved, add a filter for fetching
+      // columns with IS_RELATED_TO column prefix.
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.EQUAL, EntityColumnPrefix.IS_RELATED_TO));
+    } else if (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty()) {
+      // Even if fields to retrieve does not contain IS_RELATED_TO, we still
+      // need to have a filter to fetch some of the column qualifiers if
+      // isRelatedTo filters are specified. isRelatedTo filters will then be
+      // matched after fetching rows from HBase.
+      Set<String> isRelatedToCols =
+          TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo);
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createFiltersFromColumnQualifiers(
+              EntityColumnPrefix.IS_RELATED_TO, isRelatedToCols));
+    }
+    TimelineFilterList eventFilters = getFilters().getEventFilters();
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
+      // If EVENTS field has to be retrieved, add a filter for fetching columns
+      // with EVENT column prefix.
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.EQUAL, EntityColumnPrefix.EVENT));
+    } else if (eventFilters != null && !eventFilters.getFilterList().isEmpty()){
+      // Even if fields to retrieve does not contain EVENTS, we still need to
+      // have a filter to fetch some of the column qualifiers on the basis of
+      // event filters specified. Event filters will then be matched after
+      // fetching rows from HBase.
+      Set<String> eventCols =
+          TimelineFilterUtils.fetchColumnsFromFilterList(eventFilters);
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createFiltersFromColumnQualifiers(
+              EntityColumnPrefix.EVENT, eventCols));
+    }
+    return infoFamilyColsFilter;
+  }
+
+  /**
+   * Exclude column prefixes via filters which are not required(based on fields
+   * to retrieve) from info column family. These filters are added to filter
+   * list which contains a filter for getting info column family.
+   *
+   * @param infoColFamilyList filter list for info column family.
+   */
+  private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) {
+    EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
     // Events not required.
-    if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.EVENTS) &&
-        !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
-        (isSingleEntityRead() || filters.getEventFilters() == null)) {
+    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
       infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-          EntityColumnPrefix.EVENT.getColumnPrefixBytes(""))));
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.NOT_EQUAL, EntityColumnPrefix.EVENT));
     }
     // info not required.
-    if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.INFO) &&
-        !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
-        (isSingleEntityRead() || filters.getInfoFilters() == null)) {
+    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
       infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-              EntityColumnPrefix.INFO.getColumnPrefixBytes(""))));
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.NOT_EQUAL, EntityColumnPrefix.INFO));
     }
     // is related to not required.
-    if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.IS_RELATED_TO) &&
-        !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
-        (isSingleEntityRead() || filters.getIsRelatedTo() == null)) {
+    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
       infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-              EntityColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes(""))));
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.NOT_EQUAL, EntityColumnPrefix.IS_RELATED_TO));
     }
     // relates to not required.
-    if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.RELATES_TO) &&
-        !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
-        (isSingleEntityRead() || filters.getRelatesTo() == null)) {
+    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
       infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-              EntityColumnPrefix.RELATES_TO.getColumnPrefixBytes(""))));
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.NOT_EQUAL, EntityColumnPrefix.RELATES_TO));
     }
-    list.addFilter(infoColFamilyList);
-    if ((dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS) ||
-        (!isSingleEntityRead() && filters.getConfigFilters() != null)) ||
-        (dataToRetrieve.getConfsToRetrieve() != null &&
-        !dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty())) {
-      FilterList filterCfg =
-          new FilterList(new FamilyFilter(CompareOp.EQUAL,
-              new BinaryComparator(EntityColumnFamily.CONFIGS.getBytes())));
-      if (dataToRetrieve.getConfsToRetrieve() != null &&
-          !dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) {
-        filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList(
-            EntityColumnPrefix.CONFIG, dataToRetrieve.getConfsToRetrieve()));
-      }
-      list.addFilter(filterCfg);
+  }
+
+  /**
+   * Updates filter list based on fields for confs and metrics to retrieve.
+   *
+   * @param listBasedOnFields filter list based on fields.
+   * @throws IOException if any problem occurs while updating filter list.
+   */
+  private void updateFilterForConfsAndMetricsToRetrieve(
+      FilterList listBasedOnFields) throws IOException {
+    TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
+    // Please note that if confsToRetrieve is specified, we would have added
+    // CONFS to fields to retrieve in augmentParams() even if not specified.
+    if (dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS)) {
+      // Create a filter list for configs.
+      listBasedOnFields.addFilter(TimelineFilterUtils.
+          createFilterForConfsOrMetricsToRetrieve(
+              dataToRetrieve.getConfsToRetrieve(),
+              EntityColumnFamily.CONFIGS, EntityColumnPrefix.CONFIG));
     }
-    if ((dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS) ||
-        (!isSingleEntityRead() && filters.getMetricFilters() != null)) ||
-        (dataToRetrieve.getMetricsToRetrieve() != null &&
-        !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty())) {
-      FilterList filterMetrics =
-          new FilterList(new FamilyFilter(CompareOp.EQUAL,
-              new BinaryComparator(EntityColumnFamily.METRICS.getBytes())));
-      if (dataToRetrieve.getMetricsToRetrieve() != null &&
-          !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty()) {
-        filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList(
-            EntityColumnPrefix.METRIC, dataToRetrieve.getMetricsToRetrieve()));
-      }
-      list.addFilter(filterMetrics);
+
+    // Please note that if metricsToRetrieve is specified, we would have added
+    // METRICS to fields to retrieve in augmentParams() even if not specified.
+    if (dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS)) {
+      // Create a filter list for metrics.
+      listBasedOnFields.addFilter(TimelineFilterUtils.
+          createFilterForConfsOrMetricsToRetrieve(
+              dataToRetrieve.getMetricsToRetrieve(),
+              EntityColumnFamily.METRICS, EntityColumnPrefix.METRIC));
+    }
+  }
+
+  @Override
+  protected FilterList constructFilterListBasedOnFields() throws IOException {
+    if (!needCreateFilterListBasedOnFields()) {
+      // Fetch all the columns. No need of a filter.
+      return null;
+    }
+    FilterList listBasedOnFields = new FilterList(Operator.MUST_PASS_ONE);
+    FilterList infoColFamilyList = new FilterList();
+    // By default fetch everything in INFO column family.
+    FamilyFilter infoColumnFamily =
+        new FamilyFilter(CompareOp.EQUAL,
+           new BinaryComparator(EntityColumnFamily.INFO.getBytes()));
+    infoColFamilyList.addFilter(infoColumnFamily);
+    if (!isSingleEntityRead() && fetchPartialColsFromInfoFamily()) {
+      // We can fetch only some of the columns from info family.
+      infoColFamilyList.addFilter(createFilterListForColsOfInfoFamily());
+    } else {
+      // Exclude column prefixes in info column family which are not required
+      // based on fields to retrieve.
+      excludeFieldsFromInfoColFamily(infoColFamilyList);
     }
-    return list;
+    listBasedOnFields.addFilter(infoColFamilyList);
+    updateFilterForConfsAndMetricsToRetrieve(listBasedOnFields);
+    return listBasedOnFields;
   }
 
+  /**
+   * Looks up flow context from AppToFlow table.
+   *
+   * @param clusterId Cluster Id.
+   * @param appId App Id.
+   * @param hbaseConf HBase configuration.
+   * @param conn HBase Connection.
+   * @return flow context information.
+   * @throws IOException if any problem occurs while fetching flow information.
+   */
   protected FlowContext lookupFlowContext(String clusterId, String appId,
       Configuration hbaseConf, Connection conn) throws IOException {
     byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
@@ -200,6 +418,9 @@ class GenericEntityReader extends TimelineEntityReader {
     }
   }
 
+  /**
+   * Encapsulates flow context information.
+   */
   protected static class FlowContext {
     private final String userId;
     private final String flowName;
@@ -222,6 +443,9 @@ class GenericEntityReader extends TimelineEntityReader {
 
   @Override
   protected void validateParams() {
+    Preconditions.checkNotNull(getContext(), "context shouldn't be null");
+    Preconditions.checkNotNull(
+        getDataToRetrieve(), "data to retrieve shouldn't be null");
     Preconditions.checkNotNull(getContext().getClusterId(),
         "clusterId shouldn't be null");
     Preconditions.checkNotNull(getContext().getAppId(),
@@ -241,13 +465,19 @@ class GenericEntityReader extends TimelineEntityReader {
     // In reality all three should be null or neither should be null
     if (context.getFlowName() == null || context.getFlowRunId() == null ||
         context.getUserId() == null) {
+      // Get flow context information from AppToFlow table.
       FlowContext flowContext = lookupFlowContext(
           context.getClusterId(), context.getAppId(), hbaseConf, conn);
       context.setFlowName(flowContext.flowName);
       context.setFlowRunId(flowContext.flowRunId);
       context.setUserId(flowContext.userId);
     }
+    // Add configs/metrics to fields to retrieve if confsToRetrieve and/or
+    // metricsToRetrieve are specified.
     getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve();
+    if (!isSingleEntityRead()) {
+      createFiltersIfNull();
+    }
   }
 
   @Override
@@ -298,215 +528,84 @@ class GenericEntityReader extends TimelineEntityReader {
     // fetch created time
     Number createdTime = (Number)EntityColumn.CREATED_TIME.readResult(result);
     entity.setCreatedTime(createdTime.longValue());
-    if (!isSingleEntityRead() &&
-        (entity.getCreatedTime() < filters.getCreatedTimeBegin() ||
-        entity.getCreatedTime() > filters.getCreatedTimeEnd())) {
-      return null;
-    }
+
     EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
-    // fetch is related to entities
+    // fetch is related to entities and match isRelatedTo filter. If isRelatedTo
+    // filters do not match, entity would be dropped. We have to match filters
+    // locally as relevant HBase filters to filter out rows on the basis of
+    // isRelatedTo are not set in HBase scan.
     boolean checkIsRelatedTo =
-        filters != null && filters.getIsRelatedTo() != null &&
-        filters.getIsRelatedTo().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(), filters.getIsRelatedTo())) {
+        !isSingleEntityRead() && filters.getIsRelatedTo() != null &&
+        filters.getIsRelatedTo().getFilterList().size() > 0;
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO) ||
+        checkIsRelatedTo) {
+      TimelineStorageUtils.readRelationship(
+          entity, result, EntityColumnPrefix.IS_RELATED_TO, true);
+      if (checkIsRelatedTo && !TimelineStorageUtils.matchIsRelatedTo(entity,
+          filters.getIsRelatedTo())) {
         return null;
       }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.IS_RELATED_TO)) {
+      if (!TimelineStorageUtils.hasField(fieldsToRetrieve,
+          Field.IS_RELATED_TO)) {
         entity.getIsRelatedToEntities().clear();
       }
     }
 
-    // fetch relates to entities
+    // fetch relates to entities and match relatesTo filter. If relatesTo
+    // filters do not match, entity would be dropped. We have to match filters
+    // locally as relevant HBase filters to filter out rows on the basis of
+    // relatesTo are not set in HBase scan.
     boolean checkRelatesTo =
-        filters != null && filters.getRelatesTo() != null &&
-        filters.getRelatesTo().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(), filters.getRelatesTo())) {
+        !isSingleEntityRead() && filters.getRelatesTo() != null &&
+        filters.getRelatesTo().getFilterList().size() > 0;
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO) ||
+        checkRelatesTo) {
+      TimelineStorageUtils.readRelationship(
+          entity, result, EntityColumnPrefix.RELATES_TO, false);
+      if (checkRelatesTo && !TimelineStorageUtils.matchRelatesTo(entity,
+          filters.getRelatesTo())) {
         return null;
       }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.RELATES_TO)) {
+      if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
         entity.getRelatesToEntities().clear();
       }
     }
 
-    // fetch info
-    boolean checkInfo = filters != null && filters.getInfoFilters() != null &&
-        filters.getInfoFilters().size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
-      readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false);
-      if (checkInfo &&
-          !TimelineStorageUtils.matchFilters(
-          entity.getInfo(), filters.getInfoFilters())) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.INFO)) {
-        entity.getInfo().clear();
-      }
+    // fetch info if fieldsToRetrieve contains INFO or ALL.
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
+      TimelineStorageUtils.readKeyValuePairs(
+          entity, result, EntityColumnPrefix.INFO, false);
     }
 
-    // fetch configs
-    boolean checkConfigs =
-        filters != null && filters.getConfigFilters() != null &&
-        filters.getConfigFilters().size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
-      readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true);
-      if (checkConfigs && !TimelineStorageUtils.matchFilters(
-          entity.getConfigs(), filters.getConfigFilters())) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.CONFIGS)) {
-        entity.getConfigs().clear();
-      }
+    // fetch configs if fieldsToRetrieve contains CONFIGS or ALL.
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.CONFIGS)) {
+      TimelineStorageUtils.readKeyValuePairs(
+          entity, result, EntityColumnPrefix.CONFIG, true);
     }
 
-    // fetch events
+    // fetch events and match event filters if they exist. If event filters do
+    // not match, entity would be dropped. We have to match filters locally
+    // as relevant HBase filters to filter out rows on the basis of events
+    // are not set in HBase scan.
     boolean checkEvents =
-        filters != null && filters.getEventFilters() != null &&
-        filters.getEventFilters().size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
-      readEvents(entity, result, false);
-      if (checkEvents && !TimelineStorageUtils.matchEventFilters(
-          entity.getEvents(), filters.getEventFilters())) {
+        !isSingleEntityRead() && filters.getEventFilters() != null &&
+        filters.getEventFilters().getFilterList().size() > 0;
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS) ||
+        checkEvents) {
+      TimelineStorageUtils.readEvents(entity, result, EntityColumnPrefix.EVENT);
+      if (checkEvents && !TimelineStorageUtils.matchEventFilters(entity,
+          filters.getEventFilters())) {
         return null;
       }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.EVENTS)) {
+      if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
         entity.getEvents().clear();
       }
     }
 
-    // fetch metrics
-    boolean checkMetrics =
-        filters != null && filters.getMetricFilters() != null &&
-        filters.getMetricFilters().size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
+    // fetch metrics if fieldsToRetrieve contains METRICS or ALL.
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.METRICS)) {
       readMetrics(entity, result, EntityColumnPrefix.METRIC);
-      if (checkMetrics && !TimelineStorageUtils.matchMetricFilters(
-          entity.getMetrics(), filters.getMetricFilters())) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.METRICS)) {
-        entity.getMetrics().clear();
-      }
     }
     return entity;
   }
-
-  /**
-   * Helper method for reading relationship.
-   *
-   * @param <T> Describes the type of column prefix.
-   * @param entity entity to fill.
-   * @param result result from HBase.
-   * @param prefix column prefix.
-   * @param isRelatedTo if true, means relationship is to be added to
-   *     isRelatedTo, otherwise its added to relatesTo.
-   * @throws IOException if any problem is encountered while reading result.
-   */
-  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.
-   *
-   * @param <T> Describes the type of column prefix.
-   * @param entity entity to fill.
-   * @param result result from HBase.
-   * @param prefix column prefix.
-   * @param isConfig if true, means we are reading configs, otherwise info.
-   * @throws IOException if any problem is encountered while reading result.
-   */
-  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.
-   *
-   * @param entity entity to fill.
-   * @param result HBase Result.
-   * @param isApplication if true, event read is for application table,
-   *     otherwise its being read for entity table.
-   * @throws IOException if any problem is encountered while reading result.
-   *
-   * 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/366eb54e/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
index 281e901..4299de9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/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
@@ -107,11 +107,60 @@ public abstract class TimelineEntityReader {
   /**
    * 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.
+   * results fetched from HBase back-end storage. This is called only for
+   * multiple entity reads.
    *
    * @return a {@link FilterList} object.
+   * @throws IOException if any problem occurs while creating filter list.
    */
-  protected abstract FilterList constructFilterListBasedOnFields();
+  protected abstract FilterList constructFilterListBasedOnFields()
+      throws IOException;
+
+  /**
+   * Creates a {@link FilterList} based on info, config and metric filters. This
+   * filter list will be set in HBase Get to trim down results fetched from
+   * HBase back-end storage.
+   *
+   * @return a {@link FilterList} object.
+   * @throws IOException if any problem occurs while creating filter list.
+   */
+  protected abstract FilterList constructFilterListBasedOnFilters()
+      throws IOException;
+
+  /**
+   * Combines filter lists created based on fields and based on filters.
+   *
+   * @return a {@link FilterList} object if it can be constructed. Returns null,
+   * if filter list cannot be created either on the basis of filters or on the
+   * basis of fields.
+   * @throws IOException if any problem occurs while creating filter list.
+   */
+  private FilterList createFilterList() throws IOException {
+    FilterList listBasedOnFilters = constructFilterListBasedOnFilters();
+    boolean hasListBasedOnFilters = listBasedOnFilters != null &&
+        !listBasedOnFilters.getFilters().isEmpty();
+    FilterList listBasedOnFields = constructFilterListBasedOnFields();
+    boolean hasListBasedOnFields = listBasedOnFields != null &&
+        !listBasedOnFields.getFilters().isEmpty();
+    // If filter lists based on both filters and fields can be created,
+    // combine them in a new filter list and return it.
+    // If either one of them has been created, return that filter list.
+    // Return null, if none of the filter lists can be created. This indicates
+    // that no filter list needs to be added to HBase Scan as filters are not
+    // specified for the query or only the default view of entity needs to be
+    // returned.
+    if (hasListBasedOnFilters && hasListBasedOnFields) {
+      FilterList list = new FilterList();
+      list.addFilter(listBasedOnFilters);
+      list.addFilter(listBasedOnFields);
+      return list;
+    } else if (hasListBasedOnFilters) {
+      return listBasedOnFilters;
+    } else if (hasListBasedOnFields) {
+      return listBasedOnFields;
+    }
+    return null;
+  }
 
   protected TimelineReaderContext getContext() {
     return context;
@@ -126,6 +175,16 @@ public abstract class TimelineEntityReader {
   }
 
   /**
+   * Create a {@link TimelineEntityFilters} object with default values for
+   * filters.
+   */
+  protected void createFiltersIfNull() {
+    if (filters == null) {
+      filters = new TimelineEntityFilters();
+    }
+  }
+
+  /**
    * Reads and deserializes a single timeline entity from the HBase storage.
    *
    * @param hbaseConf HBase Configuration.
@@ -140,6 +199,9 @@ public abstract class TimelineEntityReader {
     augmentParams(hbaseConf, conn);
 
     FilterList filterList = constructFilterListBasedOnFields();
+    if (LOG.isDebugEnabled() && filterList != null) {
+      LOG.debug("FilterList created for get is - " + filterList);
+    }
     Result result = getResult(hbaseConf, conn, filterList);
     if (result == null || result.isEmpty()) {
       // Could not find a matching row.
@@ -166,7 +228,10 @@ public abstract class TimelineEntityReader {
     augmentParams(hbaseConf, conn);
 
     NavigableSet<TimelineEntity> entities = new TreeSet<>();
-    FilterList filterList = constructFilterListBasedOnFields();
+    FilterList filterList = createFilterList();
+    if (LOG.isDebugEnabled() && filterList != null) {
+      LOG.debug("FilterList created for scan is - " + filterList);
+    }
     ResultScanner results = getResults(hbaseConf, conn, filterList);
     try {
       for (Result result : results) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 b6e23a9..2bd2830 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
@@ -221,7 +221,7 @@ public class TestTimelineReaderWebServices {
       assertTrue("UID should be present",
           entity.getInfo().containsKey(TimelineReaderManager.UID_KEY));
       // Includes UID.
-      assertEquals(2, entity.getInfo().size());
+      assertEquals(3, entity.getInfo().size());
       // No events will be returned as events are not part of fields.
       assertEquals(0, entity.getEvents().size());
     } finally {
@@ -247,7 +247,7 @@ public class TestTimelineReaderWebServices {
       assertTrue("UID should be present",
           entity.getInfo().containsKey(TimelineReaderManager.UID_KEY));
       // Includes UID.
-      assertEquals(2, entity.getInfo().size());
+      assertEquals(3, entity.getInfo().size());
       assertEquals(2, entity.getEvents().size());
     } finally {
       client.destroy();
@@ -443,10 +443,8 @@ public class TestTimelineReaderWebServices {
           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")) &&
+      assertEquals(1, entities.size());
+      assertTrue("Entity with id_3 should have been present in response.",
           entities.contains(newEntity("app", "id_3")));
     } finally {
       client.destroy();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 a8a2ff8..23d64e0 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
@@ -23,6 +23,7 @@ import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -40,6 +41,13 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+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.util.timeline.TimelineUtils;
 import org.junit.AfterClass;
@@ -112,6 +120,7 @@ public class TestFileSystemTimelineReaderImpl {
     entity11.setCreatedTime(1425016502000L);
     Map<String, Object> info1 = new HashMap<String, Object>();
     info1.put("info1", "val1");
+    info1.put("info2", "val5");
     entity11.addInfo(info1);
     TimelineEvent event = new TimelineEvent();
     event.setId("event_1");
@@ -121,7 +130,7 @@ public class TestFileSystemTimelineReaderImpl {
     TimelineMetric metric1 = new TimelineMetric();
     metric1.setId("metric1");
     metric1.setType(TimelineMetric.Type.SINGLE_VALUE);
-    metric1.addValue(1425016502006L, 113.2F);
+    metric1.addValue(1425016502006L, 113);
     metrics.add(metric1);
     TimelineMetric metric2 = new TimelineMetric();
     metric2.setId("metric2");
@@ -130,7 +139,7 @@ public class TestFileSystemTimelineReaderImpl {
     metrics.add(metric2);
     entity11.setMetrics(metrics);
     Map<String,String> configs = new HashMap<String, String>();
-    configs.put("config_1", "123");
+    configs.put("config_1", "127");
     entity11.setConfigs(configs);
     entity11.addRelatesToEntity("flow", "flow1");
     entity11.addIsRelatedToEntity("type1", "tid1_1");
@@ -171,7 +180,7 @@ public class TestFileSystemTimelineReaderImpl {
     info1.put("info2", 4);
     entity2.addInfo(info2);
     Map<String,String> configs2 = new HashMap<String, String>();
-    configs2.put("config_1", "123");
+    configs2.put("config_1", "129");
     configs2.put("config_3", "def");
     entity2.setConfigs(configs2);
     TimelineEvent event2 = new TimelineEvent();
@@ -182,7 +191,7 @@ public class TestFileSystemTimelineReaderImpl {
     TimelineMetric metric21 = new TimelineMetric();
     metric21.setId("metric1");
     metric21.setType(TimelineMetric.Type.SINGLE_VALUE);
-    metric21.addValue(1425016501006L, 123.2F);
+    metric21.addValue(1425016501006L, 300);
     metrics2.add(metric21);
     TimelineMetric metric22 = new TimelineMetric();
     metric22.setId("metric2");
@@ -205,6 +214,7 @@ public class TestFileSystemTimelineReaderImpl {
     entity3.setCreatedTime(1425016501050L);
     Map<String, Object> info3 = new HashMap<String, Object>();
     info3.put("info2", 3.5);
+    info3.put("info4", 20);
     entity3.addInfo(info3);
     Map<String,String> configs3 = new HashMap<String, String>();
     configs3.put("config_1", "123");
@@ -222,7 +232,7 @@ public class TestFileSystemTimelineReaderImpl {
     TimelineMetric metric31 = new TimelineMetric();
     metric31.setId("metric1");
     metric31.setType(TimelineMetric.Type.SINGLE_VALUE);
-    metric31.addValue(1425016501006L, 124.8F);
+    metric31.addValue(1425016501006L, 124);
     metrics3.add(metric31);
     TimelineMetric metric32 = new TimelineMetric();
     metric32.setId("metric2");
@@ -317,7 +327,7 @@ public class TestFileSystemTimelineReaderImpl {
     Assert.assertEquals(1425016502000L, result.getCreatedTime());
     Assert.assertEquals(3, result.getConfigs().size());
     Assert.assertEquals(3, result.getMetrics().size());
-    Assert.assertEquals(1, result.getInfo().size());
+    Assert.assertEquals(2, result.getInfo().size());
     // No events will be returned
     Assert.assertEquals(0, result.getEvents().size());
   }
@@ -344,8 +354,8 @@ public class TestFileSystemTimelineReaderImpl {
     Set<TimelineEntity> result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null), new TimelineEntityFilters(),
-        new TimelineDataToRetrieve());
-    // All 3 entities will be returned
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    // All 4 entities will be returned
     Assert.assertEquals(4, result.size());
   }
 
@@ -425,12 +435,13 @@ public class TestFileSystemTimelineReaderImpl {
   @Test
   public void testGetFilteredEntities() throws Exception {
     // Get entities based on info filters.
-    Map<String, Object> infoFilters = new HashMap<String, Object>();
-    infoFilters.put("info2", 3.5);
+    TimelineFilterList infoFilterList = new TimelineFilterList();
+    infoFilterList.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "info2", 3.5));
     Set<TimelineEntity> result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
-        new TimelineEntityFilters(null, null, null, null, null, infoFilters,
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList,
         null, null, null),
         new TimelineDataToRetrieve());
     Assert.assertEquals(1, result.size());
@@ -442,26 +453,30 @@ public class TestFileSystemTimelineReaderImpl {
     }
 
     // Get entities based on config filters.
-    Map<String, String> configFilters = new HashMap<String, String>();
-    configFilters.put("config_1", "123");
-    configFilters.put("config_3", "abc");
+    TimelineFilterList confFilterList = new TimelineFilterList();
+    confFilterList.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_1", "123"));
+    confFilterList.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_3", "abc"));
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
-        configFilters, null, null),
+        confFilterList, null, null),
         new TimelineDataToRetrieve());
-    Assert.assertEquals(2, result.size());
+    Assert.assertEquals(1, result.size());
     for (TimelineEntity entity : result) {
-      if (!entity.getId().equals("id_1") && !entity.getId().equals("id_3")) {
+      if (!entity.getId().equals("id_3")) {
         Assert.fail("Incorrect filtering based on config filters");
       }
     }
 
     // Get entities based on event filters.
-    Set<String> eventFilters = new HashSet<String>();
-    eventFilters.add("event_2");
-    eventFilters.add("event_4");
+    TimelineFilterList eventFilters = new TimelineFilterList();
+    eventFilters.addFilter(
+        new TimelineExistsFilter(TimelineCompareOp.EQUAL,"event_2"));
+    eventFilters.addFilter(
+        new TimelineExistsFilter(TimelineCompareOp.EQUAL,"event_4"));
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
@@ -476,13 +491,14 @@ public class TestFileSystemTimelineReaderImpl {
     }
 
     // Get entities based on metric filters.
-    Set<String> metricFilters = new HashSet<String>();
-    metricFilters.add("metric3");
+    TimelineFilterList metricFilterList = new TimelineFilterList();
+    metricFilterList.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_OR_EQUAL, "metric3", 0L));
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilters, null),
+        metricFilterList, null),
         new TimelineDataToRetrieve());
     Assert.assertEquals(2, result.size());
     // Two entities with IDs' id_1 and id_2 should be returned.
@@ -491,15 +507,266 @@ public class TestFileSystemTimelineReaderImpl {
         Assert.fail("Incorrect filtering based on metric filters");
       }
     }
-  }
+
+    // Get entities based on complex config filters.
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_1", "129"));
+    list1.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_3", "def"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_2", "23"));
+    list2.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_3", "abc"));
+    TimelineFilterList confFilterList1 =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList1, null, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(2, result.size());
+    for (TimelineEntity entity : result) {
+      if (!entity.getId().equals("id_1") && !entity.getId().equals("id_2")) {
+        Assert.fail("Incorrect filtering based on config filters");
+      }
+    }
+
+    TimelineFilterList list3 = new TimelineFilterList();
+    list3.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "config_1", "123"));
+    list3.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "config_3", "abc"));
+    TimelineFilterList list4 = new TimelineFilterList();
+    list4.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "config_2", "23"));
+    TimelineFilterList confFilterList2 =
+        new TimelineFilterList(Operator.OR, list3, list4);
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList2, null, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(2, result.size());
+    for (TimelineEntity entity : result) {
+      if (!entity.getId().equals("id_1") && !entity.getId().equals("id_2")) {
+        Assert.fail("Incorrect filtering based on config filters");
+      }
+    }
+
+    TimelineFilterList confFilterList3 = new TimelineFilterList();
+    confFilterList3.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "config_1", "127"));
+    confFilterList3.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "config_3", "abc"));
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList3, null, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(1, result.size());
+    for(TimelineEntity entity : result) {
+      if (!entity.getId().equals("id_2")) {
+        Assert.fail("Incorrect filtering based on config filters");
+      }
+    }
+
+    TimelineFilterList confFilterList4 = new TimelineFilterList();
+    confFilterList4.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_dummy", "dummy"));
+    confFilterList4.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_3", "def"));
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList4, null, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(0, result.size());
+
+    TimelineFilterList confFilterList5 = new TimelineFilterList(Operator.OR);
+    confFilterList5.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_dummy", "dummy"));
+    confFilterList5.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_3", "def"));
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList5, null, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(1, result.size());
+    for (TimelineEntity entity : result) {
+      if (!entity.getId().equals("id_2")) {
+        Assert.fail("Incorrect filtering based on config filters");
+      }
+    }
+
+    // Get entities based on complex metric filters.
+    TimelineFilterList list6 = new TimelineFilterList();
+    list6.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_THAN, "metric1", 200));
+    list6.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.EQUAL, "metric3", 23));
+    TimelineFilterList list7 = new TimelineFilterList();
+    list7.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_OR_EQUAL, "metric2", 74));
+    TimelineFilterList metricFilterList1 =
+        new TimelineFilterList(Operator.OR, list6, list7);
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList1, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(2, result.size());
+    // Two entities with IDs' id_2 and id_3 should be returned.
+    for (TimelineEntity entity : result) {
+      if (!entity.getId().equals("id_2") && !entity.getId().equals("id_3")) {
+        Assert.fail("Incorrect filtering based on metric filters");
+      }
+    }
+
+    TimelineFilterList metricFilterList2 = new TimelineFilterList();
+    metricFilterList2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "metric2", 70));
+    metricFilterList2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_OR_EQUAL, "metric3", 23));
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList2, null),
+        new TimelineDataToRetrieve());
+   Assert.assertEquals(1, result.size());
+    for (TimelineEntity entity : result) {
+      if (!entity.getId().equals("id_1")) {
+        Assert.fail("Incorrect filtering based on metric filters");
+      }
+    }
+
+    TimelineFilterList metricFilterList3 = new TimelineFilterList();
+    metricFilterList3.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "dummy_metric", 30));
+    metricFilterList3.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_OR_EQUAL, "metric3", 23));
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList3, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(0, result.size());
+
+    TimelineFilterList metricFilterList4 = new TimelineFilterList(Operator.OR);
+    metricFilterList4.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "dummy_metric", 30));
+    metricFilterList4.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_OR_EQUAL, "metric3", 23));
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList4, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(2, result.size());
+    for (TimelineEntity entity : result) {
+      if (!entity.getId().equals("id_1") && !entity.getId().equals("id_2")) {
+        Assert.fail("Incorrect filtering based on metric filters");
+      }
+    }
+
+    TimelineFilterList metricFilterList5 =
+        new TimelineFilterList(new TimelineCompareFilter(
+            TimelineCompareOp.NOT_EQUAL, "metric2", 74));
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList5, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(2, result.size());
+    for (TimelineEntity entity : result) {
+      if (!entity.getId().equals("id_1") && !entity.getId().equals("id_2")) {
+        Assert.fail("Incorrect filtering based on metric filters");
+      }
+    }
+
+    TimelineFilterList infoFilterList1 = new TimelineFilterList();
+    infoFilterList1.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "info2", 3.5));
+    infoFilterList1.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL, "info4", 20));
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList1,
+        null, null, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(0, result.size());
+
+    TimelineFilterList infoFilterList2 = new TimelineFilterList(Operator.OR);
+    infoFilterList2.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "info2", 3.5));
+    infoFilterList2.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "info1", "val1"));
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList2,
+        null, null, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(2, result.size());
+    for (TimelineEntity entity : result) {
+      if (!entity.getId().equals("id_1") && !entity.getId().equals("id_3")) {
+        Assert.fail("Incorrect filtering based on info filters");
+      }
+    }
+
+    TimelineFilterList infoFilterList3 = new TimelineFilterList();
+    infoFilterList3.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "dummy_info", 1));
+    infoFilterList3.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "info2", "val5"));
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList3,
+        null, null, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(0, result.size());
+
+    TimelineFilterList infoFilterList4 = new TimelineFilterList(Operator.OR);
+    infoFilterList4.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "dummy_info", 1));
+    infoFilterList4.addFilter(
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL, "info2", "val5"));
+    result = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
+        "app", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList4,
+        null, null, null),
+        new TimelineDataToRetrieve());
+    Assert.assertEquals(1, result.size());
+    for (TimelineEntity entity : result) {
+      if (!entity.getId().equals("id_1")) {
+        Assert.fail("Incorrect filtering based on info filters");
+      }
+    }
+   }
 
   @Test
   public void testGetEntitiesByRelations() throws Exception {
     // Get entities based on relatesTo.
-    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
-    Set<String> relatesToIds = new HashSet<String>();
-    relatesToIds.add("flow1");
-    relatesTo.put("flow", relatesToIds);
+    TimelineFilterList relatesTo = new TimelineFilterList(Operator.OR);
+    Set<Object> relatesToIds =
+        new HashSet<Object>(Arrays.asList((Object)"flow1"));
+    relatesTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "flow", relatesToIds));
     Set<TimelineEntity> result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),
@@ -515,10 +782,11 @@ public class TestFileSystemTimelineReaderImpl {
     }
 
     // Get entities based on isRelatedTo.
-    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
-    Set<String> isRelatedToIds = new HashSet<String>();
-    isRelatedToIds.add("tid1_2");
-    isRelatedTo.put("type1", isRelatedToIds);
+    TimelineFilterList isRelatedTo = new TimelineFilterList(Operator.OR);
+    Set<Object> isRelatedToIds =
+        new HashSet<Object>(Arrays.asList((Object)"tid1_2"));
+    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "type1", isRelatedToIds));
     result = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "flow1", 1L, "app1",
         "app", null),


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


[22/50] [abbrv] hadoop git commit: YARN-5045. hbase unit tests fail due to dependency issues. (Sangjin Lee via varunsaxena)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/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-hbase-tests/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-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
new file mode 100644
index 0000000..8ab54bc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -0,0 +1,3461 @@
+/**
+ * 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.Arrays;
+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.TimelineMetricOperation;
+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.reader.TimelineDataToRetrieve;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
+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.reader.filter.TimelineKeyValuesFilter;
+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.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 org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+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;
+  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 = 1425016502000L;
+    entity.setCreatedTime(cTime);
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<String, Object>();
+    infoMap.put("infoMapKey1", "infoMapValue2");
+    infoMap.put("infoMapKey2", 20);
+    infoMap.put("infoMapKey3", 85.85);
+    entity.addInfo(infoMap);
+    // add the isRelatedToEntity info
+    Set<String> isRelatedToSet = new HashSet<String>();
+    isRelatedToSet.add("relatedto1");
+    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
+    isRelatedTo.put("task", isRelatedToSet);
+    entity.setIsRelatedToEntities(isRelatedTo);
+    // add the relatesTo info
+    Set<String> relatesToSet = new HashSet<String>();
+    relatesToSet.add("relatesto1");
+    relatesToSet.add("relatesto3");
+    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
+    relatesTo.put("container", relatesToSet);
+    Set<String> relatesToSet11 = new HashSet<String>();
+    relatesToSet11.add("relatesto4");
+    relatesTo.put("container1", relatesToSet11);
+    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("start_event");
+    event.setTimestamp(ts);
+    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 + 20L);
+
+    // 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
+    Set<String> isRelatedToSet1 = new HashSet<String>();
+    isRelatedToSet1.add("relatedto3");
+    isRelatedToSet1.add("relatedto5");
+    Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
+    isRelatedTo1.put("task1", isRelatedToSet1);
+    Set<String> isRelatedToSet11 = new HashSet<String>();
+    isRelatedToSet11.add("relatedto4");
+    isRelatedTo1.put("task2", isRelatedToSet11);
+    entity1.setIsRelatedToEntities(isRelatedTo1);
+
+    // add the relatesTo info
+    Set<String> relatesToSet1 = new HashSet<String>();
+    relatesToSet1.add("relatesto1");
+    relatesToSet1.add("relatesto2");
+    Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
+    relatesTo1.put("container", 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);
+    TimelineEvent event11 = new TimelineEvent();
+    event11.setId("end_event");
+    event11.setTimestamp(ts);
+    entity1.addEvent(event11);
+    TimelineEvent event12 = new TimelineEvent();
+    event12.setId("update_event");
+    event12.setTimestamp(ts - 10);
+    entity1.addEvent(event12);
+    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 + 40L);
+    TimelineEvent event21 = new TimelineEvent();
+    event21.setId("update_event");
+    event21.setTimestamp(ts - 20);
+    entity2.addEvent(event21);
+    Set<String> isRelatedToSet2 = new HashSet<String>();
+    isRelatedToSet2.add("relatedto3");
+    Map<String, Set<String>> isRelatedTo2 = new HashMap<String, Set<String>>();
+    isRelatedTo2.put("task1", isRelatedToSet2);
+    entity2.setIsRelatedToEntities(isRelatedTo2);
+    Map<String, Set<String>> relatesTo3 = new HashMap<String, Set<String>>();
+    Set<String> relatesToSet14 = new HashSet<String>();
+    relatesToSet14.add("relatesto7");
+    relatesTo3.put("container2", relatesToSet14);
+    entity2.setRelatesToEntities(relatesTo3);
+
+    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 = 1425016502000L;
+    entity.setCreatedTime(cTime);
+    // add the info map in Timeline Entity
+    Map<String, Object> infoMap = new HashMap<String, Object>();
+    infoMap.put("infoMapKey1", "infoMapValue2");
+    infoMap.put("infoMapKey2", 20);
+    infoMap.put("infoMapKey3", 71.4);
+    entity.addInfo(infoMap);
+    // add the isRelatedToEntity info
+    Set<String> isRelatedToSet = new HashSet<String>();
+    isRelatedToSet.add("relatedto1");
+    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
+    isRelatedTo.put("task", isRelatedToSet);
+    entity.setIsRelatedToEntities(isRelatedTo);
+
+    // add the relatesTo info
+    Set<String> relatesToSet = new HashSet<String>();
+    relatesToSet.add("relatesto1");
+    relatesToSet.add("relatesto3");
+    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
+    relatesTo.put("container", relatesToSet);
+    Set<String> relatesToSet11 = new HashSet<String>();
+    relatesToSet11.add("relatesto4");
+    relatesTo.put("container1", relatesToSet11);
+    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, 70000000000L);
+    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("start_event");
+    event.setTimestamp(ts);
+    entity.addEvent(event);
+    te.addEntity(entity);
+
+    TimelineEntity entity1 = new TimelineEntity();
+    String id1 = "hello1";
+    entity1.setId(id1);
+    entity1.setType(type);
+    entity1.setCreatedTime(cTime + 20L);
+
+    // 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 event.
+    TimelineEvent event11 = new TimelineEvent();
+    event11.setId("end_event");
+    event11.setTimestamp(ts);
+    entity1.addEvent(event11);
+    TimelineEvent event12 = new TimelineEvent();
+    event12.setId("update_event");
+    event12.setTimestamp(ts - 10);
+    entity1.addEvent(event12);
+
+
+    // add the isRelatedToEntity info
+    Set<String> isRelatedToSet1 = new HashSet<String>();
+    isRelatedToSet1.add("relatedto3");
+    isRelatedToSet1.add("relatedto5");
+    Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
+    isRelatedTo1.put("task1", isRelatedToSet1);
+    Set<String> isRelatedToSet11 = new HashSet<String>();
+    isRelatedToSet11.add("relatedto4");
+    isRelatedTo1.put("task2", isRelatedToSet11);
+    entity1.setIsRelatedToEntities(isRelatedTo1);
+
+    // add the relatesTo info
+    Set<String> relatesToSet1 = new HashSet<String>();
+    relatesToSet1.add("relatesto1");
+    relatesToSet1.add("relatesto2");
+    Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
+    relatesTo1.put("container", 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 + 40L);
+    TimelineEvent event21 = new TimelineEvent();
+    event21.setId("update_event");
+    event21.setTimestamp(ts - 20);
+    entity2.addEvent(event21);
+    Set<String> isRelatedToSet2 = new HashSet<String>();
+    isRelatedToSet2.add("relatedto3");
+    Map<String, Set<String>> isRelatedTo2 = new HashMap<String, Set<String>>();
+    isRelatedTo2.put("task1", isRelatedToSet2);
+    entity2.setIsRelatedToEntities(isRelatedTo2);
+    Map<String, Set<String>> relatesTo3 = new HashMap<String, Set<String>>();
+    Set<String> relatesToSet14 = new HashSet<String>();
+    relatesToSet14.add("relatesto7");
+    relatesTo3.put("container2", relatesToSet14);
+    entity2.setRelatesToEntities(relatesTo3);
+    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()) {
+      Number val = m1.get(entry.getKey());
+      assertNotNull(val);
+      assertEquals(val.longValue(), entry.getValue().longValue());
+    }
+  }
+
+  @Test
+  public void testWriteApplicationToHBase() throws Exception {
+    TimelineEntities te = new TimelineEntities();
+    ApplicationEntity entity = new ApplicationEntity();
+    String appId = "application_1000178881110_2002";
+    entity.setId(appId);
+    long cTime = 1425016501000L;
+    entity.setCreatedTime(cTime);
+
+    // 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);
+
+    // add aggregated metrics
+    TimelineEntity aggEntity = new TimelineEntity();
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    aggEntity.setId(appId);
+    aggEntity.setType(type);
+    long cTime2 = 1425016502000L;
+    long mTime2 = 1425026902000L;
+    aggEntity.setCreatedTime(cTime2);
+
+    TimelineMetric aggMetric = new TimelineMetric();
+    aggMetric.setId("MEM_USAGE");
+    Map<Long, Number> aggMetricValues = new HashMap<Long, Number>();
+    ts = System.currentTimeMillis();
+    aggMetricValues.put(ts - 120000, 102400000);
+    aggMetric.setType(Type.SINGLE_VALUE);
+    aggMetric.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
+    aggMetric.setValues(aggMetricValues);
+    Set<TimelineMetric> aggMetrics = new HashSet<>();
+    aggMetrics.add(aggMetric);
+    entity.addMetrics(aggMetrics);
+    te.addEntity(entity);
+
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.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, appId, te);
+      hbi.stop();
+
+      // retrieve the row
+      byte[] rowKey =
+          ApplicationRowKey.getRowKey(cluster, user, flow, runid, appId);
+      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,
+          appId));
+
+      // check info column family
+      String id1 = ApplicationColumn.ID.readResult(result).toString();
+      assertEquals(appId, id1);
+
+      Number val =
+          (Number) ApplicationColumn.CREATED_TIME.readResult(result);
+      long cTime1 = val.longValue();
+      assertEquals(cTime1, cTime);
+
+      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());
+      matchMetrics(metricValues, metricMap);
+
+      // read the timeline entity using the reader this time
+      TimelineEntity e1 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, appId,
+          entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(
+          null, null, EnumSet.of(TimelineReader.Field.ALL)));
+      assertNotNull(e1);
+
+      // verify attributes
+      assertEquals(appId, e1.getId());
+      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
+          e1.getType());
+      assertEquals(cTime, e1.getCreatedTime());
+      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(2, metrics2.size());
+      for (TimelineMetric metric2 : metrics2) {
+        Map<Long, Number> metricValues2 = metric2.getValues();
+        assertTrue(metric2.getId().equals("MAP_SLOT_MILLIS") ||
+            metric2.getId().equals("MEM_USAGE"));
+        if (metric2.getId().equals("MAP_SLOT_MILLIS")) {
+          matchMetrics(metricValues, metricValues2);
+        }
+        if (metric2.getId().equals("MEM_USAGE")) {
+          matchMetrics(aggMetricValues, metricValues2);
+        }
+      }
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.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;
+    entity.setCreatedTime(cTime);
+
+    // 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;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      String cluster = "cluster_test_write_entity";
+      String user = "user1";
+      String flow = "some_flow_name";
+      String flowVersion = "AB7822C10F1111";
+      long runid = 1002345678919L;
+      String appName =
+          ApplicationId.newInstance(System.currentTimeMillis(), 1).toString();
+      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);
+
+          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());
+          matchMetrics(metricValues, metricMap);
+        }
+      }
+      assertEquals(1, rowCount);
+      assertEquals(16, colCount);
+
+      // read the timeline entity using the reader this time
+      TimelineEntity e1 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, appName,
+          entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+      Set<TimelineEntity> es1 = reader.getEntities(
+          new TimelineReaderContext(cluster, user, flow, runid, appName,
+          entity.getType(), null),
+          new TimelineEntityFilters(),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+      assertNotNull(e1);
+      assertEquals(1, es1.size());
+
+      // verify attributes
+      assertEquals(id, e1.getId());
+      assertEquals(type, e1.getType());
+      assertEquals(cTime, e1.getCreatedTime());
+      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();
+        matchMetrics(metricValues, metricValues2);
+      }
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
+  private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user,
+      String flow, long runid, String appName, TimelineEntity te) {
+
+    EntityRowKey key = EntityRowKey.parseRowKey(rowKey);
+
+    assertEquals(user, key.getUserId());
+    assertEquals(cluster, key.getClusterId());
+    assertEquals(flow, key.getFlowName());
+    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) {
+
+    ApplicationRowKey key = ApplicationRowKey.parseRowKey(rowKey);
+
+    assertEquals(cluster, key.getClusterId());
+    assertEquals(user, key.getUserId());
+    assertEquals(flow, key.getFlowName());
+    assertEquals(runid, key.getFlowRunId());
+    assertEquals(appName, key.getAppId());
+    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;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      String cluster = "cluster_test_events";
+      String user = "user2";
+      String flow = "other_flow_name";
+      String flowVersion = "1111F01C2287BA";
+      long runid = 1009876543218L;
+      String appName = "application_123465899910_1001";
+      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(
+            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
+        assertEquals(expVal, value.toString());
+      }
+
+      // read the timeline entity using the reader this time
+      TimelineEntity e1 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, appName,
+          entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+      TimelineEntity e2 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, null, null, appName,
+          entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+      assertNotNull(e1);
+      assertNotNull(e2);
+      assertEquals(e1, e2);
+
+      // 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();
+      }
+    }
+  }
+
+  @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;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      String cluster = "cluster_test_empty_eventkey";
+      String user = "user_emptyeventkey";
+      String flow = "other_flow_name";
+      String flowVersion = "1111F01C2287BA";
+      long runid = 1009876543218L;
+      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);
+      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(TimelineStorageUtils.invertLong(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 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, appName,
+          entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+      Set<TimelineEntity> es1 = reader.getEntities(
+          new TimelineReaderContext(cluster, user, flow, runid, appName,
+          entity.getType(), null),
+          new TimelineEntityFilters(),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(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();
+    }
+  }
+
+  @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);
+    // 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);
+    // 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();
+      }
+    }
+  }
+
+  @Test
+  public void testReadEntities() throws Exception {
+    TimelineEntity entity = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", "hello"),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertNotNull(entity);
+    assertEquals(3, entity.getConfigs().size());
+    assertEquals(1, entity.getIsRelatedToEntities().size());
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world",
+        null), new TimelineEntityFilters(),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(3, entities.size());
+    int cfgCnt = 0;
+    int metricCnt = 0;
+    int infoCnt = 0;
+    int eventCnt = 0;
+    int relatesToCnt = 0;
+    int isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      cfgCnt += (timelineEntity.getConfigs() == null) ? 0 :
+          timelineEntity.getConfigs().size();
+      metricCnt += (timelineEntity.getMetrics() == null) ? 0 :
+          timelineEntity.getMetrics().size();
+      infoCnt += (timelineEntity.getInfo() == null) ? 0 :
+          timelineEntity.getInfo().size();
+      eventCnt += (timelineEntity.getEvents() == null) ? 0 :
+          timelineEntity.getEvents().size();
+      relatesToCnt += (timelineEntity.getRelatesToEntities() == null) ? 0 :
+          timelineEntity.getRelatesToEntities().size();
+      isRelatedToCnt += (timelineEntity.getIsRelatedToEntities() == null) ? 0 :
+          timelineEntity.getIsRelatedToEntities().size();
+    }
+    assertEquals(5, cfgCnt);
+    assertEquals(3, metricCnt);
+    assertEquals(5, infoCnt);
+    assertEquals(4, eventCnt);
+    assertEquals(4, relatesToCnt);
+    assertEquals(4, isRelatedToCnt);
+  }
+
+  @Test
+  public void testFilterEntitiesByCreatedTime() throws Exception {
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, 1425016502000L, 1425016502040L, null,
+        null, null, null, null, null), new TimelineDataToRetrieve());
+    assertEquals(3, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("hello") && !entity.getId().equals("hello1") &&
+          !entity.getId().equals("hello2")) {
+        Assert.fail("Entities with ids' hello, hello1 and hello2 should be" +
+           " present");
+      }
+    }
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, 1425016502015L, null, null, null, null,
+        null, null, null), new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("hello1") &&
+          !entity.getId().equals("hello2")) {
+        Assert.fail("Entities with ids' hello1 and hello2 should be present");
+      }
+    }
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, 1425016502015L, null, null, null,
+        null, null, null), new TimelineDataToRetrieve());
+     assertEquals(1, entities.size());
+     for (TimelineEntity entity : entities) {
+       if (!entity.getId().equals("hello")) {
+         Assert.fail("Entity with id hello should be present");
+       }
+     }
+  }
+
+  @Test
+  public void testReadEntitiesRelationsAndEventFiltersDefaultView()
+      throws Exception {
+    TimelineFilterList eventFilter = new TimelineFilterList();
+    eventFilter.addFilter(new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL,
+        "end_event"));
+    TimelineFilterList relatesTo = new TimelineFilterList(Operator.OR);
+    relatesTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    relatesTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList isRelatedTo = new TimelineFilterList();
+    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, relatesTo, isRelatedTo,
+        null, null, null, eventFilter), new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    int eventCnt = 0;
+    int isRelatedToCnt = 0;
+    int relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity id should have been hello2");
+      }
+    }
+    assertEquals(0, eventCnt);
+    assertEquals(0, isRelatedToCnt);
+    assertEquals(0, relatesToCnt);
+  }
+
+  @Test
+  public void testReadEntitiesEventFilters() throws Exception {
+    TimelineFilterList ef = new TimelineFilterList();
+    ef.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    ef.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(1, entities.size());
+    int eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity id should have been hello2");
+      }
+    }
+    assertEquals(1, eventCnt);
+
+    TimelineFilterList ef1 = new TimelineFilterList();
+    ef1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    ef1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef1),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity id should have been hello2");
+      }
+    }
+    assertEquals(0, eventCnt);
+
+    TimelineFilterList ef2 = new TimelineFilterList();
+    ef2.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef2),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity ids' should have been hello and hello2");
+      }
+    }
+    assertEquals(0, eventCnt);
+
+    TimelineFilterList ef3 = new TimelineFilterList();
+    ef3.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    ef3.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "dummy_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef3),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    list1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "dummy_event"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "start_event"));
+    TimelineFilterList ef4 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef4),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello")) {
+        Assert.fail("Entity id should have been hello");
+      }
+    }
+    assertEquals(0, eventCnt);
+
+    TimelineFilterList ef5 = new TimelineFilterList();
+    ef5.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "update_event"));
+    ef5.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef5),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello")) {
+          Assert.fail("Entity id should have been hello");
+        }
+    }
+    assertEquals(0, eventCnt);
+  }
+
+  @Test
+  public void testReadEntitiesIsRelatedTo() throws Exception {
+    TimelineFilterList irt = new TimelineFilterList(Operator.OR);
+    irt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    irt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, irt, null, null, null,
+        null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(2, entities.size());
+    int isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity ids' should have been hello and hello1");
+      }
+    }
+    assertEquals(3, isRelatedToCnt);
+
+    TimelineFilterList irt1 = new TimelineFilterList();
+    irt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    irt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt1, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity id should have been hello2");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt2 = new TimelineFilterList(Operator.OR);
+    irt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    irt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt2, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity ids' should have been hello and hello1");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt3 = new TimelineFilterList();
+    irt3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt3, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity id should have been hello1");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt4 = new TimelineFilterList();
+    irt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    irt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_task",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt4, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList irt5 = new TimelineFilterList();
+    irt5.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto7"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt5, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_task",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    TimelineFilterList irt6 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt6, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity id should have been hello1");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+  }
+
+  @Test
+  public void testReadEntitiesRelatesTo() throws Exception {
+    TimelineFilterList rt = new TimelineFilterList(Operator.OR);
+    rt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    rt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, rt, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(2, entities.size());
+    int relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity ids' should have been hello and hello2");
+      }
+    }
+    assertEquals(3, relatesToCnt);
+
+    TimelineFilterList rt1 = new TimelineFilterList();
+    rt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    rt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt1, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity id should have been hello1");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt2 = new TimelineFilterList(Operator.OR);
+    rt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    rt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt2, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity ids' should have been hello and hello2");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt3 = new TimelineFilterList();
+    rt3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1", "relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt3, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello")) {
+        Assert.fail("Entity id should have been hello");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt4 = new TimelineFilterList();
+    rt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    rt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_container",
+        new HashSet<Object>(Arrays.asList("relatesto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt4, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList rt5 = new TimelineFilterList();
+    rt5.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatedto1", "relatesto8"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt5, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_container",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList rt6 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt6, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello")) {
+        Assert.fail("Entity id should have been hello");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList list3 = new TimelineFilterList();
+    list3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    list3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList list4 = new TimelineFilterList();
+    list4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    list4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto2"))));
+    TimelineFilterList combinedList =
+        new TimelineFilterList(Operator.OR, list3, list4);
+    TimelineFilterList rt7 = new TimelineFilterList(Operator.AND, combinedList,
+        new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt7, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity id should have been hello1");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+  }
+
+  @Test
+  public void testReadEntitiesDefaultView() throws Exception {
+    TimelineEntity e1 = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", "hello"),
+        new TimelineDataToRetrieve());
+    assertNotNull(e1);
+    assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() &&
+        e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() &&
+        e1.getRelatesToEntities().isEmpty());
+    Set<TimelineEntity> es1 = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve());
+    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(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", "hello"),
+        new TimelineDataToRetrieve(
+        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(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve(
+        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(4, 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(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", "hello"),
+        new TimelineDataToRetrieve(list, null, null));
+    assertNotNull(e1);
+    assertEquals(1, e1.getConfigs().size());
+    Set<TimelineEntity> es1 = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve(list, null, null));
+    int cfgCnt = 0;
+    for (TimelineEntity entity : es1) {
+      cfgCnt += entity.getConfigs().size();
+      for (String confKey : entity.getConfigs().keySet()) {
+        assertTrue("Config key returned should start with cfg_",
+            confKey.startsWith("cfg_"));
+      }
+    }
+    assertEquals(3, cfgCnt);
+  }
+
+  @Test
+  public void testReadEntitiesConfigFilters() throws Exception {
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param2", "value2"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value3"));
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_param2", "value2"));
+    TimelineFilterList confFilterList =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(2, entities.size());
+    int cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(5, cfgCnt);
+
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(2, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(5, cfgCnt);
+
+    TimelineFilterList confFilterList1 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList1, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(1, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(3, cfgCnt);
+
+    TimelineFilterList confFilterList2 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"),
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "config_param2", "value2"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList2, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList confFilterList3 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "dummy_config", "value1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList3, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList confFilterList4 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1"));
+    entities = reader.getEntities(
+            new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+            1002345678919L, "application_1231111111_1111","world", null),
+            new TimelineEntityFilters(null, null, null, null, null, null,
+            confFilterList4, null, null),
+            new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList confFilterList5 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1", false));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList5, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(3, entities.size());
+  }
+
+  @Test
+  public void testReadEntitiesConfigFilterPrefix() throws Exception {
+    TimelineFilterList confFilterList = new TimelineFilterList();
+    confFilterList.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(list, null, null));
+    assertEquals(1, entities.size());
+    int cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+      for (String confKey : entity.getConfigs().keySet()) {
+        assertTrue("Config key returned should start with cfg_",
+            confKey.startsWith("cfg_"));
+      }
+    }
+    assertEquals(2, cfgCnt);
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param2", "value2"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value3"));
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_param2", "value2"));
+    TimelineFilterList confFilterList1 =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    TimelineFilterList confsToRetrieve =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "config_"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList1, null, null),
+        new TimelineDataToRetrieve(confsToRetrieve, null, null));
+    assertEquals(2, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+      for (String confKey : entity.getConfigs().keySet()) {
+        assertTrue("Config key returned should start with config_",
+            confKey.startsWith("config_"));
+       }
+    }
+    assertEquals(2, cfgCnt);
+  }
+
+  @Test
+  public void testReadEntitiesMetricPrefix() throws Exception {
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    TimelineEntity e1 = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", "hello"),
+        new TimelineDataToRetrieve(null, list, null));
+    assertNotNull(e1);
+    assertEquals(1, e1.getMetrics().size());
+    Set<TimelineEntity> es1 = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(),
+        new TimelineDataToRetrieve(null, list, null));
+    int metricCnt = 0;
+    for (TimelineEntity entity : es1) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
+    }
+    assertEquals(2, metricCnt);
+  }
+
+  @Test
+  public void testReadEntitiesMetricFilters() throws Exception {
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 50000000900L));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 80000000000L));
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.EQUAL, "MAP1_BYTES", 50));
+    TimelineFilterList metricFilterList =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+    assertEquals(2, entities.size());
+    int metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(3, metricCnt);
+
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(2, entities.size());
+    metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(3, metricCnt);
+
+    TimelineFilterList metricFilterList1 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.LESS_OR_EQUAL, "MAP_SLOT_MILLIS", 80000000000L),
+        new TimelineCompareFilter(
+        Timel

<TRUNCATED>

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


[17/50] [abbrv] hadoop git commit: YARN-5045. hbase unit tests fail due to dependency issues. (Sangjin Lee via varunsaxena)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/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
deleted file mode 100644
index 58d5e61..0000000
--- 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
+++ /dev/null
@@ -1,161 +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.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);
-
-    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, 0L, 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/ccdec4a1/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
deleted file mode 100644
index 3b8036d..0000000
--- 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
+++ /dev/null
@@ -1,383 +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.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;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Generates the data/entities for the FlowRun and FlowActivity Tables
- */
-class TestFlowDataGenerator {
-
-  private static final String metric1 = "MAP_SLOT_MILLIS";
-  private static final String metric2 = "HDFS_BYTES_READ";
-  public static final long END_TS_INCR = 10000L;
-
-  static TimelineEntity getEntityMetricsApp1(long insertTs, Configuration c1) {
-    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 = insertTs;
-
-    for (int k=1; k< 100 ; k++) {
-    metricValues.put(ts - k*200000, 20L);
-    }
-    metricValues.put(ts - 80000, 40L);
-    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();
-    for (int k=1; k< 100 ; k++) {
-      metricValues.put(ts - k*100000, 31L);
-    }
-
-    metricValues.put(ts - 80000, 57L);
-    m2.setType(Type.TIME_SERIES);
-    m2.setValues(metricValues);
-    metrics.add(m2);
-
-    entity.addMetrics(metrics);
-    return entity;
-  }
-
-
-  static TimelineEntity getEntityMetricsApp1Complete(long insertTs, Configuration c1) {
-    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 = insertTs;
-
-    metricValues.put(ts - 80000, 40L);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-
-    TimelineMetric m2 = new TimelineMetric();
-    m2.setId(metric2);
-    metricValues = new HashMap<Long, Number>();
-    ts = insertTs;
-    metricValues.put(ts - 80000, 57L);
-    m2.setType(Type.TIME_SERIES);
-    m2.setValues(metricValues);
-    metrics.add(m2);
-
-    entity.addMetrics(metrics);
-
-    TimelineEvent event = new TimelineEvent();
-    event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
-    event.setTimestamp(insertTs);
-    event.addInfo("done", "insertTs=" + insertTs);
-    entity.addEvent(event);
-    return entity;
-  }
-
-
-  static TimelineEntity getEntityMetricsApp1(long insertTs) {
-    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 = insertTs;
-    metricValues.put(ts - 100000, 2L);
-    metricValues.put(ts - 80000, 40L);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-
-    TimelineMetric m2 = new TimelineMetric();
-    m2.setId(metric2);
-    metricValues = new HashMap<Long, Number>();
-    ts = insertTs;
-    metricValues.put(ts - 100000, 31L);
-    metricValues.put(ts - 80000, 57L);
-    m2.setType(Type.TIME_SERIES);
-    m2.setValues(metricValues);
-    metrics.add(m2);
-
-    entity.addMetrics(metrics);
-    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 getEntityMetricsApp2(long insertTs) {
-    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 = insertTs;
-    metricValues.put(ts - 100000, 5L);
-    metricValues.put(ts - 80000, 101L);
-    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 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 getEntity1() {
-    TimelineEntity entity = new TimelineEntity();
-    String id = "flowRunHello";
-    String type = TimelineEntityType.YARN_APPLICATION.toString();
-    entity.setId(id);
-    entity.setType(type);
-    long cTime = 1425026901000L;
-    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 - 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);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    entity.addMetrics(metrics);
-
-    TimelineEvent event = new TimelineEvent();
-    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    event.setTimestamp(cTime);
-    String expKey = "foo_event";
-    Object expVal = "test";
-    event.addInfo(expKey, expVal);
-    entity.addEvent(event);
-
-    event = new TimelineEvent();
-    event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
-    long expTs = cTime + 21600000;// start time + 6hrs
-    event.setTimestamp(expTs);
-    event.addInfo(expKey, expVal);
-    entity.addEvent(event);
-
-    return entity;
-  }
-
-  static TimelineEntity getAFullEntity(long ts, long endTs) {
-    TimelineEntity entity = new TimelineEntity();
-    String id = "flowRunFullEntity";
-    String type = TimelineEntityType.YARN_APPLICATION.toString();
-    entity.setId(id);
-    entity.setType(type);
-    entity.setCreatedTime(ts);
-    // add metrics
-    Set<TimelineMetric> metrics = new HashSet<>();
-    TimelineMetric m1 = new TimelineMetric();
-    m1.setId(metric1);
-    Map<Long, Number> metricValues = new HashMap<Long, Number>();
-    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);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    TimelineMetric m2 = new TimelineMetric();
-    m2.setId(metric2);
-    metricValues = new HashMap<Long, Number>();
-    metricValues.put(ts - 900000, 31L);
-    metricValues.put(ts - 30000, 57L);
-    m2.setType(Type.TIME_SERIES);
-    m2.setValues(metricValues);
-    metrics.add(m2);
-    entity.addMetrics(metrics);
-
-    TimelineEvent event = new TimelineEvent();
-    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    event.setTimestamp(ts);
-    String expKey = "foo_event";
-    Object expVal = "test";
-    event.addInfo(expKey, expVal);
-    entity.addEvent(event);
-
-    event = new TimelineEvent();
-    event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
-    long expTs = ts + 21600000;// start time + 6hrs
-    event.setTimestamp(expTs);
-    event.addInfo(expKey, expVal);
-    entity.addEvent(event);
-
-    return entity;
-  }
-
-  static TimelineEntity getEntityGreaterStartTime(long startTs) {
-    TimelineEntity entity = new TimelineEntity();
-    entity.setCreatedTime(startTs);
-    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);
-    event.setTimestamp(startTs);
-    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(long startTs) {
-    TimelineEntity entity = new TimelineEntity();
-    String id = "flowRunHelloMInStartTime";
-    String type = TimelineEntityType.YARN_APPLICATION.toString();
-    entity.setId(id);
-    entity.setType(type);
-    entity.setCreatedTime(startTs);
-    TimelineEvent event = new TimelineEvent();
-    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    event.setTimestamp(startTs);
-    entity.addEvent(event);
-    return entity;
-  }
-
-  static TimelineEntity getMinFlushEntity(long startTs) {
-    TimelineEntity entity = new TimelineEntity();
-    String id = "flowRunHelloFlushEntityMin";
-    String type = TimelineEntityType.YARN_APPLICATION.toString();
-    entity.setId(id);
-    entity.setType(type);
-    entity.setCreatedTime(startTs);
-    TimelineEvent event = new TimelineEvent();
-    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    event.setTimestamp(startTs);
-    entity.addEvent(event);
-    return entity;
-  }
-
-  static TimelineEntity getMaxFlushEntity(long startTs) {
-    TimelineEntity entity = new TimelineEntity();
-    String id = "flowRunHelloFlushEntityMax";
-    String type = TimelineEntityType.YARN_APPLICATION.toString();
-    entity.setId(id);
-    entity.setType(type);
-    entity.setCreatedTime(startTs);
-
-    TimelineEvent event = new TimelineEvent();
-    event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
-    event.setTimestamp(startTs + END_TS_INCR);
-    entity.addEvent(event);
-    return entity;
-  }
-
-  static TimelineEntity getFlowApp1(long appCreatedTime) {
-    TimelineEntity entity = new TimelineEntity();
-    String id = "flowActivity_test";
-    String type = TimelineEntityType.YARN_APPLICATION.toString();
-    entity.setId(id);
-    entity.setType(type);
-    entity.setCreatedTime(appCreatedTime);
-
-    TimelineEvent event = new TimelineEvent();
-    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    event.setTimestamp(appCreatedTime);
-    String expKey = "foo_event";
-    Object expVal = "test";
-    event.addInfo(expKey, expVal);
-    entity.addEvent(event);
-
-    return entity;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/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
deleted file mode 100644
index 6b23b6c..0000000
--- 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
+++ /dev/null
@@ -1,469 +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.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.Map;
-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.TableName;
-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.FlowActivityEntity;
-import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
-import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
-import org.apache.hadoop.yarn.server.timelineservice.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.common.ColumnHelper;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
-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 minStartTs = 1424995200300L;
-    long greaterStartTs = 1424995200300L + 864000L;
-    long endTs = 1424995200300L + 86000000L;;
-    TimelineEntity entityMinStartTime = TestFlowDataGenerator
-        .getEntityMinStartTime(minStartTs);
-
-    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(greaterStartTs);
-      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, minStartTs, 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.getFlowName());
-    long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(minStartTs);
-    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(
-          new TimelineReaderContext(cluster, null, null, null, null,
-          TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
-          new TimelineEntityFilters(10L, null, null, null, null, null,
-          null, null, null),
-          new TimelineDataToRetrieve());
-      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();
-    }
-  }
-
-  /**
-   * 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();
-    long appCreatedTime = 1425016501000L;
-    TimelineEntity entityApp1 =
-        TestFlowDataGenerator.getFlowApp1(appCreatedTime);
-    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,
-        appCreatedTime);
-
-    // use the reader to verify the data
-    HBaseTimelineReaderImpl hbr = null;
-    try {
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
-
-      Set<TimelineEntity> entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, user, flow, null, null,
-          TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
-          new TimelineEntityFilters(10L, null, null, null, null, null,
-          null, null, null),
-          new TimelineDataToRetrieve());
-      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, long appCreatedTime)
-          throws IOException {
-    Scan s = new Scan();
-    s.addFamily(FlowActivityColumnFamily.INFO.getBytes());
-    byte[] startRow =
-        FlowActivityRowKey.getRowKey(cluster, appCreatedTime, user, flow);
-    s.setStartRow(startRow);
-    String clusterStop = cluster + "1";
-    byte[] stopRow =
-        FlowActivityRowKey.getRowKey(clusterStop, appCreatedTime, 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.getFlowName());
-      long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(appCreatedTime);
-      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();
-    long appCreatedTime = 1425016501000L;
-    TimelineEntity entityApp1 =
-        TestFlowDataGenerator.getFlowApp1(appCreatedTime);
-    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, appCreatedTime);
-
-    // use the timeline reader to verify data
-    HBaseTimelineReaderImpl hbr = null;
-    try {
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
-
-      Set<TimelineEntity> entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, null, null, null, null,
-          TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
-          new TimelineEntityFilters(10L, null, null, null, null, null,
-          null, null, null),
-          new TimelineDataToRetrieve());
-      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 =
-            TimelineStorageUtils.getTopOfTheDayTimestamp(appCreatedTime);
-        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,
-      long appCreatedTime)
-      throws IOException {
-    Scan s = new Scan();
-    s.addFamily(FlowActivityColumnFamily.INFO.getBytes());
-    byte[] startRow =
-        FlowActivityRowKey.getRowKey(cluster, appCreatedTime, user, flow);
-    s.setStartRow(startRow);
-    String clusterStop = cluster + "1";
-    byte[] stopRow =
-        FlowActivityRowKey.getRowKey(clusterStop, appCreatedTime, 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.getFlowName());
-      long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(appCreatedTime);
-      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/ccdec4a1/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
deleted file mode 100644
index 801d43c..0000000
--- 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
+++ /dev/null
@@ -1,851 +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.flow;
-
-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 static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-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.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-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.TimelineMetric;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
-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.TimelineFilterList.Operator;
-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.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
-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);
-  }
-
-  @Test
-  public void checkCoProcessorOff() throws IOException, InterruptedException {
-    Configuration hbaseConf = util.getConfiguration();
-    TableName table = TableName.valueOf(hbaseConf.get(
-        FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
-    Connection conn = null;
-    conn = ConnectionFactory.createConnection(hbaseConf);
-    Admin admin = conn.getAdmin();
-    if (admin == null) {
-      throw new IOException("Can't check tables since admin is null");
-    }
-    if (admin.tableExists(table)) {
-      // check the regions.
-      // check in flow run table
-      util.waitUntilAllRegionsAssigned(table);
-      HRegionServer server = util.getRSForFirstRegionInTable(table);
-      List<HRegion> regions = server.getOnlineRegions(table);
-      for (HRegion region : regions) {
-        assertTrue(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
-            hbaseConf));
-      }
-    }
-
-    table = TableName.valueOf(hbaseConf.get(
-        FlowActivityTable.TABLE_NAME_CONF_NAME,
-        FlowActivityTable.DEFAULT_TABLE_NAME));
-    if (admin.tableExists(table)) {
-      // check the regions.
-      // check in flow activity table
-      util.waitUntilAllRegionsAssigned(table);
-      HRegionServer server = util.getRSForFirstRegionInTable(table);
-      List<HRegion> regions = server.getOnlineRegions(table);
-      for (HRegion region : regions) {
-        assertFalse(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
-            hbaseConf));
-      }
-    }
-
-    table = TableName.valueOf(hbaseConf.get(
-        EntityTable.TABLE_NAME_CONF_NAME,
-        EntityTable.DEFAULT_TABLE_NAME));
-    if (admin.tableExists(table)) {
-      // check the regions.
-      // check in entity run table
-      util.waitUntilAllRegionsAssigned(table);
-      HRegionServer server = util.getRSForFirstRegionInTable(table);
-      List<HRegion> regions = server.getOnlineRegions(table);
-      for (HRegion region : regions) {
-        assertFalse(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
-            hbaseConf));
-      }
-    }
-  }
-
-  /**
-   * 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 minStartTs = 1425026900000L;
-    long greaterStartTs = 30000000000000L;
-    long endTs = 1439750690000L;
-    TimelineEntity entityMinStartTime = TestFlowDataGenerator
-        .getEntityMinStartTime(minStartTs);
-
-    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(greaterStartTs);
-      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 = Bytes.toLong(values.get(
-        FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes()));
-    assertEquals(minStartTs, starttime);
-    assertEquals(endTs, Bytes.toLong(values
-        .get(FlowRunColumn.MAX_END_TIME.getColumnQualifierBytes())));
-
-    // 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(
-          new TimelineReaderContext(cluster, user, flow, runid, null,
-          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineDataToRetrieve());
-      assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
-      FlowRunEntity flowRun = (FlowRunEntity)entity;
-      assertEquals(minStartTs, flowRun.getStartTime());
-      assertEquals(endTs, flowRun.getMaxEndTime());
-    } finally {
-      hbr.close();
-    }
-  }
-
-  /**
-   * 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(System.currentTimeMillis());
-    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(System.currentTimeMillis());
-      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();
-      TimelineEntity entity = hbr.getEntity(
-          new TimelineReaderContext(cluster, user, flow, runid, null,
-          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineDataToRetrieve());
-      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(141L, value);
-          break;
-        case metric2:
-          assertEquals(57L, value);
-          break;
-        default:
-          fail("unrecognized metric: " + id);
-        }
-      }
-    } finally {
-      hbr.close();
-    }
-  }
-
-  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(141L, Bytes.toLong(values.get(q)));
-
-      // check metric2
-      assertEquals(3, values.size());
-      q = ColumnHelper.getColumnQualifier(
-          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric2);
-      assertTrue(values.containsKey(q));
-      assertEquals(57L, Bytes.toLong(values.get(q)));
-    }
-    assertEquals(1, rowCount);
-  }
-
-  @Test
-  public void testWriteFlowRunMetricsPrefix() throws Exception {
-    String cluster = "testWriteFlowRunMetricsPrefix_cluster1";
-    String user = "testWriteFlowRunMetricsPrefix_user1";
-    String flow = "testWriteFlowRunMetricsPrefix_flow_name";
-    String flowVersion = "CF7022C10F1354";
-
-    TimelineEntities te = new TimelineEntities();
-    TimelineEntity entityApp1 = TestFlowDataGenerator
-        .getEntityMetricsApp1(System.currentTimeMillis());
-    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, 1002345678919L, appName, te);
-      // write another application with same metric to this flow
-      te = new TimelineEntities();
-      TimelineEntity entityApp2 = TestFlowDataGenerator
-          .getEntityMetricsApp2(System.currentTimeMillis());
-      te.addEntity(entityApp2);
-      appName = "application_11111111111111_2222";
-      hbi.write(cluster, user, flow, flowVersion, 1002345678918L, appName, te);
-      hbi.flush();
-    } finally {
-      hbi.close();
-    }
-
-    // use the timeline reader to verify data
-    HBaseTimelineReaderImpl hbr = null;
-    try {
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
-      TimelineFilterList metricsToRetrieve = new TimelineFilterList(
-          Operator.OR, new TimelinePrefixFilter(TimelineCompareOp.EQUAL,
-              metric1.substring(0, metric1.indexOf("_") + 1)));
-      TimelineEntity entity = hbr.getEntity(
-          new TimelineReaderContext(cluster, user, flow, 1002345678919L, null,
-          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineDataToRetrieve(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(40L, value);
-          break;
-        default:
-          fail("unrecognized metric: " + id);
-        }
-      }
-
-      Set<TimelineEntity> entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, user, flow, null, null,
-          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(),
-          new TimelineDataToRetrieve(null, metricsToRetrieve, null));
-      assertEquals(2, entities.size());
-      int metricCnt = 0;
-      for (TimelineEntity timelineEntity : entities) {
-        metricCnt += timelineEntity.getMetrics().size();
-      }
-      assertEquals(2, metricCnt);
-    } finally {
-      hbr.close();
-    }
-  }
-
-  @Test
-  public void testWriteFlowRunsMetricFields() throws Exception {
-    String cluster = "testWriteFlowRunsMetricFields_cluster1";
-    String user = "testWriteFlowRunsMetricFields_user1";
-    String flow = "testWriteFlowRunsMetricFields_flow_name";
-    String flowVersion = "CF7022C10F1354";
-    long runid = 1002345678919L;
-
-    TimelineEntities te = new TimelineEntities();
-    TimelineEntity entityApp1 = TestFlowDataGenerator
-        .getEntityMetricsApp1(System.currentTimeMillis());
-    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(System.currentTimeMillis());
-      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(
-          new TimelineReaderContext(cluster, user, flow, runid, null,
-          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(),
-          new TimelineDataToRetrieve());
-      assertEquals(1, entities.size());
-      for (TimelineEntity timelineEntity : entities) {
-        assertEquals(0, timelineEntity.getMetrics().size());
-      }
-
-      entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, user, flow, runid, null,
-          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
-      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();
-    }
-  }
-
-  @Test
-  public void testWriteFlowRunFlush() throws Exception {
-    String cluster = "atestFlushFlowRun_cluster1";
-    String user = "atestFlushFlowRun__user1";
-    String flow = "atestFlushFlowRun_flow_name";
-    String flowVersion = "AF1021C19F1351";
-    long runid = 1449526652000L;
-
-    int start = 10;
-    int count = 20000;
-    int appIdSuffix = 1;
-    HBaseTimelineWriterImpl hbi = null;
-    long insertTs = 1449796654827L - count;
-    long minTS = insertTs + 1;
-    long startTs = insertTs;
-    Configuration c1 = util.getConfiguration();
-    TimelineEntities te1 = null;
-    TimelineEntity entityApp1 = null;
-    TimelineEntity entityApp2 = null;
-    try {
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-
-      for (int i = start; i < count; i++) {
-        String appName = "application_1060350000000_" + appIdSuffix;
-        insertTs++;
-        te1 = new TimelineEntities();
-        entityApp1 = TestFlowDataGenerator.getMinFlushEntity(insertTs);
-        te1.addEntity(entityApp1);
-        entityApp2 = TestFlowDataGenerator.getMaxFlushEntity(insertTs);
-        te1.addEntity(entityApp2);
-        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
-        Thread.sleep(1);
-
-        appName = "application_1001199480000_7" + appIdSuffix;
-        insertTs++;
-        appIdSuffix++;
-        te1 = new TimelineEntities();
-        entityApp1 = TestFlowDataGenerator.getMinFlushEntity(insertTs);
-        te1.addEntity(entityApp1);
-        entityApp2 = TestFlowDataGenerator.getMaxFlushEntity(insertTs);
-        te1.addEntity(entityApp2);
-
-        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
-        if (i % 1000 == 0) {
-          hbi.flush();
-          checkMinMaxFlush(c1, minTS, startTs, count, cluster, user, flow,
-              runid, false);
-        }
-      }
-    } finally {
-      hbi.flush();
-      hbi.close();
-      checkMinMaxFlush(c1, minTS, startTs, count, cluster, user, flow, runid,
-          true);
-    }
-  }
-
-  private void checkMinMaxFlush(Configuration c1, long minTS, long startTs,
-      int count, String cluster, String user, String flow, long runid,
-      boolean checkMax) throws IOException {
-    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());
-    int start = 10;
-    assertEquals(2, r1.size());
-    long starttime = Bytes.toLong(values
-        .get(FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes()));
-    assertEquals(minTS, starttime);
-    if (checkMax) {
-      assertEquals(startTs + 2 * (count - start)
-          + TestFlowDataGenerator.END_TS_INCR,
-          Bytes.toLong(values
-          .get(FlowRunColumn.MAX_END_TIME.getColumnQualifierBytes())));
-    }
-  }
-
-  @Test
-  public void testFilterFlowRunsByCreatedTime() throws Exception {
-    String cluster = "cluster2";
-    String user = "user2";
-    String flow = "flow_name2";
-
-    TimelineEntities te = new TimelineEntities();
-    TimelineEntity entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1(
-        System.currentTimeMillis());
-    entityApp1.setCreatedTime(1425016501000L);
-    te.addEntity(entityApp1);
-
-    HBaseTimelineWriterImpl hbi = null;
-    Configuration c1 = util.getConfiguration();
-    try {
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678919L,
-          "application_11111111111111_1111", te);
-      // write another application with same metric to this flow
-      te = new TimelineEntities();
-      TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2(
-          System.currentTimeMillis());
-      entityApp2.setCreatedTime(1425016502000L);
-      te.addEntity(entityApp2);
-      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678918L,
-          "application_11111111111111_2222", te);
-      hbi.flush();
-    } finally {
-      hbi.close();
-    }
-
-    // use the timeline reader to verify data
-    HBaseTimelineReaderImpl hbr = null;
-    try {
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
-
-      Set<TimelineEntity> entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, user, flow,
-          null, null, TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, 1425016501000L, 1425016502001L, null,
-          null, null, null, null, null), new TimelineDataToRetrieve());
-      assertEquals(2, entities.size());
-      for (TimelineEntity entity : entities) {
-        if (!entity.getId().equals("user2@flow_name2/1002345678918") &&
-            !entity.getId().equals("user2@flow_name2/1002345678919")) {
-          fail("Entities with flow runs 1002345678918 and 1002345678919" +
-              "should be present.");
-        }
-      }
-      entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, user, flow, null, null,
-          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, 1425016501050L, null, null, null,
-          null, null, null, null), new TimelineDataToRetrieve());
-      assertEquals(1, entities.size());
-      for (TimelineEntity entity : entities) {
-        if (!entity.getId().equals("user2@flow_name2/1002345678918")) {
-          fail("Entity with flow run 1002345678918 should be present.");
-        }
-      }
-      entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, user, flow, null, null,
-          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, null, 1425016501050L, null, null,
-          null, null, null, null), new TimelineDataToRetrieve());
-      assertEquals(1, entities.size());
-      for (TimelineEntity entity : entities) {
-        if (!entity.getId().equals("user2@flow_name2/1002345678919")) {
-          fail("Entity with flow run 1002345678919 should be present.");
-        }
-      }
-    } finally {
-      hbr.close();
-    }
-  }
-
-  @Test
-  public void testMetricFilters() throws Exception {
-    String cluster = "cluster1";
-    String user = "user1";
-    String flow = "flow_name1";
-
-    TimelineEntities te = new TimelineEntities();
-    TimelineEntity entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1(
-        System.currentTimeMillis());
-    te.addEntity(entityApp1);
-
-    HBaseTimelineWriterImpl hbi = null;
-    Configuration c1 = util.getConfiguration();
-    try {
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678919L,
-          "application_11111111111111_1111", te);
-      // write another application with same metric to this flow
-      te = new TimelineEntities();
-      TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2(
-          System.currentTimeMillis());
-      te.addEntity(entityApp2);
-      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678918L,
-          "application_11111111111111_2222", te);
-      hbi.flush();
-    } finally {
-      hbi.close();
-    }
-
-    // use the timeline reader to verify data
-    HBaseTimelineReaderImpl hbr = null;
-    try {
-      hbr = new HBaseTimelineReaderImpl();
-      hbr.init(c1);
-      hbr.start();
-
-      TimelineFilterList list1 = new TimelineFilterList();
-      list1.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.GREATER_OR_EQUAL, metric1, 101));
-      TimelineFilterList list2 = new TimelineFilterList();
-      list2.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.LESS_THAN, metric1, 43));
-      list2.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.EQUAL, metric2, 57));
-      TimelineFilterList metricFilterList =
-          new TimelineFilterList(Operator.OR, list1, list2);
-      Set<TimelineEntity> entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, user, flow, null,
-          null, TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList, null),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
-      assertEquals(2, entities.size());
-      int metricCnt = 0;
-      for (TimelineEntity entity : entities) {
-        metricCnt += entity.getMetrics().size();
-      }
-      assertEquals(3, metricCnt);
-
-      TimelineFilterList metricFilterList1 = new TimelineFilterList(
-          new TimelineCompareFilter(
-          TimelineCompareOp.LESS_OR_EQUAL, metric1, 127),
-          new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, metric2, 30));
-      entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, user, flow, null, null,
-          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList1, null),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
-      assertEquals(1, entities.size());
-      metricCnt = 0;
-      for (TimelineEntity entity : entities) {
-        metricCnt += entity.getMetrics().size();
-      }
-      assertEquals(2, metricCnt);
-
-      TimelineFilterList metricFilterList2 = new TimelineFilterList(
-          new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, metric1, 32),
-          new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, metric2, 57));
-      entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, user, flow, null, null,
-          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList2, null),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
-      assertEquals(0, entities.size());
-
-      TimelineFilterList metricFilterList3 = new TimelineFilterList(
-          new TimelineCompareFilter(TimelineCompareOp.EQUAL, "s_metric", 32));
-      entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, user, flow, null, null,
-          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList3, null),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
-      assertEquals(0, entities.size());
-
-      TimelineFilterList list3 = new TimelineFilterList();
-      list3.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.GREATER_OR_EQUAL, metric1, 101));
-      TimelineFilterList list4 = new TimelineFilterList();
-      list4.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.LESS_THAN, metric1, 43));
-      list4.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.EQUAL, metric2, 57));
-      TimelineFilterList metricFilterList4 =
-          new TimelineFilterList(Operator.OR, list3, list4);
-      TimelineFilterList metricsToRetrieve = new TimelineFilterList(Operator.OR,
-          new TimelinePrefixFilter(TimelineCompareOp.EQUAL,
-          metric2.substring(0, metric2.indexOf("_") + 1)));
-      entities = hbr.getEntities(
-          new TimelineReaderContext(cluster, user, flow, null, null,
-          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList4, null),
-          new TimelineDataToRetrieve(null, metricsToRetrieve,
-          EnumSet.of(Field.ALL)));
-      assertEquals(2, entities.size());
-      metricCnt = 0;
-      for (TimelineEntity entity : entities) {
-        metricCnt += entity.getMetrics().size();
-      }
-      assertEquals(1, metricCnt);
-    } finally {
-      hbr.close();
-    }
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    util.shutdownMiniCluster();
-  }
-}


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


[32/50] [abbrv] hadoop git commit: YARN-5138. fix "no findbugs output file" error for hadoop-yarn-server-timelineservice-hbase-tests. (Vrushali C via gtcarrera9)

Posted by vr...@apache.org.
YARN-5138. fix "no findbugs output file" error for hadoop-yarn-server-timelineservice-hbase-tests. (Vrushali C via gtcarrera9)


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

Branch: refs/heads/YARN-2928
Commit: d10932a88afd808a9a7c79c1466b6fca24d9aee4
Parents: 123ebb5
Author: Li Lu <gt...@apache.org>
Authored: Thu May 26 11:12:31 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:11 2016 -0700

----------------------------------------------------------------------
 .../hadoop-yarn-server-timelineservice-hbase-tests/pom.xml    | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d10932a8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
index b5dc3c0..a8e5195 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
@@ -362,6 +362,13 @@
   <build>
     <plugins>
       <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+         <configuration>
+          <includeTests>true</includeTests>
+        </configuration>
+      </plugin>
+      <plugin>
         <artifactId>maven-jar-plugin</artifactId>
         <executions>
           <execution>


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


[28/50] [abbrv] hadoop git commit: YARN-5093. created time shows 0 in most REST output (Varun Saxena via sjlee)

Posted by vr...@apache.org.
YARN-5093. created time shows 0 in most REST output (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/ef12e0ef
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ef12e0ef
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ef12e0ef

Branch: refs/heads/YARN-2928
Commit: ef12e0efacd012f36e340463c30d7a1f05b5e04c
Parents: c9283e1
Author: Sangjin Lee <sj...@apache.org>
Authored: Tue May 24 10:33:04 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:10 2016 -0700

----------------------------------------------------------------------
 .../records/timelineservice/TimelineEntity.java | 16 +++++++++--
 .../storage/TestHBaseTimelineStorage.java       | 30 +++++++++++++-------
 .../storage/FileSystemTimelineReaderImpl.java   |  2 +-
 .../storage/reader/ApplicationEntityReader.java |  5 ++--
 .../storage/reader/GenericEntityReader.java     |  4 +--
 .../reader/TestTimelineReaderWebServices.java   |  4 +--
 .../TestFileSystemTimelineReaderImpl.java       | 10 +++----
 7 files changed, 45 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef12e0ef/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 7ce8279..9c0a983 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
@@ -144,7 +144,7 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
   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;
+  private Long createdTime;
 
   public TimelineEntity() {
     identifier = new Identifier();
@@ -490,7 +490,7 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
   }
 
   @XmlElement(name = "createdtime")
-  public long getCreatedTime() {
+  public Long getCreatedTime() {
     if (real == null) {
       return createdTime;
     } else {
@@ -499,7 +499,7 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
   }
 
   @JsonSetter("createdtime")
-  public void setCreatedTime(long createdTs) {
+  public void setCreatedTime(Long createdTs) {
     if (real == null) {
       this.createdTime = createdTs;
     } else {
@@ -547,6 +547,16 @@ public class TimelineEntity implements Comparable<TimelineEntity> {
   public int compareTo(TimelineEntity other) {
     int comparison = getType().compareTo(other.getType());
     if (comparison == 0) {
+      if (getCreatedTime() == null) {
+        if (other.getCreatedTime() == null) {
+          return getId().compareTo(other.getId());
+        } else {
+          return 1;
+        }
+      }
+      if (other.getCreatedTime() == null) {
+        return -1;
+      }
       if (getCreatedTime() > other.getCreatedTime()) {
         // Order by created time desc
         return -1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef12e0ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/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-hbase-tests/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-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index 8ab54bc..aebd936 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -488,7 +488,7 @@ public class TestHBaseTimelineStorage {
     ApplicationEntity entity = new ApplicationEntity();
     String appId = "application_1000178881110_2002";
     entity.setId(appId);
-    long cTime = 1425016501000L;
+    Long cTime = 1425016501000L;
     entity.setCreatedTime(cTime);
 
     // add the info map in Timeline Entity
@@ -546,7 +546,6 @@ public class TestHBaseTimelineStorage {
     aggEntity.setId(appId);
     aggEntity.setType(type);
     long cTime2 = 1425016502000L;
-    long mTime2 = 1425026902000L;
     aggEntity.setCreatedTime(cTime2);
 
     TimelineMetric aggMetric = new TimelineMetric();
@@ -574,8 +573,21 @@ public class TestHBaseTimelineStorage {
       String flowVersion = "AB7822C10F1111";
       long runid = 1002345678919L;
       hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
+
+      // Write entity again, this time without created time.
+      entity = new ApplicationEntity();
+      appId = "application_1000178881110_2002";
+      entity.setId(appId);
+      // add the info map in Timeline Entity
+      Map<String, Object> infoMap1 = new HashMap<>();
+      infoMap1.put("infoMapKey3", "infoMapValue1");
+      entity.addInfo(infoMap1);
+      te = new TimelineEntities();
+      te.addEntity(entity);
+      hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
       hbi.stop();
 
+      infoMap.putAll(infoMap1);
       // retrieve the row
       byte[] rowKey =
           ApplicationRowKey.getRowKey(cluster, user, flow, runid, appId);
@@ -585,7 +597,7 @@ public class TestHBaseTimelineStorage {
       Result result = new ApplicationTable().getResult(c1, conn, get);
 
       assertTrue(result != null);
-      assertEquals(16, result.size());
+      assertEquals(17, result.size());
 
       // check the row key
       byte[] row1 = result.getRow();
@@ -596,10 +608,9 @@ public class TestHBaseTimelineStorage {
       String id1 = ApplicationColumn.ID.readResult(result).toString();
       assertEquals(appId, id1);
 
-      Number val =
-          (Number) ApplicationColumn.CREATED_TIME.readResult(result);
-      long cTime1 = val.longValue();
-      assertEquals(cTime1, cTime);
+      Long cTime1 =
+          (Long) ApplicationColumn.CREATED_TIME.readResult(result);
+      assertEquals(cTime, cTime1);
 
       Map<String, Object> infoColumns =
           ApplicationColumnPrefix.INFO.readResults(result);
@@ -701,7 +712,7 @@ public class TestHBaseTimelineStorage {
     String type = "world";
     entity.setId(id);
     entity.setType(type);
-    long cTime = 1425016501000L;
+    Long cTime = 1425016501000L;
     entity.setCreatedTime(cTime);
 
     // add the info map in Timeline Entity
@@ -796,8 +807,7 @@ public class TestHBaseTimelineStorage {
           String type1 = EntityColumn.TYPE.readResult(result).toString();
           assertEquals(type, type1);
 
-          Number val = (Number) EntityColumn.CREATED_TIME.readResult(result);
-          long cTime1 = val.longValue();
+          Long cTime1 = (Long) EntityColumn.CREATED_TIME.readResult(result);
           assertEquals(cTime1, cTime);
 
           Map<String, Object> infoColumns =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef12e0ef/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 bdddd7e..00aa686 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
@@ -204,7 +204,7 @@ public class FileSystemTimelineReaderImpl extends AbstractService
   private static void mergeEntities(TimelineEntity entity1,
       TimelineEntity entity2) {
     // Ideally created time wont change except in the case of issue from client.
-    if (entity2.getCreatedTime() > 0) {
+    if (entity2.getCreatedTime() != null && entity2.getCreatedTime() > 0) {
       entity1.setCreatedTime(entity2.getCreatedTime());
     }
     for (Entry<String, String> configEntry : entity2.getConfigs().entrySet()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef12e0ef/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
index 53210f8..71c90fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
@@ -397,9 +397,8 @@ class ApplicationEntityReader extends GenericEntityReader {
 
     TimelineEntityFilters filters = getFilters();
     // fetch created time
-    Number createdTime =
-        (Number)ApplicationColumn.CREATED_TIME.readResult(result);
-    entity.setCreatedTime(createdTime.longValue());
+    Long createdTime = (Long) ApplicationColumn.CREATED_TIME.readResult(result);
+    entity.setCreatedTime(createdTime);
 
     EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
     // fetch is related to entities and match isRelatedTo filter. If isRelatedTo

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef12e0ef/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
index 6696ac5..878695c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/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
@@ -526,8 +526,8 @@ class GenericEntityReader extends TimelineEntityReader {
 
     TimelineEntityFilters filters = getFilters();
     // fetch created time
-    Number createdTime = (Number)EntityColumn.CREATED_TIME.readResult(result);
-    entity.setCreatedTime(createdTime.longValue());
+    Long createdTime = (Long) EntityColumn.CREATED_TIME.readResult(result);
+    entity.setCreatedTime(createdTime);
 
     EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
     // fetch is related to entities and match isRelatedTo filter. If isRelatedTo

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef12e0ef/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 6c3f3e3..0bddf1b 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
@@ -173,7 +173,7 @@ public class TestTimelineReaderWebServices {
       assertNotNull(entity);
       assertEquals("id_1", entity.getId());
       assertEquals("app", entity.getType());
-      assertEquals(1425016502000L, entity.getCreatedTime());
+      assertEquals((Long)1425016502000L, entity.getCreatedTime());
       // Default view i.e. when no fields are specified, entity contains only
       // entity id, entity type and created time.
       assertEquals(0, entity.getConfigs().size());
@@ -196,7 +196,7 @@ public class TestTimelineReaderWebServices {
       assertNotNull(entity);
       assertEquals("id_1", entity.getId());
       assertEquals("app", entity.getType());
-      assertEquals(1425016502000L, entity.getCreatedTime());
+      assertEquals((Long)1425016502000L, entity.getCreatedTime());
     } finally {
       client.destroy();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ef12e0ef/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 23d64e0..ad2122c 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
@@ -278,7 +278,7 @@ public class TestFileSystemTimelineReaderImpl {
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
-    Assert.assertEquals(1425016502000L, result.getCreatedTime());
+    Assert.assertEquals((Long)1425016502000L, result.getCreatedTime());
     Assert.assertEquals(0, result.getConfigs().size());
     Assert.assertEquals(0, result.getMetrics().size());
   }
@@ -293,7 +293,7 @@ public class TestFileSystemTimelineReaderImpl {
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
-    Assert.assertEquals(1425016502000L, result.getCreatedTime());
+    Assert.assertEquals((Long)1425016502000L, result.getCreatedTime());
     Assert.assertEquals(0, result.getConfigs().size());
     Assert.assertEquals(0, result.getMetrics().size());
   }
@@ -310,7 +310,7 @@ public class TestFileSystemTimelineReaderImpl {
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_5")).toString(),
         result.getIdentifier().toString());
-    Assert.assertEquals(1425016502050L, result.getCreatedTime());
+    Assert.assertEquals((Long)1425016502050L, result.getCreatedTime());
   }
 
   @Test
@@ -324,7 +324,7 @@ public class TestFileSystemTimelineReaderImpl {
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
-    Assert.assertEquals(1425016502000L, result.getCreatedTime());
+    Assert.assertEquals((Long)1425016502000L, result.getCreatedTime());
     Assert.assertEquals(3, result.getConfigs().size());
     Assert.assertEquals(3, result.getMetrics().size());
     Assert.assertEquals(2, result.getInfo().size());
@@ -342,7 +342,7 @@ public class TestFileSystemTimelineReaderImpl {
     Assert.assertEquals(
         (new TimelineEntity.Identifier("app", "id_1")).toString(),
         result.getIdentifier().toString());
-    Assert.assertEquals(1425016502000L, result.getCreatedTime());
+    Assert.assertEquals((Long)1425016502000L, result.getCreatedTime());
     Assert.assertEquals(3, result.getConfigs().size());
     Assert.assertEquals(3, result.getMetrics().size());
     // All fields including events will be returned.


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


[45/50] [abbrv] hadoop git commit: YARN-5170. Eliminate singleton converters and static method access. (Joep Rottinghuis via Varun Saxena)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
index 8e806bc..aa2bfda 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
@@ -46,8 +46,11 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
 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.ApplicationRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 
 import com.google.common.base.Preconditions;
@@ -150,13 +153,13 @@ class ApplicationEntityReader extends GenericEntityReader {
     EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
     // If INFO field has to be retrieved, add a filter for fetching columns
     // with INFO column prefix.
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
+    if (hasField(fieldsToRetrieve, Field.INFO)) {
       infoFamilyColsFilter.addFilter(
           TimelineFilterUtils.createHBaseQualifierFilter(
               CompareOp.EQUAL, ApplicationColumnPrefix.INFO));
     }
     TimelineFilterList relatesTo = getFilters().getRelatesTo();
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
+    if (hasField(fieldsToRetrieve, Field.RELATES_TO)) {
       // If RELATES_TO field has to be retrieved, add a filter for fetching
       // columns with RELATES_TO column prefix.
       infoFamilyColsFilter.addFilter(
@@ -169,12 +172,11 @@ class ApplicationEntityReader extends GenericEntityReader {
       // matched after fetching rows from HBase.
       Set<String> relatesToCols =
           TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo);
-      infoFamilyColsFilter.addFilter(
-          TimelineFilterUtils.createFiltersFromColumnQualifiers(
-              ApplicationColumnPrefix.RELATES_TO, relatesToCols));
+      infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
+          ApplicationColumnPrefix.RELATES_TO, relatesToCols));
     }
     TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo();
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
+    if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
       // If IS_RELATED_TO field has to be retrieved, add a filter for fetching
       // columns with IS_RELATED_TO column prefix.
       infoFamilyColsFilter.addFilter(
@@ -187,12 +189,11 @@ class ApplicationEntityReader extends GenericEntityReader {
       // matched after fetching rows from HBase.
       Set<String> isRelatedToCols =
           TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo);
-      infoFamilyColsFilter.addFilter(
-          TimelineFilterUtils.createFiltersFromColumnQualifiers(
-              ApplicationColumnPrefix.IS_RELATED_TO, isRelatedToCols));
+      infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
+          ApplicationColumnPrefix.IS_RELATED_TO, isRelatedToCols));
     }
     TimelineFilterList eventFilters = getFilters().getEventFilters();
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
+    if (hasField(fieldsToRetrieve, Field.EVENTS)) {
       // If EVENTS field has to be retrieved, add a filter for fetching columns
       // with EVENT column prefix.
       infoFamilyColsFilter.addFilter(
@@ -205,9 +206,8 @@ class ApplicationEntityReader extends GenericEntityReader {
       // fetching rows from HBase.
       Set<String> eventCols =
           TimelineFilterUtils.fetchColumnsFromFilterList(eventFilters);
-      infoFamilyColsFilter.addFilter(
-          TimelineFilterUtils.createFiltersFromColumnQualifiers(
-              ApplicationColumnPrefix.EVENT, eventCols));
+      infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
+          ApplicationColumnPrefix.EVENT, eventCols));
     }
     return infoFamilyColsFilter;
   }
@@ -222,25 +222,25 @@ class ApplicationEntityReader extends GenericEntityReader {
   private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) {
     EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
     // Events not required.
-    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
+    if (!hasField(fieldsToRetrieve, Field.EVENTS)) {
       infoColFamilyList.addFilter(
           TimelineFilterUtils.createHBaseQualifierFilter(
               CompareOp.NOT_EQUAL, ApplicationColumnPrefix.EVENT));
     }
     // info not required.
-    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
+    if (!hasField(fieldsToRetrieve, Field.INFO)) {
       infoColFamilyList.addFilter(
           TimelineFilterUtils.createHBaseQualifierFilter(
               CompareOp.NOT_EQUAL, ApplicationColumnPrefix.INFO));
     }
     // is related to not required.
-    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
+    if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
       infoColFamilyList.addFilter(
           TimelineFilterUtils.createHBaseQualifierFilter(
               CompareOp.NOT_EQUAL, ApplicationColumnPrefix.IS_RELATED_TO));
     }
     // relates to not required.
-    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
+    if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) {
       infoColFamilyList.addFilter(
           TimelineFilterUtils.createHBaseQualifierFilter(
               CompareOp.NOT_EQUAL, ApplicationColumnPrefix.RELATES_TO));
@@ -308,9 +308,10 @@ class ApplicationEntityReader extends GenericEntityReader {
   protected Result getResult(Configuration hbaseConf, Connection conn,
       FilterList filterList) throws IOException {
     TimelineReaderContext context = getContext();
-    byte[] rowKey =
-        ApplicationRowKey.getRowKey(context.getClusterId(), context.getUserId(),
+    ApplicationRowKey applicationRowKey =
+        new ApplicationRowKey(context.getClusterId(), context.getUserId(),
             context.getFlowName(), context.getFlowRunId(), context.getAppId());
+    byte[] rowKey = applicationRowKey.getRowKey();
     Get get = new Get(rowKey);
     get.setMaxVersions(getDataToRetrieve().getMetricsLimit());
     if (filterList != null && !filterList.getFilters().isEmpty()) {
@@ -345,10 +346,13 @@ class ApplicationEntityReader extends GenericEntityReader {
     TimelineReaderContext context = getContext();
     if (isSingleEntityRead()) {
       // Get flow context information from AppToFlow table.
-      if (context.getFlowName() == null || context.getFlowRunId() == null ||
-          context.getUserId() == null) {
-        FlowContext flowContext = lookupFlowContext(
-            context.getClusterId(), context.getAppId(), hbaseConf, conn);
+      if (context.getFlowName() == null || context.getFlowRunId() == null
+          || context.getUserId() == null) {
+        AppToFlowRowKey appToFlowRowKey =
+            new AppToFlowRowKey(context.getClusterId(), context.getAppId());
+        FlowContext flowContext =
+            lookupFlowContext(appToFlowRowKey,
+                hbaseConf, conn);
         context.setFlowName(flowContext.getFlowName());
         context.setFlowRunId(flowContext.getFlowRunId());
         context.setUserId(flowContext.getUserId());
@@ -367,15 +371,13 @@ class ApplicationEntityReader extends GenericEntityReader {
       Connection conn, FilterList filterList) throws IOException {
     Scan scan = new Scan();
     TimelineReaderContext context = getContext();
-    if (context.getFlowRunId() != null) {
-      scan.setRowPrefixFilter(ApplicationRowKey.
-          getRowKeyPrefix(context.getClusterId(), context.getUserId(),
-              context.getFlowName(), context.getFlowRunId()));
-    } else {
-      scan.setRowPrefixFilter(ApplicationRowKey.
-          getRowKeyPrefix(context.getClusterId(), context.getUserId(),
-              context.getFlowName()));
-    }
+    // Whether or not flowRunID is null doesn't matter, the
+    // ApplicationRowKeyPrefix will do the right thing.
+    RowKeyPrefix<ApplicationRowKey> applicationRowKeyPrefix =
+        new ApplicationRowKeyPrefix(context.getClusterId(),
+            context.getUserId(), context.getFlowName(),
+            context.getFlowRunId());
+    scan.setRowPrefixFilter(applicationRowKeyPrefix.getRowKeyPrefix());
     FilterList newList = new FilterList();
     newList.addFilter(new PageFilter(getFilters().getLimit()));
     if (filterList != null && !filterList.getFilters().isEmpty()) {
@@ -409,15 +411,14 @@ class ApplicationEntityReader extends GenericEntityReader {
     boolean checkIsRelatedTo =
         !isSingleEntityRead() && filters.getIsRelatedTo() != null &&
         filters.getIsRelatedTo().getFilterList().size() > 0;
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO) ||
-        checkIsRelatedTo) {
-      TimelineStorageUtils.readRelationship(
-          entity, result, ApplicationColumnPrefix.IS_RELATED_TO, true);
+    if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) {
+      readRelationship(entity, result, ApplicationColumnPrefix.IS_RELATED_TO,
+          true);
       if (checkIsRelatedTo && !TimelineStorageUtils.matchIsRelatedTo(entity,
           filters.getIsRelatedTo())) {
         return null;
       }
-      if (!TimelineStorageUtils.hasField(fieldsToRetrieve,
+      if (!hasField(fieldsToRetrieve,
           Field.IS_RELATED_TO)) {
         entity.getIsRelatedToEntities().clear();
       }
@@ -430,29 +431,27 @@ class ApplicationEntityReader extends GenericEntityReader {
     boolean checkRelatesTo =
         !isSingleEntityRead() && filters.getRelatesTo() != null &&
         filters.getRelatesTo().getFilterList().size() > 0;
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO) ||
+    if (hasField(fieldsToRetrieve, Field.RELATES_TO) ||
         checkRelatesTo) {
-      TimelineStorageUtils.readRelationship(
-          entity, result, ApplicationColumnPrefix.RELATES_TO, false);
+      readRelationship(entity, result, ApplicationColumnPrefix.RELATES_TO,
+          false);
       if (checkRelatesTo && !TimelineStorageUtils.matchRelatesTo(entity,
           filters.getRelatesTo())) {
         return null;
       }
-      if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
+      if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) {
         entity.getRelatesToEntities().clear();
       }
     }
 
     // fetch info if fieldsToRetrieve contains INFO or ALL.
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
-      TimelineStorageUtils.readKeyValuePairs(
-          entity, result, ApplicationColumnPrefix.INFO, false);
+    if (hasField(fieldsToRetrieve, Field.INFO)) {
+      readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false);
     }
 
     // fetch configs if fieldsToRetrieve contains CONFIGS or ALL.
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.CONFIGS)) {
-      TimelineStorageUtils.readKeyValuePairs(
-          entity, result, ApplicationColumnPrefix.CONFIG, true);
+    if (hasField(fieldsToRetrieve, Field.CONFIGS)) {
+      readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true);
     }
 
     // fetch events and match event filters if they exist. If event filters do
@@ -462,21 +461,19 @@ class ApplicationEntityReader extends GenericEntityReader {
     boolean checkEvents =
         !isSingleEntityRead() && filters.getEventFilters() != null &&
         filters.getEventFilters().getFilterList().size() > 0;
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS) ||
-        checkEvents) {
-      TimelineStorageUtils.readEvents(
-          entity, result, ApplicationColumnPrefix.EVENT);
+    if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) {
+      readEvents(entity, result, ApplicationColumnPrefix.EVENT);
       if (checkEvents && !TimelineStorageUtils.matchEventFilters(entity,
           filters.getEventFilters())) {
         return null;
       }
-      if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
+      if (!hasField(fieldsToRetrieve, Field.EVENTS)) {
         entity.getEvents().clear();
       }
     }
 
     // fetch metrics if fieldsToRetrieve contains METRICS or ALL.
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.METRICS)) {
+    if (hasField(fieldsToRetrieve, Field.METRICS)) {
       readMetrics(entity, result, ApplicationColumnPrefix.METRIC);
     }
     return entity;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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
index faecd14..9ba5e38 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/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
@@ -35,9 +35,11 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrie
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongKeyConverter;
 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.FlowActivityRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityTable;
 
 import com.google.common.base.Preconditions;
@@ -50,6 +52,12 @@ class FlowActivityEntityReader extends TimelineEntityReader {
   private static final FlowActivityTable FLOW_ACTIVITY_TABLE =
       new FlowActivityTable();
 
+  /**
+   * Used to convert Long key components to and from storage format.
+   */
+  private final KeyConverter<Long> longKeyConverter = new LongKeyConverter();
+
+
   public FlowActivityEntityReader(TimelineReaderContext ctxt,
       TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve) {
     super(ctxt, entityFilters, toRetrieve, true);
@@ -105,15 +113,14 @@ class FlowActivityEntityReader extends TimelineEntityReader {
     if (getFilters().getCreatedTimeBegin() == 0L &&
         getFilters().getCreatedTimeEnd() == Long.MAX_VALUE) {
        // All records have to be chosen.
-      scan.setRowPrefixFilter(FlowActivityRowKey.getRowKeyPrefix(clusterId));
+      scan.setRowPrefixFilter(new FlowActivityRowKeyPrefix(clusterId)
+          .getRowKeyPrefix());
     } else {
-      scan.setStartRow(
-          FlowActivityRowKey.getRowKeyPrefix(clusterId,
-              getFilters().getCreatedTimeEnd()));
-      scan.setStopRow(
-          FlowActivityRowKey.getRowKeyPrefix(clusterId,
-              (getFilters().getCreatedTimeBegin() <= 0 ? 0 :
-              (getFilters().getCreatedTimeBegin() - 1))));
+      scan.setStartRow(new FlowActivityRowKeyPrefix(clusterId, getFilters()
+          .getCreatedTimeEnd()).getRowKeyPrefix());
+      scan.setStopRow(new FlowActivityRowKeyPrefix(clusterId, (getFilters()
+          .getCreatedTimeBegin() <= 0 ? 0
+          : (getFilters().getCreatedTimeBegin() - 1))).getRowKeyPrefix());
     }
     // 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
@@ -137,8 +144,7 @@ class FlowActivityEntityReader extends TimelineEntityReader {
     // get the list of run ids along with the version that are associated with
     // this flow on this day
     Map<Long, Object> runIdsMap =
-        FlowActivityColumnPrefix.RUN_ID.readResults(result,
-            LongKeyConverter.getInstance());
+        FlowActivityColumnPrefix.RUN_ID.readResults(result, longKeyConverter);
     for (Map.Entry<Long, Object> e : runIdsMap.entrySet()) {
       Long runId = e.getKey();
       String version = (String)e.getValue();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
index e1695ef..986a28f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
@@ -28,12 +28,12 @@ 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.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.TimelineDataToRetrieve;
@@ -43,11 +43,12 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilte
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumn;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunColumnFamily;
 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.FlowRunRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
 import org.apache.hadoop.yarn.webapp.BadRequestException;
 
@@ -81,8 +82,8 @@ class FlowRunEntityReader extends TimelineEntityReader {
   @Override
   protected void validateParams() {
     Preconditions.checkNotNull(getContext(), "context shouldn't be null");
-    Preconditions.checkNotNull(
-        getDataToRetrieve(), "data to retrieve shouldn't be null");
+    Preconditions.checkNotNull(getDataToRetrieve(),
+        "data to retrieve shouldn't be null");
     Preconditions.checkNotNull(getContext().getClusterId(),
         "clusterId shouldn't be null");
     Preconditions.checkNotNull(getContext().getUserId(),
@@ -97,8 +98,8 @@ class FlowRunEntityReader extends TimelineEntityReader {
     if (!isSingleEntityRead() && fieldsToRetrieve != null) {
       for (Field field : fieldsToRetrieve) {
         if (field != Field.ALL && field != Field.METRICS) {
-          throw new BadRequestException("Invalid field " + field +
-              " specified while querying flow runs.");
+          throw new BadRequestException("Invalid field " + field
+              + " specified while querying flow runs.");
         }
       }
     }
@@ -119,23 +120,22 @@ class FlowRunEntityReader extends TimelineEntityReader {
     Long createdTimeBegin = getFilters().getCreatedTimeBegin();
     Long createdTimeEnd = getFilters().getCreatedTimeEnd();
     if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) {
-      listBasedOnFilters.addFilter(
-          TimelineFilterUtils.createSingleColValueFiltersByRange(
-          FlowRunColumn.MIN_START_TIME, createdTimeBegin, createdTimeEnd));
+      listBasedOnFilters.addFilter(TimelineFilterUtils
+          .createSingleColValueFiltersByRange(FlowRunColumn.MIN_START_TIME,
+              createdTimeBegin, createdTimeEnd));
     }
     // Filter based on metric filters.
     TimelineFilterList metricFilters = getFilters().getMetricFilters();
     if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) {
-      listBasedOnFilters.addFilter(
-          TimelineFilterUtils.createHBaseFilterList(
-              FlowRunColumnPrefix.METRIC, metricFilters));
+      listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList(
+          FlowRunColumnPrefix.METRIC, metricFilters));
     }
     return listBasedOnFilters;
   }
 
   /**
-   * Add {@link QualifierFilter} filters to filter list for each column of
-   * flow run table.
+   * Add {@link QualifierFilter} filters to filter list for each column of flow
+   * run table.
    *
    * @return filter list to which qualifier filters have been added.
    */
@@ -153,20 +153,19 @@ class FlowRunEntityReader extends TimelineEntityReader {
     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()));
+        new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(
+            FlowRunColumnFamily.INFO.getBytes()));
     TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
     // If multiple entities have to be retrieved, check if metrics have to be
     // retrieved and if not, add a filter so that metrics can be excluded.
     // Metrics are always returned if we are reading a single entity.
-    if (!isSingleEntityRead() && !TimelineStorageUtils.hasField(
-        dataToRetrieve.getFieldsToRetrieve(), Field.METRICS)) {
+    if (!isSingleEntityRead()
+        && !hasField(dataToRetrieve.getFieldsToRetrieve(), Field.METRICS)) {
       FilterList infoColFamilyList = new FilterList(Operator.MUST_PASS_ONE);
       infoColFamilyList.addFilter(infoColumnFamily);
-      infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-              FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(""))));
+      infoColFamilyList.addFilter(new QualifierFilter(CompareOp.NOT_EQUAL,
+          new BinaryPrefixComparator(FlowRunColumnPrefix.METRIC
+              .getColumnPrefixBytes(""))));
       list.addFilter(infoColFamilyList);
     } else {
       // Check if metricsToRetrieve are specified and if they are, create a
@@ -176,14 +175,13 @@ class FlowRunEntityReader extends TimelineEntityReader {
       // (in augmentParams()).
       TimelineFilterList metricsToRetrieve =
           dataToRetrieve.getMetricsToRetrieve();
-      if (metricsToRetrieve != null &&
-          !metricsToRetrieve.getFilterList().isEmpty()) {
+      if (metricsToRetrieve != null
+          && !metricsToRetrieve.getFilterList().isEmpty()) {
         FilterList infoColFamilyList = new FilterList();
         infoColFamilyList.addFilter(infoColumnFamily);
         FilterList columnsList = updateFixedColumns();
-        columnsList.addFilter(
-            TimelineFilterUtils.createHBaseFilterList(
-                FlowRunColumnPrefix.METRIC, metricsToRetrieve));
+        columnsList.addFilter(TimelineFilterUtils.createHBaseFilterList(
+            FlowRunColumnPrefix.METRIC, metricsToRetrieve));
         infoColFamilyList.addFilter(columnsList);
         list.addFilter(infoColFamilyList);
       }
@@ -195,9 +193,10 @@ class FlowRunEntityReader extends TimelineEntityReader {
   protected Result getResult(Configuration hbaseConf, Connection conn,
       FilterList filterList) throws IOException {
     TimelineReaderContext context = getContext();
-    byte[] rowKey =
-        FlowRunRowKey.getRowKey(context.getClusterId(), context.getUserId(),
+    FlowRunRowKey flowRunRowKey =
+        new FlowRunRowKey(context.getClusterId(), context.getUserId(),
             context.getFlowName(), context.getFlowRunId());
+    byte[] rowKey = flowRunRowKey.getRowKey();
     Get get = new Get(rowKey);
     get.setMaxVersions(Integer.MAX_VALUE);
     if (filterList != null && !filterList.getFilters().isEmpty()) {
@@ -207,13 +206,14 @@ class FlowRunEntityReader extends TimelineEntityReader {
   }
 
   @Override
-  protected ResultScanner getResults(Configuration hbaseConf,
-      Connection conn, FilterList filterList) throws IOException {
+  protected ResultScanner getResults(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException {
     Scan scan = new Scan();
     TimelineReaderContext context = getContext();
-    scan.setRowPrefixFilter(
-        FlowRunRowKey.getRowKeyPrefix(context.getClusterId(),
-            context.getUserId(), context.getFlowName()));
+    RowKeyPrefix<FlowRunRowKey> flowRunRowKeyPrefix =
+        new FlowRunRowKeyPrefix(context.getClusterId(), context.getUserId(),
+            context.getFlowName());
+    scan.setRowPrefixFilter(flowRunRowKeyPrefix.getRowKeyPrefix());
     FilterList newList = new FilterList();
     newList.addFilter(new PageFilter(getFilters().getLimit()));
     if (filterList != null && !filterList.getFilters().isEmpty()) {
@@ -238,27 +238,27 @@ class FlowRunEntityReader extends TimelineEntityReader {
     }
 
     // read the start time
-    Long startTime = (Long)FlowRunColumn.MIN_START_TIME.readResult(result);
+    Long startTime = (Long) FlowRunColumn.MIN_START_TIME.readResult(result);
     if (startTime != null) {
       flowRun.setStartTime(startTime.longValue());
     }
 
     // read the end time if available
-    Long endTime = (Long)FlowRunColumn.MAX_END_TIME.readResult(result);
+    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);
+    String version = (String) FlowRunColumn.FLOW_VERSION.readResult(result);
     if (version != null) {
       flowRun.setVersion(version);
     }
 
     // read metrics if its a single entity query or if METRICS are part of
     // fieldsToRetrieve.
-    if (isSingleEntityRead() || TimelineStorageUtils.hasField(
-        getDataToRetrieve().getFieldsToRetrieve(), Field.METRICS)) {
+    if (isSingleEntityRead()
+        || hasField(getDataToRetrieve().getFieldsToRetrieve(), Field.METRICS)) {
       readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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
index 22583b5..4e1ab8a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/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
@@ -19,6 +19,7 @@ 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;
@@ -28,11 +29,11 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.FamilyFilter;
 import org.apache.hadoop.hbase.filter.FilterList;
-import org.apache.hadoop.hbase.filter.QualifierFilter;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.FilterList.Operator;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
@@ -44,11 +45,16 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlow
 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.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
 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.EntityRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
 import org.apache.hadoop.yarn.webapp.NotFoundException;
 
@@ -66,6 +72,12 @@ class GenericEntityReader extends TimelineEntityReader {
    */
   private final AppToFlowTable appToFlowTable = new AppToFlowTable();
 
+  /**
+   * Used to convert strings key components to and from storage format.
+   */
+  private final KeyConverter<String> stringKeyConverter =
+      new StringKeyConverter();
+
   public GenericEntityReader(TimelineReaderContext ctxt,
       TimelineEntityFilters entityFilters, TimelineDataToRetrieve toRetrieve,
       boolean sortedKeys) {
@@ -95,32 +107,29 @@ class GenericEntityReader extends TimelineEntityReader {
     long createdTimeBegin = filters.getCreatedTimeBegin();
     long createdTimeEnd = filters.getCreatedTimeEnd();
     if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) {
-      listBasedOnFilters.addFilter(
-          TimelineFilterUtils.createSingleColValueFiltersByRange(
-              EntityColumn.CREATED_TIME, createdTimeBegin, createdTimeEnd));
+      listBasedOnFilters.addFilter(TimelineFilterUtils
+          .createSingleColValueFiltersByRange(EntityColumn.CREATED_TIME,
+              createdTimeBegin, createdTimeEnd));
     }
     // Create filter list based on metric filters and add it to
     // listBasedOnFilters.
     TimelineFilterList metricFilters = filters.getMetricFilters();
     if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) {
-      listBasedOnFilters.addFilter(
-          TimelineFilterUtils.createHBaseFilterList(
-              EntityColumnPrefix.METRIC, metricFilters));
+      listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList(
+          EntityColumnPrefix.METRIC, metricFilters));
     }
     // Create filter list based on config filters and add it to
     // listBasedOnFilters.
     TimelineFilterList configFilters = filters.getConfigFilters();
     if (configFilters != null && !configFilters.getFilterList().isEmpty()) {
-      listBasedOnFilters.addFilter(
-          TimelineFilterUtils.createHBaseFilterList(
-              EntityColumnPrefix.CONFIG, configFilters));
+      listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList(
+          EntityColumnPrefix.CONFIG, configFilters));
     }
     // Create filter list based on info filters and add it to listBasedOnFilters
     TimelineFilterList infoFilters = filters.getInfoFilters();
     if (infoFilters != null && !infoFilters.getFilterList().isEmpty()) {
-      listBasedOnFilters.addFilter(
-          TimelineFilterUtils.createHBaseFilterList(
-              EntityColumnPrefix.INFO, infoFilters));
+      listBasedOnFilters.addFilter(TimelineFilterUtils.createHBaseFilterList(
+          EntityColumnPrefix.INFO, infoFilters));
     }
     return listBasedOnFilters;
   }
@@ -130,10 +139,10 @@ class GenericEntityReader extends TimelineEntityReader {
    *
    * @return true if we need to fetch some of the columns, false otherwise.
    */
-  private static boolean fetchPartialEventCols(TimelineFilterList eventFilters,
+  private boolean fetchPartialEventCols(TimelineFilterList eventFilters,
       EnumSet<Field> fieldsToRetrieve) {
     return (eventFilters != null && !eventFilters.getFilterList().isEmpty() &&
-        !TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS));
+        !hasField(fieldsToRetrieve, Field.EVENTS));
   }
 
   /**
@@ -141,10 +150,10 @@ class GenericEntityReader extends TimelineEntityReader {
    *
    * @return true if we need to fetch some of the columns, false otherwise.
    */
-  private static boolean fetchPartialRelatesToCols(TimelineFilterList relatesTo,
+  private boolean fetchPartialRelatesToCols(TimelineFilterList relatesTo,
       EnumSet<Field> fieldsToRetrieve) {
     return (relatesTo != null && !relatesTo.getFilterList().isEmpty() &&
-        !TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO));
+        !hasField(fieldsToRetrieve, Field.RELATES_TO));
   }
 
   /**
@@ -152,10 +161,10 @@ class GenericEntityReader extends TimelineEntityReader {
    *
    * @return true if we need to fetch some of the columns, false otherwise.
    */
-  private static boolean fetchPartialIsRelatedToCols(
-      TimelineFilterList isRelatedTo, EnumSet<Field> fieldsToRetrieve) {
+  private boolean fetchPartialIsRelatedToCols(TimelineFilterList isRelatedTo,
+      EnumSet<Field> fieldsToRetrieve) {
     return (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty() &&
-        !TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO));
+        !hasField(fieldsToRetrieve, Field.IS_RELATED_TO));
   }
 
   /**
@@ -163,19 +172,20 @@ class GenericEntityReader extends TimelineEntityReader {
    * relatesto and isrelatedto from info family.
    *
    * @return true, if we need to fetch only some of the columns, false if we
-   *     need to fetch all the columns under info column family.
+   *         need to fetch all the columns under info column family.
    */
   protected boolean fetchPartialColsFromInfoFamily() {
     EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
     TimelineEntityFilters filters = getFilters();
-    return fetchPartialEventCols(filters.getEventFilters(), fieldsToRetrieve) ||
-        fetchPartialRelatesToCols(filters.getRelatesTo(), fieldsToRetrieve) ||
-        fetchPartialIsRelatedToCols(filters.getIsRelatedTo(), fieldsToRetrieve);
+    return fetchPartialEventCols(filters.getEventFilters(), fieldsToRetrieve)
+        || fetchPartialRelatesToCols(filters.getRelatesTo(), fieldsToRetrieve)
+        || fetchPartialIsRelatedToCols(filters.getIsRelatedTo(),
+            fieldsToRetrieve);
   }
 
   /**
-   * Check if we need to create filter list based on fields. We need to create
-   * a filter list iff all fields need not be retrieved or we have some specific
+   * Check if we need to create filter list based on fields. We need to create a
+   * filter list iff all fields need not be retrieved or we have some specific
    * fields or metrics to retrieve. We also need to create a filter list if we
    * have relationships(relatesTo/isRelatedTo) and event filters specified for
    * the query.
@@ -188,22 +198,24 @@ class GenericEntityReader extends TimelineEntityReader {
     // be retrieved, also check if we have some metrics or configs to
     // retrieve specified for the query because then a filter list will have
     // to be created.
-    boolean flag = !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) ||
-        (dataToRetrieve.getConfsToRetrieve() != null &&
-        !dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) ||
-        (dataToRetrieve.getMetricsToRetrieve() != null &&
-        !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty());
+    boolean flag =
+        !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL)
+            || (dataToRetrieve.getConfsToRetrieve() != null && !dataToRetrieve
+                .getConfsToRetrieve().getFilterList().isEmpty())
+            || (dataToRetrieve.getMetricsToRetrieve() != null && !dataToRetrieve
+                .getMetricsToRetrieve().getFilterList().isEmpty());
     // Filters need to be checked only if we are reading multiple entities. If
     // condition above is false, we check if there are relationships(relatesTo/
     // isRelatedTo) and event filters specified for the query.
     if (!flag && !isSingleEntityRead()) {
       TimelineEntityFilters filters = getFilters();
-      flag = (filters.getEventFilters() != null &&
-          !filters.getEventFilters().getFilterList().isEmpty()) ||
-          (filters.getIsRelatedTo() != null &&
-          !filters.getIsRelatedTo().getFilterList().isEmpty()) ||
-          (filters.getRelatesTo() != null &&
-          !filters.getRelatesTo().getFilterList().isEmpty());
+      flag =
+          (filters.getEventFilters() != null && !filters.getEventFilters()
+              .getFilterList().isEmpty())
+              || (filters.getIsRelatedTo() != null && !filters.getIsRelatedTo()
+                  .getFilterList().isEmpty())
+              || (filters.getRelatesTo() != null && !filters.getRelatesTo()
+                  .getFilterList().isEmpty());
     }
     return flag;
   }
@@ -216,8 +228,8 @@ class GenericEntityReader extends TimelineEntityReader {
    */
   protected void updateFixedColumns(FilterList list) {
     for (EntityColumn column : EntityColumn.values()) {
-      list.addFilter(new QualifierFilter(CompareOp.EQUAL,
-          new BinaryComparator(column.getColumnQualifierBytes())));
+      list.addFilter(new QualifierFilter(CompareOp.EQUAL, new BinaryComparator(
+          column.getColumnQualifierBytes())));
     }
   }
 
@@ -226,30 +238,29 @@ class GenericEntityReader extends TimelineEntityReader {
    * qualifiers in the info column family will be returned in result.
    *
    * @param isApplication If true, it means operations are to be performed for
-   *     application table, otherwise for entity table.
+   *          application table, otherwise for entity table.
    * @return filter list.
    * @throws IOException if any problem occurs while creating filter list.
    */
-  private FilterList createFilterListForColsOfInfoFamily()
-      throws IOException {
+  private FilterList createFilterListForColsOfInfoFamily() throws IOException {
     FilterList infoFamilyColsFilter = new FilterList(Operator.MUST_PASS_ONE);
     // Add filters for each column in entity table.
     updateFixedColumns(infoFamilyColsFilter);
     EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
     // If INFO field has to be retrieved, add a filter for fetching columns
     // with INFO column prefix.
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
-      infoFamilyColsFilter.addFilter(
-          TimelineFilterUtils.createHBaseQualifierFilter(
+    if (hasField(fieldsToRetrieve, Field.INFO)) {
+      infoFamilyColsFilter
+          .addFilter(TimelineFilterUtils.createHBaseQualifierFilter(
               CompareOp.EQUAL, EntityColumnPrefix.INFO));
     }
     TimelineFilterList relatesTo = getFilters().getRelatesTo();
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
+    if (hasField(fieldsToRetrieve, Field.RELATES_TO)) {
       // If RELATES_TO field has to be retrieved, add a filter for fetching
       // columns with RELATES_TO column prefix.
-      infoFamilyColsFilter.addFilter(
-          TimelineFilterUtils.createHBaseQualifierFilter(
-              CompareOp.EQUAL, EntityColumnPrefix.RELATES_TO));
+      infoFamilyColsFilter.addFilter(TimelineFilterUtils
+          .createHBaseQualifierFilter(CompareOp.EQUAL,
+              EntityColumnPrefix.RELATES_TO));
     } else if (relatesTo != null && !relatesTo.getFilterList().isEmpty()) {
       // Even if fields to retrieve does not contain RELATES_TO, we still
       // need to have a filter to fetch some of the column qualifiers if
@@ -257,17 +268,16 @@ class GenericEntityReader extends TimelineEntityReader {
       // matched after fetching rows from HBase.
       Set<String> relatesToCols =
           TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo);
-      infoFamilyColsFilter.addFilter(
-          TimelineFilterUtils.createFiltersFromColumnQualifiers(
-              EntityColumnPrefix.RELATES_TO, relatesToCols));
+      infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
+          EntityColumnPrefix.RELATES_TO, relatesToCols));
     }
     TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo();
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
+    if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
       // If IS_RELATED_TO field has to be retrieved, add a filter for fetching
       // columns with IS_RELATED_TO column prefix.
-      infoFamilyColsFilter.addFilter(
-          TimelineFilterUtils.createHBaseQualifierFilter(
-              CompareOp.EQUAL, EntityColumnPrefix.IS_RELATED_TO));
+      infoFamilyColsFilter.addFilter(TimelineFilterUtils
+          .createHBaseQualifierFilter(CompareOp.EQUAL,
+              EntityColumnPrefix.IS_RELATED_TO));
     } else if (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty()) {
       // Even if fields to retrieve does not contain IS_RELATED_TO, we still
       // need to have a filter to fetch some of the column qualifiers if
@@ -275,27 +285,26 @@ class GenericEntityReader extends TimelineEntityReader {
       // matched after fetching rows from HBase.
       Set<String> isRelatedToCols =
           TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo);
-      infoFamilyColsFilter.addFilter(
-          TimelineFilterUtils.createFiltersFromColumnQualifiers(
-              EntityColumnPrefix.IS_RELATED_TO, isRelatedToCols));
+      infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
+          EntityColumnPrefix.IS_RELATED_TO, isRelatedToCols));
     }
     TimelineFilterList eventFilters = getFilters().getEventFilters();
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
+    if (hasField(fieldsToRetrieve, Field.EVENTS)) {
       // If EVENTS field has to be retrieved, add a filter for fetching columns
       // with EVENT column prefix.
-      infoFamilyColsFilter.addFilter(
-          TimelineFilterUtils.createHBaseQualifierFilter(
+      infoFamilyColsFilter
+          .addFilter(TimelineFilterUtils.createHBaseQualifierFilter(
               CompareOp.EQUAL, EntityColumnPrefix.EVENT));
-    } else if (eventFilters != null && !eventFilters.getFilterList().isEmpty()){
+    } else if (eventFilters != null &&
+        !eventFilters.getFilterList().isEmpty()) {
       // Even if fields to retrieve does not contain EVENTS, we still need to
       // have a filter to fetch some of the column qualifiers on the basis of
       // event filters specified. Event filters will then be matched after
       // fetching rows from HBase.
       Set<String> eventCols =
           TimelineFilterUtils.fetchColumnsFromFilterList(eventFilters);
-      infoFamilyColsFilter.addFilter(
-          TimelineFilterUtils.createFiltersFromColumnQualifiers(
-              EntityColumnPrefix.EVENT, eventCols));
+      infoFamilyColsFilter.addFilter(createFiltersFromColumnQualifiers(
+          EntityColumnPrefix.EVENT, eventCols));
     }
     return infoFamilyColsFilter;
   }
@@ -310,28 +319,28 @@ class GenericEntityReader extends TimelineEntityReader {
   private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) {
     EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
     // Events not required.
-    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
-      infoColFamilyList.addFilter(
-          TimelineFilterUtils.createHBaseQualifierFilter(
-              CompareOp.NOT_EQUAL, EntityColumnPrefix.EVENT));
+    if (!hasField(fieldsToRetrieve, Field.EVENTS)) {
+      infoColFamilyList.addFilter(TimelineFilterUtils
+          .createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
+              EntityColumnPrefix.EVENT));
     }
     // info not required.
-    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
-      infoColFamilyList.addFilter(
-          TimelineFilterUtils.createHBaseQualifierFilter(
-              CompareOp.NOT_EQUAL, EntityColumnPrefix.INFO));
+    if (!hasField(fieldsToRetrieve, Field.INFO)) {
+      infoColFamilyList.addFilter(TimelineFilterUtils
+          .createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
+              EntityColumnPrefix.INFO));
     }
     // is related to not required.
-    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
-      infoColFamilyList.addFilter(
-          TimelineFilterUtils.createHBaseQualifierFilter(
-              CompareOp.NOT_EQUAL, EntityColumnPrefix.IS_RELATED_TO));
+    if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
+      infoColFamilyList.addFilter(TimelineFilterUtils
+          .createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
+              EntityColumnPrefix.IS_RELATED_TO));
     }
     // relates to not required.
-    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
-      infoColFamilyList.addFilter(
-          TimelineFilterUtils.createHBaseQualifierFilter(
-              CompareOp.NOT_EQUAL, EntityColumnPrefix.RELATES_TO));
+    if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) {
+      infoColFamilyList.addFilter(TimelineFilterUtils
+          .createHBaseQualifierFilter(CompareOp.NOT_EQUAL,
+              EntityColumnPrefix.RELATES_TO));
     }
   }
 
@@ -348,18 +357,18 @@ class GenericEntityReader extends TimelineEntityReader {
     // CONFS to fields to retrieve in augmentParams() even if not specified.
     if (dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS)) {
       // Create a filter list for configs.
-      listBasedOnFields.addFilter(TimelineFilterUtils.
-          createFilterForConfsOrMetricsToRetrieve(
-              dataToRetrieve.getConfsToRetrieve(),
-              EntityColumnFamily.CONFIGS, EntityColumnPrefix.CONFIG));
+      listBasedOnFields.addFilter(TimelineFilterUtils
+          .createFilterForConfsOrMetricsToRetrieve(
+              dataToRetrieve.getConfsToRetrieve(), EntityColumnFamily.CONFIGS,
+              EntityColumnPrefix.CONFIG));
     }
 
     // Please note that if metricsToRetrieve is specified, we would have added
     // METRICS to fields to retrieve in augmentParams() even if not specified.
     if (dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS)) {
       // Create a filter list for metrics.
-      listBasedOnFields.addFilter(TimelineFilterUtils.
-          createFilterForConfsOrMetricsToRetrieve(
+      listBasedOnFields.addFilter(TimelineFilterUtils
+          .createFilterForConfsOrMetricsToRetrieve(
               dataToRetrieve.getMetricsToRetrieve(),
               EntityColumnFamily.METRICS, EntityColumnPrefix.METRIC));
     }
@@ -375,8 +384,8 @@ class GenericEntityReader extends TimelineEntityReader {
     FilterList infoColFamilyList = new FilterList();
     // By default fetch everything in INFO column family.
     FamilyFilter infoColumnFamily =
-        new FamilyFilter(CompareOp.EQUAL,
-           new BinaryComparator(EntityColumnFamily.INFO.getBytes()));
+        new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(
+            EntityColumnFamily.INFO.getBytes()));
     infoColFamilyList.addFilter(infoColumnFamily);
     if (!isSingleEntityRead() && fetchPartialColsFromInfoFamily()) {
       // We can fetch only some of the columns from info family.
@@ -394,27 +403,27 @@ class GenericEntityReader extends TimelineEntityReader {
   /**
    * Looks up flow context from AppToFlow table.
    *
-   * @param clusterId Cluster Id.
-   * @param appId App Id.
+   * @param appToFlowRowKey to identify Cluster and App Ids.
    * @param hbaseConf HBase configuration.
    * @param conn HBase Connection.
    * @return flow context information.
    * @throws IOException if any problem occurs while fetching flow information.
    */
-  protected FlowContext lookupFlowContext(String clusterId, String appId,
+  protected FlowContext lookupFlowContext(AppToFlowRowKey appToFlowRowKey,
       Configuration hbaseConf, Connection conn) throws IOException {
-    byte[] rowKey = AppToFlowRowKey.getRowKey(clusterId, appId);
+    byte[] rowKey = appToFlowRowKey.getRowKey();
     Get get = new Get(rowKey);
     Result result = appToFlowTable.getResult(hbaseConf, conn, get);
     if (result != null && !result.isEmpty()) {
-      return new FlowContext(
-          AppToFlowColumn.USER_ID.readResult(result).toString(),
-          AppToFlowColumn.FLOW_ID.readResult(result).toString(),
-          ((Number)AppToFlowColumn.FLOW_RUN_ID.readResult(result)).longValue());
+      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 NotFoundException(
-          "Unable to find the context flow ID and flow run ID for clusterId=" +
-          clusterId + ", appId=" + appId);
+          "Unable to find the context flow ID and flow run ID for clusterId="
+              + appToFlowRowKey.getClusterId() + ", appId="
+              + appToFlowRowKey.getAppId());
     }
   }
 
@@ -425,17 +434,21 @@ class GenericEntityReader extends TimelineEntityReader {
     private final String userId;
     private final String flowName;
     private final Long flowRunId;
+
     public FlowContext(String user, String flowName, Long flowRunId) {
       this.userId = user;
       this.flowName = flowName;
       this.flowRunId = flowRunId;
     }
+
     protected String getUserId() {
       return userId;
     }
+
     protected String getFlowName() {
       return flowName;
     }
+
     protected Long getFlowRunId() {
       return flowRunId;
     }
@@ -444,8 +457,8 @@ class GenericEntityReader extends TimelineEntityReader {
   @Override
   protected void validateParams() {
     Preconditions.checkNotNull(getContext(), "context shouldn't be null");
-    Preconditions.checkNotNull(
-        getDataToRetrieve(), "data to retrieve shouldn't be null");
+    Preconditions.checkNotNull(getDataToRetrieve(),
+        "data to retrieve shouldn't be null");
     Preconditions.checkNotNull(getContext().getClusterId(),
         "clusterId shouldn't be null");
     Preconditions.checkNotNull(getContext().getAppId(),
@@ -463,11 +476,13 @@ class GenericEntityReader extends TimelineEntityReader {
       throws IOException {
     TimelineReaderContext context = getContext();
     // In reality all three should be null or neither should be null
-    if (context.getFlowName() == null || context.getFlowRunId() == null ||
-        context.getUserId() == null) {
+    if (context.getFlowName() == null || context.getFlowRunId() == null
+        || context.getUserId() == null) {
       // Get flow context information from AppToFlow table.
-      FlowContext flowContext = lookupFlowContext(
-          context.getClusterId(), context.getAppId(), hbaseConf, conn);
+      AppToFlowRowKey appToFlowRowKey =
+          new AppToFlowRowKey(context.getClusterId(), context.getAppId());
+      FlowContext flowContext =
+          lookupFlowContext(appToFlowRowKey, hbaseConf, conn);
       context.setFlowName(flowContext.flowName);
       context.setFlowRunId(flowContext.flowRunId);
       context.setUserId(flowContext.userId);
@@ -485,9 +500,9 @@ class GenericEntityReader extends TimelineEntityReader {
       FilterList filterList) throws IOException {
     TimelineReaderContext context = getContext();
     byte[] rowKey =
-        EntityRowKey.getRowKey(context.getClusterId(), context.getUserId(),
+        new EntityRowKey(context.getClusterId(), context.getUserId(),
             context.getFlowName(), context.getFlowRunId(), context.getAppId(),
-            context.getEntityType(), context.getEntityId());
+            context.getEntityType(), context.getEntityId()).getRowKey();
     Get get = new Get(rowKey);
     get.setMaxVersions(getDataToRetrieve().getMetricsLimit());
     if (filterList != null && !filterList.getFilters().isEmpty()) {
@@ -497,15 +512,17 @@ class GenericEntityReader extends TimelineEntityReader {
   }
 
   @Override
-  protected ResultScanner getResults(Configuration hbaseConf,
-      Connection conn, FilterList filterList) throws IOException {
+  protected ResultScanner getResults(Configuration hbaseConf, Connection conn,
+      FilterList filterList) throws IOException {
     // Scan through part of the table to find the entities belong to one app
     // and one type
     Scan scan = new Scan();
     TimelineReaderContext context = getContext();
-    scan.setRowPrefixFilter(EntityRowKey.getRowKeyPrefix(
-        context.getClusterId(), context.getUserId(), context.getFlowName(),
-        context.getFlowRunId(), context.getAppId(), context.getEntityType()));
+    RowKeyPrefix<EntityRowKey> entityRowKeyPrefix =
+        new EntityRowKeyPrefix(context.getClusterId(), context.getUserId(),
+            context.getFlowName(), context.getFlowRunId(), context.getAppId(),
+            context.getEntityType());
+    scan.setRowPrefixFilter(entityRowKeyPrefix.getRowKeyPrefix());
     scan.setMaxVersions(getDataToRetrieve().getMetricsLimit());
     if (filterList != null && !filterList.getFilters().isEmpty()) {
       scan.setFilter(filterList);
@@ -535,18 +552,16 @@ class GenericEntityReader extends TimelineEntityReader {
     // locally as relevant HBase filters to filter out rows on the basis of
     // isRelatedTo are not set in HBase scan.
     boolean checkIsRelatedTo =
-        !isSingleEntityRead() && filters.getIsRelatedTo() != null &&
-        filters.getIsRelatedTo().getFilterList().size() > 0;
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO) ||
-        checkIsRelatedTo) {
-      TimelineStorageUtils.readRelationship(
-          entity, result, EntityColumnPrefix.IS_RELATED_TO, true);
-      if (checkIsRelatedTo && !TimelineStorageUtils.matchIsRelatedTo(entity,
-          filters.getIsRelatedTo())) {
+        !isSingleEntityRead() && filters.getIsRelatedTo() != null
+            && filters.getIsRelatedTo().getFilterList().size() > 0;
+    if (hasField(fieldsToRetrieve, Field.IS_RELATED_TO) || checkIsRelatedTo) {
+      readRelationship(entity, result, EntityColumnPrefix.IS_RELATED_TO, true);
+      if (checkIsRelatedTo
+          && !TimelineStorageUtils.matchIsRelatedTo(entity,
+              filters.getIsRelatedTo())) {
         return null;
       }
-      if (!TimelineStorageUtils.hasField(fieldsToRetrieve,
-          Field.IS_RELATED_TO)) {
+      if (!hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
         entity.getIsRelatedToEntities().clear();
       }
     }
@@ -556,31 +571,29 @@ class GenericEntityReader extends TimelineEntityReader {
     // locally as relevant HBase filters to filter out rows on the basis of
     // relatesTo are not set in HBase scan.
     boolean checkRelatesTo =
-        !isSingleEntityRead() && filters.getRelatesTo() != null &&
-        filters.getRelatesTo().getFilterList().size() > 0;
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO) ||
-        checkRelatesTo) {
-      TimelineStorageUtils.readRelationship(
-          entity, result, EntityColumnPrefix.RELATES_TO, false);
-      if (checkRelatesTo && !TimelineStorageUtils.matchRelatesTo(entity,
-          filters.getRelatesTo())) {
+        !isSingleEntityRead() && filters.getRelatesTo() != null
+            && filters.getRelatesTo().getFilterList().size() > 0;
+    if (hasField(fieldsToRetrieve, Field.RELATES_TO)
+        || checkRelatesTo) {
+      readRelationship(entity, result, EntityColumnPrefix.RELATES_TO, false);
+      if (checkRelatesTo
+          && !TimelineStorageUtils.matchRelatesTo(entity,
+              filters.getRelatesTo())) {
         return null;
       }
-      if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
+      if (!hasField(fieldsToRetrieve, Field.RELATES_TO)) {
         entity.getRelatesToEntities().clear();
       }
     }
 
     // fetch info if fieldsToRetrieve contains INFO or ALL.
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
-      TimelineStorageUtils.readKeyValuePairs(
-          entity, result, EntityColumnPrefix.INFO, false);
+    if (hasField(fieldsToRetrieve, Field.INFO)) {
+      readKeyValuePairs(entity, result, EntityColumnPrefix.INFO, false);
     }
 
     // fetch configs if fieldsToRetrieve contains CONFIGS or ALL.
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.CONFIGS)) {
-      TimelineStorageUtils.readKeyValuePairs(
-          entity, result, EntityColumnPrefix.CONFIG, true);
+    if (hasField(fieldsToRetrieve, Field.CONFIGS)) {
+      readKeyValuePairs(entity, result, EntityColumnPrefix.CONFIG, true);
     }
 
     // fetch events and match event filters if they exist. If event filters do
@@ -588,24 +601,48 @@ class GenericEntityReader extends TimelineEntityReader {
     // as relevant HBase filters to filter out rows on the basis of events
     // are not set in HBase scan.
     boolean checkEvents =
-        !isSingleEntityRead() && filters.getEventFilters() != null &&
-        filters.getEventFilters().getFilterList().size() > 0;
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS) ||
-        checkEvents) {
-      TimelineStorageUtils.readEvents(entity, result, EntityColumnPrefix.EVENT);
-      if (checkEvents && !TimelineStorageUtils.matchEventFilters(entity,
-          filters.getEventFilters())) {
+        !isSingleEntityRead() && filters.getEventFilters() != null
+            && filters.getEventFilters().getFilterList().size() > 0;
+    if (hasField(fieldsToRetrieve, Field.EVENTS) || checkEvents) {
+      readEvents(entity, result, EntityColumnPrefix.EVENT);
+      if (checkEvents
+          && !TimelineStorageUtils.matchEventFilters(entity,
+              filters.getEventFilters())) {
         return null;
       }
-      if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
+      if (!hasField(fieldsToRetrieve, Field.EVENTS)) {
         entity.getEvents().clear();
       }
     }
 
     // fetch metrics if fieldsToRetrieve contains METRICS or ALL.
-    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.METRICS)) {
+    if (hasField(fieldsToRetrieve, Field.METRICS)) {
       readMetrics(entity, result, EntityColumnPrefix.METRIC);
     }
     return entity;
   }
+
+  /**
+   * Helper method for reading key-value pairs for either info or config.
+   *
+   * @param <T> Describes the type of column prefix.
+   * @param entity entity to fill.
+   * @param result result from HBase.
+   * @param prefix column prefix.
+   * @param isConfig if true, means we are reading configs, otherwise info.
+   * @throws IOException if any problem is encountered while reading result.
+   */
+  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, stringKeyConverter);
+    if (isConfig) {
+      for (Map.Entry<String, Object> column : columns.entrySet()) {
+        entity.addConfig(column.getKey(), column.getValue().toString());
+      }
+    } else {
+      entity.addInfo(columns);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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
index 852834e..7b294a8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/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
@@ -18,6 +18,9 @@
 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.NavigableMap;
 import java.util.NavigableSet;
@@ -30,15 +33,27 @@ 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.BinaryPrefixComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.FilterList.Operator;
+import org.apache.hadoop.hbase.filter.QualifierFilter;
 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.reader.TimelineDataToRetrieve;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationColumnPrefix;
 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.EventColumnName;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
 
 /**
  * The base class for reading and deserializing timeline entities from the
@@ -68,6 +83,12 @@ public abstract class TimelineEntityReader {
   private boolean sortedKeys = false;
 
   /**
+   * Used to convert strings key components to and from storage format.
+   */
+  private final KeyConverter<String> stringKeyConverter =
+      new StringKeyConverter();
+
+  /**
    * Instantiates a reader for multiple-entity reads.
    *
    * @param ctxt Reader context which defines the scope in which query has to be
@@ -331,7 +352,7 @@ public abstract class TimelineEntityReader {
       ColumnPrefix<?> columnPrefix) throws IOException {
     NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
         columnPrefix.readResultsWithTimestamps(
-            result, StringKeyConverter.getInstance());
+            result, stringKeyConverter);
     for (Map.Entry<String, NavigableMap<Long, Number>> metricResult:
         metricsResult.entrySet()) {
       TimelineMetric metric = new TimelineMetric();
@@ -359,4 +380,117 @@ public abstract class TimelineEntityReader {
   protected void setTable(BaseTable<?> baseTable) {
     this.table = baseTable;
   }
+
+  /**
+   * Check if we have a certain field amongst fields to retrieve. This method
+   * checks against {@link Field#ALL} as well because that would mean field
+   * passed needs to be matched.
+   *
+   * @param fieldsToRetrieve fields to be retrieved.
+   * @param requiredField fields to be checked in fieldsToRetrieve.
+   * @return true if has the required field, false otherwise.
+   */
+  protected boolean hasField(EnumSet<Field> fieldsToRetrieve,
+      Field requiredField) {
+    return fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(requiredField);
+  }
+
+  /**
+   * Create a filter list of qualifier filters based on passed set of columns.
+   *
+   * @param <T> Describes the type of column prefix.
+   * @param colPrefix Column Prefix.
+   * @param columns set of column qualifiers.
+   * @return filter list.
+   */
+  protected <T> FilterList createFiltersFromColumnQualifiers(
+      ColumnPrefix<T> colPrefix, Set<String> columns) {
+    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
+    for (String column : columns) {
+      // For columns which have compound column qualifiers (eg. events), we need
+      // to include the required separator.
+      byte[] compoundColQual = createColQualifierPrefix(colPrefix, column);
+      list.addFilter(new QualifierFilter(CompareOp.EQUAL,
+          new BinaryPrefixComparator(colPrefix
+              .getColumnPrefixBytes(compoundColQual))));
+    }
+    return list;
+  }
+
+  protected <T> byte[] createColQualifierPrefix(ColumnPrefix<T> colPrefix,
+      String column) {
+    if (colPrefix == ApplicationColumnPrefix.EVENT
+        || colPrefix == EntityColumnPrefix.EVENT) {
+      return new EventColumnName(column, null, null).getColumnQualifier();
+    } else {
+      return stringKeyConverter.encode(column);
+    }
+  }
+
+  /**
+   * Helper method for reading relationship.
+   *
+   * @param <T> Describes the type of column prefix.
+   * @param entity entity to fill.
+   * @param result result from HBase.
+   * @param prefix column prefix.
+   * @param isRelatedTo if true, means relationship is to be added to
+   *          isRelatedTo, otherwise its added to relatesTo.
+   * @throws IOException if any problem is encountered while reading result.
+   */
+  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, stringKeyConverter);
+    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);
+        }
+      }
+    }
+  }
+
+  /**
+   * 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.
+   *
+   * @param <T> Describes the type of column prefix.
+   * @param entity entity to fill.
+   * @param result HBase Result.
+   * @param prefix column prefix.
+   * @throws IOException if any problem is encountered while reading result.
+   */
+  protected static <T> void readEvents(TimelineEntity entity, Result result,
+      ColumnPrefix<T> prefix) throws IOException {
+    Map<String, TimelineEvent> eventsMap = new HashMap<>();
+    Map<EventColumnName, Object> eventsResult =
+        prefix.readResults(result, new EventColumnNameConverter());
+    for (Map.Entry<EventColumnName, Object>
+             eventResult : eventsResult.entrySet()) {
+      EventColumnName eventColumnName = eventResult.getKey();
+      String key = eventColumnName.getId() +
+          Long.toString(eventColumnName.getTimestamp());
+      // Retrieve previously seen event to add to it
+      TimelineEvent event = eventsMap.get(key);
+      if (event == null) {
+        // First time we're seeing this event, add it to the eventsMap
+        event = new TimelineEvent();
+        event.setId(eventColumnName.getId());
+        event.setTimestamp(eventColumnName.getTimestamp());
+        eventsMap.put(key, event);
+      }
+      if (eventColumnName.getInfoKey() != null) {
+        event.addInfo(eventColumnName.getInfoKey(), eventResult.getValue());
+      }
+    }
+    Set<TimelineEvent> eventsSet = new HashSet<>(eventsMap.values());
+    entity.addEvents(eventsSet);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java
index 74e4b5d..58df970 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.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/TestKeyConverters.java
@@ -24,220 +24,13 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
-import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKeyConverter;
 import org.junit.Test;
 
 public class TestKeyConverters {
-  private final static String QUALIFIER_SEP = Separator.QUALIFIERS.getValue();
-  private final static byte[] QUALIFIER_SEP_BYTES =
-      Bytes.toBytes(QUALIFIER_SEP);
-  private final static String CLUSTER = "cl" + QUALIFIER_SEP + "uster";
-  private final static String USER = QUALIFIER_SEP + "user";
-  private final static String FLOW_NAME =
-      "dummy_" + QUALIFIER_SEP + "flow" + QUALIFIER_SEP;
-  private final static Long FLOW_RUN_ID;
-  private final static String APPLICATION_ID;
-  static {
-    long runid = Long.MAX_VALUE - 900L;
-    byte[] longMaxByteArr = Bytes.toBytes(Long.MAX_VALUE);
-    byte[] byteArr = Bytes.toBytes(runid);
-    int sepByteLen = QUALIFIER_SEP_BYTES.length;
-    if (sepByteLen <= byteArr.length) {
-      for (int i = 0; i < sepByteLen; i++) {
-        byteArr[i] = (byte)(longMaxByteArr[i] - QUALIFIER_SEP_BYTES[i]);
-      }
-    }
-    FLOW_RUN_ID = Bytes.toLong(byteArr);
-    long clusterTs = System.currentTimeMillis();
-    byteArr = Bytes.toBytes(clusterTs);
-    if (sepByteLen <= byteArr.length) {
-      for (int i = 0; i < sepByteLen; i++) {
-        byteArr[byteArr.length - sepByteLen + i] =
-            (byte)(longMaxByteArr[byteArr.length - sepByteLen + i] -
-                QUALIFIER_SEP_BYTES[i]);
-      }
-    }
-    clusterTs = Bytes.toLong(byteArr);
-    int seqId = 222;
-    APPLICATION_ID = ApplicationId.newInstance(clusterTs, seqId).toString();
-  }
-
-  private static void verifyRowPrefixBytes(byte[] byteRowKeyPrefix) {
-    int sepLen = QUALIFIER_SEP_BYTES.length;
-    for (int i = 0; i < sepLen; i++) {
-      assertTrue("Row key prefix not encoded properly.",
-        byteRowKeyPrefix[byteRowKeyPrefix.length - sepLen  + i] ==
-            QUALIFIER_SEP_BYTES[i]);
-    }
-  }
-
-  @Test
-  public void testFlowActivityRowKeyConverter() {
-    Long ts = TimelineStorageUtils.getTopOfTheDayTimestamp(1459900830000L);
-    byte[] byteRowKey = FlowActivityRowKeyConverter.getInstance().encode(
-        new FlowActivityRowKey(CLUSTER, ts, USER, FLOW_NAME));
-    FlowActivityRowKey rowKey =
-        FlowActivityRowKeyConverter.getInstance().decode(byteRowKey);
-    assertEquals(CLUSTER, rowKey.getClusterId());
-    assertEquals(ts, rowKey.getDayTimestamp());
-    assertEquals(USER, rowKey.getUserId());
-    assertEquals(FLOW_NAME, rowKey.getFlowName());
-
-    byte[] byteRowKeyPrefix = FlowActivityRowKeyConverter.getInstance().encode(
-        new FlowActivityRowKey(CLUSTER, null, null, null));
-    byte[][] splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
-        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
-    assertEquals(2, splits.length);
-    assertEquals(0, splits[1].length);
-    assertEquals(CLUSTER,
-        Separator.QUALIFIERS.decode(Bytes.toString(splits[0])));
-    verifyRowPrefixBytes(byteRowKeyPrefix);
-
-    byteRowKeyPrefix = FlowActivityRowKeyConverter.getInstance().encode(
-        new FlowActivityRowKey(CLUSTER, ts, null, null));
-    splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
-        Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE });
-    assertEquals(3, splits.length);
-    assertEquals(0, splits[2].length);
-    assertEquals(CLUSTER,
-        Separator.QUALIFIERS.decode(Bytes.toString(splits[0])));
-    assertEquals(ts, (Long) TimelineStorageUtils.invertLong(
-        Bytes.toLong(splits[1])));
-    verifyRowPrefixBytes(byteRowKeyPrefix);
-  }
-
-  @Test
-  public void testFlowRunRowKeyConverter() {
-    byte[] byteRowKey = FlowRunRowKeyConverter.getInstance().encode(
-        new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID));
-    FlowRunRowKey rowKey =
-        FlowRunRowKeyConverter.getInstance().decode(byteRowKey);
-    assertEquals(CLUSTER, rowKey.getClusterId());
-    assertEquals(USER, rowKey.getUserId());
-    assertEquals(FLOW_NAME, rowKey.getFlowName());
-    assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
-
-    byte[] byteRowKeyPrefix = FlowRunRowKeyConverter.getInstance().encode(
-        new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, null));
-    byte[][] splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
-        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
-        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
-    assertEquals(4, splits.length);
-    assertEquals(0, splits[3].length);
-    assertEquals(FLOW_NAME,
-        Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
-    verifyRowPrefixBytes(byteRowKeyPrefix);
-  }
-
-  @Test
-  public void testApplicationRowKeyConverter() {
-    byte[] byteRowKey = ApplicationRowKeyConverter.getInstance().encode(
-        new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID,
-            APPLICATION_ID));
-    ApplicationRowKey rowKey =
-        ApplicationRowKeyConverter.getInstance().decode(byteRowKey);
-    assertEquals(CLUSTER, rowKey.getClusterId());
-    assertEquals(USER, rowKey.getUserId());
-    assertEquals(FLOW_NAME, rowKey.getFlowName());
-    assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
-    assertEquals(APPLICATION_ID, rowKey.getAppId());
-
-    byte[] byteRowKeyPrefix = ApplicationRowKeyConverter.getInstance().encode(
-        new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, null));
-    byte[][] splits =
-        Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
-            Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
-            Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
-            Separator.VARIABLE_SIZE });
-    assertEquals(5, splits.length);
-    assertEquals(0, splits[4].length);
-    assertEquals(FLOW_NAME,
-        Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
-    assertEquals(FLOW_RUN_ID, (Long)TimelineStorageUtils.invertLong(
-        Bytes.toLong(splits[3])));
-    verifyRowPrefixBytes(byteRowKeyPrefix);
-
-    byteRowKeyPrefix = ApplicationRowKeyConverter.getInstance().encode(
-        new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, null, null));
-    splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
-        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
-        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
-    assertEquals(4, splits.length);
-    assertEquals(0, splits[3].length);
-    assertEquals(FLOW_NAME,
-        Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
-    verifyRowPrefixBytes(byteRowKeyPrefix);
-  }
-
-  @Test
-  public void testEntityRowKeyConverter() {
-    String entityId = "!ent!ity!!id!";
-    String entityType = "entity!Type";
-    byte[] byteRowKey = EntityRowKeyConverter.getInstance().encode(
-        new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID,
-            entityType, entityId));
-    EntityRowKey rowKey =
-        EntityRowKeyConverter.getInstance().decode(byteRowKey);
-    assertEquals(CLUSTER, rowKey.getClusterId());
-    assertEquals(USER, rowKey.getUserId());
-    assertEquals(FLOW_NAME, rowKey.getFlowName());
-    assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
-    assertEquals(APPLICATION_ID, rowKey.getAppId());
-    assertEquals(entityType, rowKey.getEntityType());
-    assertEquals(entityId, rowKey.getEntityId());
-
-    byte[] byteRowKeyPrefix = EntityRowKeyConverter.getInstance().encode(
-        new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID,
-            entityType, null));
-    byte[][] splits =
-        Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
-            Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
-            Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
-            AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE,
-            Separator.VARIABLE_SIZE });
-    assertEquals(7, splits.length);
-    assertEquals(0, splits[6].length);
-    assertEquals(APPLICATION_ID,
-        AppIdKeyConverter.getInstance().decode(splits[4]));
-    assertEquals(entityType, Separator.QUALIFIERS.decode(
-        Bytes.toString(splits[5])));
-    verifyRowPrefixBytes(byteRowKeyPrefix);
-
-    byteRowKeyPrefix = EntityRowKeyConverter.getInstance().encode(
-        new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID,
-        null, null));
-    splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
-        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
-        Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
-        AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE });
-    assertEquals(6, splits.length);
-    assertEquals(0, splits[5].length);
-    assertEquals(APPLICATION_ID,
-        AppIdKeyConverter.getInstance().decode(splits[4]));
-    verifyRowPrefixBytes(byteRowKeyPrefix);
-  }
-
-  @Test
-  public void testAppToFlowRowKeyConverter() {
-    byte[] byteRowKey = AppToFlowRowKeyConverter.getInstance().encode(
-        new AppToFlowRowKey(CLUSTER, APPLICATION_ID));
-    AppToFlowRowKey rowKey =
-        AppToFlowRowKeyConverter.getInstance().decode(byteRowKey);
-    assertEquals(CLUSTER, rowKey.getClusterId());
-    assertEquals(APPLICATION_ID, rowKey.getAppId());
-  }
 
   @Test
   public void testAppIdKeyConverter() {
+    AppIdKeyConverter appIdKeyConverter = new AppIdKeyConverter();
     long currentTs = System.currentTimeMillis();
     ApplicationId appId1 = ApplicationId.newInstance(currentTs, 1);
     ApplicationId appId2 = ApplicationId.newInstance(currentTs, 2);
@@ -245,18 +38,19 @@ public class TestKeyConverters {
     String appIdStr1 = appId1.toString();
     String appIdStr2 = appId2.toString();
     String appIdStr3 = appId3.toString();
-    byte[] appIdBytes1 = AppIdKeyConverter.getInstance().encode(appIdStr1);
-    byte[] appIdBytes2 = AppIdKeyConverter.getInstance().encode(appIdStr2);
-    byte[] appIdBytes3 = AppIdKeyConverter.getInstance().encode(appIdStr3);
+    byte[] appIdBytes1 = appIdKeyConverter.encode(appIdStr1);
+    byte[] appIdBytes2 = appIdKeyConverter.encode(appIdStr2);
+    byte[] appIdBytes3 = appIdKeyConverter.encode(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 = AppIdKeyConverter.getInstance().decode(appIdBytes1);
-    String decodedAppId2 = AppIdKeyConverter.getInstance().decode(appIdBytes2);
-    String decodedAppId3 = AppIdKeyConverter.getInstance().decode(appIdBytes3);
+    assertTrue(
+        "Ordering of app ids' is incorrect",
+        Bytes.compareTo(appIdBytes1, appIdBytes2) > 0
+            && Bytes.compareTo(appIdBytes1, appIdBytes3) > 0
+            && Bytes.compareTo(appIdBytes2, appIdBytes3) > 0);
+    String decodedAppId1 = appIdKeyConverter.decode(appIdBytes1);
+    String decodedAppId2 = appIdKeyConverter.decode(appIdBytes2);
+    String decodedAppId3 = appIdKeyConverter.decode(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",
@@ -273,21 +67,64 @@ public class TestKeyConverters {
         Bytes.createMaxByteArray(Bytes.SIZEOF_LONG - valSepBytes.length);
     byte[] ts = Bytes.add(valSepBytes, maxByteArr);
     Long eventTs = Bytes.toLong(ts);
-    byte[] byteEventColName = EventColumnNameConverter.getInstance().encode(
-        new EventColumnName(eventId, eventTs, null));
+    byte[] byteEventColName =
+        new EventColumnName(eventId, eventTs, null).getColumnQualifier();
+    KeyConverter<EventColumnName> eventColumnNameConverter =
+        new EventColumnNameConverter();
     EventColumnName eventColName =
-        EventColumnNameConverter.getInstance().decode(byteEventColName);
+        eventColumnNameConverter.decode(byteEventColName);
     assertEquals(eventId, eventColName.getId());
     assertEquals(eventTs, eventColName.getTimestamp());
     assertNull(eventColName.getInfoKey());
 
     String infoKey = "f=oo_event_in=fo=_key";
-    byteEventColName = EventColumnNameConverter.getInstance().encode(
-        new EventColumnName(eventId, eventTs, infoKey));
-    eventColName =
-        EventColumnNameConverter.getInstance().decode(byteEventColName);
+    byteEventColName =
+        new EventColumnName(eventId, eventTs, infoKey).getColumnQualifier();
+    eventColName = eventColumnNameConverter.decode(byteEventColName);
     assertEquals(eventId, eventColName.getId());
     assertEquals(eventTs, eventColName.getTimestamp());
     assertEquals(infoKey, eventColName.getInfoKey());
   }
+
+  @Test
+  public void testLongKeyConverter() {
+    LongKeyConverter longKeyConverter = new LongKeyConverter();
+    confirmLongKeyConverter(longKeyConverter, Long.MIN_VALUE);
+    confirmLongKeyConverter(longKeyConverter, -1234567890L);
+    confirmLongKeyConverter(longKeyConverter, -128L);
+    confirmLongKeyConverter(longKeyConverter, -127L);
+    confirmLongKeyConverter(longKeyConverter, -1L);
+    confirmLongKeyConverter(longKeyConverter, 0L);
+    confirmLongKeyConverter(longKeyConverter, 1L);
+    confirmLongKeyConverter(longKeyConverter, 127L);
+    confirmLongKeyConverter(longKeyConverter, 128L);
+    confirmLongKeyConverter(longKeyConverter, 1234567890L);
+    confirmLongKeyConverter(longKeyConverter, Long.MAX_VALUE);
+  }
+
+  private void confirmLongKeyConverter(LongKeyConverter longKeyConverter,
+      Long testValue) {
+    Long decoded = longKeyConverter.decode(longKeyConverter.encode(testValue));
+    assertEquals(testValue, decoded);
+  }
+
+  @Test
+  public void testStringKeyConverter() {
+    StringKeyConverter stringKeyConverter = new StringKeyConverter();
+    String phrase = "QuackAttack now!";
+
+    for (int i = 0; i < phrase.length(); i++) {
+      String sub = phrase.substring(i, phrase.length());
+      confirmStrignKeyConverter(stringKeyConverter, sub);
+      confirmStrignKeyConverter(stringKeyConverter, sub + sub);
+    }
+  }
+
+  private void confirmStrignKeyConverter(StringKeyConverter stringKeyConverter,
+      String testValue) {
+    String decoded =
+        stringKeyConverter.decode(stringKeyConverter.encode(testValue));
+    assertEquals(testValue, decoded);
+  }
+
 }


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


[26/50] [abbrv] hadoop git commit: YARN-5050. Code cleanup for TestDistributedShell (Li Lu via sjlee)

Posted by vr...@apache.org.
YARN-5050. Code cleanup for TestDistributedShell (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/c9283e16
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c9283e16
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c9283e16

Branch: refs/heads/YARN-2928
Commit: c9283e16675ed56436e382ee94b64b648180837d
Parents: 1c0d74d
Author: Sangjin Lee <sj...@apache.org>
Authored: Thu May 19 17:25:05 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:09 2016 -0700

----------------------------------------------------------------------
 .../distributedshell/TestDistributedShell.java         | 13 +++++++------
 1 file changed, 7 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c9283e16/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 ba11e60..c02cd85 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
@@ -135,6 +135,8 @@ public class TestDistributedShell {
 
     conf = new YarnConfiguration();
     conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128);
+    // reduce the teardown waiting time
+    conf.setLong(YarnConfiguration.DISPATCHER_DRAIN_EVENTS_TIMEOUT, 1000);
     conf.set("yarn.log.dir", "target");
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     // mark if we need to launch the v1 timeline server
@@ -183,9 +185,6 @@ public class TestDistributedShell {
       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);
     } else {
       Assert.fail("Wrong timeline version number: " + timelineVersion);
     }
@@ -280,7 +279,7 @@ public class TestDistributedShell {
     testDSShell(true);
   }
 
-  @Test(timeout=90000)
+  @Test
   @TimelineVersion(2.0f)
   public void testDSShellWithoutDomainV2() throws Exception {
     testDSShell(false);
@@ -290,12 +289,14 @@ public class TestDistributedShell {
     testDSShell(haveDomain, true);
   }
 
-  @Test(timeout=90000)
+  @Test
+  @TimelineVersion(2.0f)
   public void testDSShellWithoutDomainV2DefaultFlow() throws Exception {
     testDSShell(false, true);
   }
 
-  @Test(timeout=90000)
+  @Test
+  @TimelineVersion(2.0f)
   public void testDSShellWithoutDomainV2CustomizedFlow() throws Exception {
     testDSShell(false, false);
   }


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


[06/50] [abbrv] hadoop git commit: Addendum to YARN-3863. Deleted files that were added incorrectly.

Posted by vr...@apache.org.
Addendum to YARN-3863. Deleted files that were added incorrectly.


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

Branch: refs/heads/YARN-2928
Commit: a5bf4fa37c119128c2f82fbdab77e3c73c60c5ef
Parents: 366eb54
Author: Sangjin Lee <sj...@apache.org>
Authored: Tue Apr 12 12:32:43 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:05 2016 -0700

----------------------------------------------------------------------
 .../reader/filter/TimelineExistsFilter.java     | 62 -----------------
 .../reader/filter/TimelineKeyValueFilter.java   | 48 -------------
 .../reader/filter/TimelineKeyValuesFilter.java  | 71 --------------------
 .../common/TimelineEntityFiltersType.java       | 71 --------------------
 4 files changed, 252 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5bf4fa3/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java b/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java
deleted file mode 100644
index 36d0d7b..0000000
--- a/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java
+++ /dev/null
@@ -1,62 +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.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 existence of a
- * value.
- */
-@Private
-@Unstable
-public class TimelineExistsFilter extends TimelineFilter {
-
-  private final TimelineCompareOp compareOp;
-  private final String value;
-
-  public TimelineExistsFilter(TimelineCompareOp op, String value) {
-    this.value = value;
-    if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
-      throw new IllegalArgumentException("CompareOp for exists filter should " +
-          "be EQUAL or NOT_EQUAL");
-    }
-    this.compareOp = op;
-  }
-
-  @Override
-  public TimelineFilterType getFilterType() {
-    return TimelineFilterType.EXISTS;
-  }
-
-  public String getValue() {
-    return value;
-  }
-
-  public TimelineCompareOp getCompareOp() {
-    return compareOp;
-  }
-
-  @Override
-  public String toString() {
-    return String.format("%s (%s %s)",
-        this.getClass().getSimpleName(), this.compareOp.name(), this.value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5bf4fa3/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java b/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java
deleted file mode 100644
index 58f0ee9..0000000
--- a/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java
+++ /dev/null
@@ -1,48 +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.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
- * being equal or not to the values in back-end store.
- */
-@Private
-@Unstable
-public class TimelineKeyValueFilter extends TimelineCompareFilter {
-  public TimelineKeyValueFilter(TimelineCompareOp op, String key, Object val,
-      boolean keyMustExistFlag) {
-    super(op, key, val, keyMustExistFlag);
-    if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
-      throw new IllegalArgumentException("TimelineCompareOp for equality"
-          + " filter should be EQUAL or NOT_EQUAL");
-    }
-  }
-
-  public TimelineKeyValueFilter(TimelineCompareOp op, String key, Object val) {
-    this(op, key, val, true);
-  }
-
-  @Override
-  public TimelineFilterType getFilterType() {
-    return TimelineFilterType.KEY_VALUE;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5bf4fa3/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java b/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java
deleted file mode 100644
index 0d34d47..0000000
--- a/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java
+++ /dev/null
@@ -1,71 +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.filter;
-
-import java.util.Set;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-
-/**
- * Filter class which represents filter to be applied based on multiple values
- * for a key and these values being equal or not equal to values in back-end
- * store.
- */
-@Private
-@Unstable
-public class TimelineKeyValuesFilter extends TimelineFilter {
-  private final TimelineCompareOp compareOp;
-  private final String key;
-  private final Set<Object> values;
-  public TimelineKeyValuesFilter(TimelineCompareOp op, String key,
-      Set<Object> values) {
-    if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
-      throw new IllegalArgumentException("TimelineCompareOp for multi value "
-          + "equality filter should be EQUAL or NOT_EQUAL");
-    }
-    this.compareOp = op;
-    this.key = key;
-    this.values = values;
-  }
-
-  @Override
-  public TimelineFilterType getFilterType() {
-    return TimelineFilterType.KEY_VALUES;
-  }
-
-  public String getKey() {
-    return key;
-  }
-
-  public Set<Object> getValues() {
-    return values;
-  }
-
-  public TimelineCompareOp getCompareOp() {
-    return compareOp;
-  }
-
-  @Override
-  public String toString() {
-    return String.format("%s (%s, %s:%s)",
-        this.getClass().getSimpleName(), this.compareOp.name(),
-        this.key, (values == null) ? "" : values.toString());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5bf4fa3/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java b/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java
deleted file mode 100644
index 4099e92..0000000
--- a/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java
+++ /dev/null
@@ -1,71 +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.server.timelineservice.reader.filter.TimelineFilter.TimelineFilterType;
-
-/**
- * Used to define which filter to match.
- */
-enum TimelineEntityFiltersType {
-  CONFIG {
-    boolean isValidFilter(TimelineFilterType filterType) {
-      return filterType == TimelineFilterType.LIST ||
-          filterType == TimelineFilterType.KEY_VALUE;
-    }
-  },
-  INFO {
-    boolean isValidFilter(TimelineFilterType filterType) {
-      return filterType == TimelineFilterType.LIST ||
-          filterType == TimelineFilterType.KEY_VALUE;
-    }
-  },
-  METRIC {
-    boolean isValidFilter(TimelineFilterType filterType) {
-      return filterType == TimelineFilterType.LIST ||
-          filterType == TimelineFilterType.COMPARE;
-    }
-  },
-  EVENT {
-    boolean isValidFilter(TimelineFilterType filterType) {
-      return filterType == TimelineFilterType.LIST ||
-          filterType == TimelineFilterType.EXISTS;
-    }
-  },
-  IS_RELATED_TO {
-    boolean isValidFilter(TimelineFilterType filterType) {
-      return filterType == TimelineFilterType.LIST ||
-          filterType == TimelineFilterType.KEY_VALUES;
-    }
-  },
-  RELATES_TO {
-    boolean isValidFilter(TimelineFilterType filterType) {
-      return filterType == TimelineFilterType.LIST ||
-          filterType == TimelineFilterType.KEY_VALUES;
-    }
-  };
-
-  /**
-   * Checks whether filter type is valid for the filter being matched.
-   *
-   * @param filterType filter type.
-   * @return true, if its a valid filter, false otherwise.
-   */
-  abstract boolean isValidFilter(TimelineFilterType filterType);
-}
\ No newline at end of file


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


[13/50] [abbrv] hadoop git commit: MAPREDUCE-6424. Store MR counters as timeline metrics instead of event. (Naganarasimha G R via varunsaxena)

Posted by vr...@apache.org.
MAPREDUCE-6424. Store MR counters as timeline metrics instead of event. (Naganarasimha G R via varunsaxena)


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

Branch: refs/heads/YARN-2928
Commit: 5f5c98e53e4918487c8a450daa197130019427ac
Parents: 89e4a94
Author: Varun Saxena <va...@apache.org>
Authored: Sun May 1 17:17:24 2016 +0530
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:07 2016 -0700

----------------------------------------------------------------------
 .../jobhistory/JobHistoryEventHandler.java      | 29 +++++++-
 .../hadoop/mapreduce/jobhistory/TestEvents.java | 12 +++-
 .../mapreduce/jobhistory/AMStartedEvent.java    | 12 +++-
 .../mapreduce/jobhistory/HistoryEvent.java      |  6 ++
 .../mapreduce/jobhistory/JobFinishedEvent.java  | 20 ++++--
 .../jobhistory/JobInfoChangeEvent.java          | 10 ++-
 .../mapreduce/jobhistory/JobInitedEvent.java    | 11 ++-
 .../jobhistory/JobPriorityChangeEvent.java      | 11 ++-
 .../jobhistory/JobQueueChangeEvent.java         |  8 +++
 .../jobhistory/JobStatusChangedEvent.java       | 11 ++-
 .../mapreduce/jobhistory/JobSubmittedEvent.java | 10 ++-
 .../JobUnsuccessfulCompletionEvent.java         | 11 ++-
 .../jobhistory/MapAttemptFinishedEvent.java     | 12 +++-
 .../jobhistory/NormalizedResourceEvent.java     |  8 +++
 .../jobhistory/ReduceAttemptFinishedEvent.java  | 12 +++-
 .../jobhistory/TaskAttemptFinishedEvent.java    | 11 ++-
 .../jobhistory/TaskAttemptStartedEvent.java     | 13 +++-
 .../TaskAttemptUnsuccessfulCompletionEvent.java | 17 +++--
 .../mapreduce/jobhistory/TaskFailedEvent.java   | 14 ++--
 .../mapreduce/jobhistory/TaskFinishedEvent.java | 13 +++-
 .../mapreduce/jobhistory/TaskStartedEvent.java  |  8 +++
 .../mapreduce/jobhistory/TaskUpdatedEvent.java  | 11 ++-
 .../mapreduce/util/JobHistoryEventUtils.java    | 21 +++++-
 .../mapred/TestMRTimelineEventHandling.java     | 74 +++++++++++++++++++-
 .../org/apache/hadoop/mapred/UtilsForTests.java | 39 +++++++++--
 25 files changed, 345 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/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 d7cf191..528b450 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
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
 import java.util.concurrent.BlockingQueue;
@@ -68,6 +69,8 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
 import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
+import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -1072,6 +1075,15 @@ public class JobHistoryEventHandler extends AbstractService
     return entity;
   }
   
+  // create ApplicationEntity with job finished Metrics from HistoryEvent
+  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+      createAppEntityWithJobMetrics(HistoryEvent event, JobId jobId) {
+    ApplicationEntity entity = new ApplicationEntity();
+    entity.setId(jobId.getAppId().toString());
+    entity.setMetrics(event.getTimelineMetrics());
+    return entity;
+  }
+
   // create BaseEntity from HistoryEvent with adding other info, like: 
   // timestamp and entityType.
   private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity 
@@ -1088,6 +1100,10 @@ public class JobHistoryEventHandler extends AbstractService
     if (setCreatedTime) {
       entity.setCreatedTime(timestamp);
     }
+    Set<TimelineMetric> timelineMetrics = event.getTimelineMetrics();
+    if (timelineMetrics != null) {
+      entity.setMetrics(timelineMetrics);
+    }
     return entity;
   }
   
@@ -1203,10 +1219,17 @@ public class JobHistoryEventHandler extends AbstractService
           " and handled by timeline service.");
       return;
     }
+
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+        appEntityWithJobMetrics = null;
     if (taskId == null) {
       // JobEntity
       tEntity = createJobEntity(event, timestamp, jobId,
           MAPREDUCE_JOB_ENTITY_TYPE, setCreatedTime);
+      if (event.getEventType() == EventType.JOB_FINISHED
+          && event.getTimelineMetrics() != null) {
+        appEntityWithJobMetrics = createAppEntityWithJobMetrics(event, jobId);
+      }
     } else {
       if (taskAttemptId == null) {
         // TaskEntity
@@ -1221,7 +1244,11 @@ public class JobHistoryEventHandler extends AbstractService
       }
     }
     try {
-      timelineClient.putEntitiesAsync(tEntity);
+      if (appEntityWithJobMetrics == null) {
+        timelineClient.putEntitiesAsync(tEntity);
+      } else {
+        timelineClient.putEntities(tEntity, appEntityWithJobMetrics);
+      }
     } catch (IOException | YarnException e) {
       LOG.error("Failed to process Event " + event.getEventType()
           + " for the job : " + jobId, e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
index 6eb8a43..f5f4a3c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
@@ -18,13 +18,15 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
-
-import static org.junit.Assert.*;
+import java.util.Set;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -36,6 +38,7 @@ import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.junit.Test;
 
 public class TestEvents {
@@ -410,6 +413,11 @@ public class TestEvents {
     public TimelineEvent toTimelineEvent() {
       return null;
     }
+
+    @Override
+    public Set<TimelineMetric> getTimelineMetrics() {
+      return null;
+    }
     
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
index f98dee5..82db9f3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
@@ -18,16 +18,18 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
-import org.apache.avro.util.Utf8;
-
 /**
  * Event to record start of a task attempt
  * 
@@ -183,5 +185,9 @@ public class AMStartedEvent implements HistoryEvent {
     tEvent.addInfo("START_TIME", getStartTime());
     return tEvent;
   }
-  
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java
index 61ce217..cdafe46 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java
@@ -18,9 +18,12 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Interface for event wrapper classes.  Implementations each wrap an
@@ -41,4 +44,7 @@ public interface HistoryEvent {
   
   /** Map HistoryEvent to TimelineEvent */
   TimelineEvent toTimelineEvent();
+
+  /** Counters or Metrics if any else return null. */
+  Set<TimelineMetric> getTimelineMetrics();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
index 80d3ee6..2efbc57 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -26,6 +28,7 @@ import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record successful completion of job
@@ -148,14 +151,19 @@ public class JobFinishedEvent  implements HistoryEvent {
     tEvent.addInfo("FAILED_REDUCES", getFailedReduces());
     tEvent.addInfo("FINISHED_MAPS", getFinishedMaps());
     tEvent.addInfo("FINISHED_REDUCES", getFinishedReduces());
-    tEvent.addInfo("MAP_COUNTERS_GROUPS",
-        JobHistoryEventUtils.countersToJSON(getMapCounters()));
-    tEvent.addInfo("REDUCE_COUNTERS_GROUPS",
-        JobHistoryEventUtils.countersToJSON(getReduceCounters()));
-    tEvent.addInfo("TOTAL_COUNTERS_GROUPS",
-        JobHistoryEventUtils.countersToJSON(getTotalCounters()));
     // TODO replace SUCCEEDED with JobState.SUCCEEDED.toString()
     tEvent.addInfo("JOB_STATUS", "SUCCEEDED");
     return tEvent;
   }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> jobMetrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getMapCounters(), finishTime);
+    jobMetrics.addAll(JobHistoryEventUtils
+        .countersToTimelineMetric(getReduceCounters(), finishTime));
+    jobMetrics.addAll(JobHistoryEventUtils
+        .countersToTimelineMetric(getTotalCounters(), finishTime));
+    return jobMetrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
index ad82443..eb9ad0d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
@@ -18,15 +18,15 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
-import java.io.IOException;
+import java.util.Set;
 
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record changes in the submit and launch time of
@@ -76,4 +76,8 @@ public class JobInfoChangeEvent implements HistoryEvent {
     return tEvent;
   }
 
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
index 3e0f2f7..d91208c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
@@ -18,13 +18,15 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the initialization of a job
@@ -87,4 +89,9 @@ public class JobInitedEvent implements HistoryEvent {
     tEvent.addInfo("UBERIZED", getUberized());
     return tEvent;
   }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
index 5deea0a..ac4943e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
@@ -18,16 +18,16 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
-import java.io.IOException;
+import java.util.Set;
 
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapred.JobPriority;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the change of priority of a job
@@ -75,4 +75,9 @@ public class JobPriorityChangeEvent implements HistoryEvent {
     return tEvent;
   }
 
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java
index b9dd359..b12006b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java
@@ -18,10 +18,13 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 @SuppressWarnings("deprecation")
 public class JobQueueChangeEvent implements HistoryEvent {
@@ -70,4 +73,9 @@ public class JobQueueChangeEvent implements HistoryEvent {
     return tEvent;
   }
 
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
index a4f2da2..0b33b4f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
@@ -18,15 +18,15 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
-import java.io.IOException;
+import java.util.Set;
 
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the change of status for a job
@@ -71,4 +71,9 @@ public class JobStatusChangedEvent implements HistoryEvent {
     return tEvent;
   }
 
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
index 47b2840..07edb58 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
@@ -21,7 +21,9 @@ package org.apache.hadoop.mapreduce.jobhistory;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.JobACL;
@@ -29,8 +31,7 @@ import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the submission of a job
@@ -229,4 +230,9 @@ public class JobSubmittedEvent implements HistoryEvent {
     return tEvent;
   }
 
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
index ea9798c..5d3c45f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
@@ -18,7 +18,8 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
-import com.google.common.base.Joiner;
+import java.util.Collections;
+import java.util.Set;
 
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -26,8 +27,9 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
-import java.util.Collections;
+import com.google.common.base.Joiner;
 
 /**
  * Event to record Failed and Killed completion of jobs
@@ -135,4 +137,9 @@ public class JobUnsuccessfulCompletionEvent implements HistoryEvent {
     tEvent.addInfo("FINISHED_REDUCES", getFinishedReduces());
     return tEvent;
   }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
index 36737e9..1d28335 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -29,6 +31,7 @@ import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record successful completion of a map attempt
@@ -230,8 +233,6 @@ public class MapAttemptFinishedEvent implements HistoryEvent {
     tEvent.addInfo("STATUS", getTaskStatus());
     tEvent.addInfo("STATE", getState());
     tEvent.addInfo("MAP_FINISH_TIME", getMapFinishTime());
-    tEvent.addInfo("COUNTERS_GROUPS",
-        JobHistoryEventUtils.countersToJSON(getCounters()));
     tEvent.addInfo("HOSTNAME", getHostname());
     tEvent.addInfo("PORT", getPort());
     tEvent.addInfo("RACK_NAME", getRackName());
@@ -239,5 +240,12 @@ public class MapAttemptFinishedEvent implements HistoryEvent {
         "" : getAttemptId().toString());
     return tEvent;
   }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> metrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getCounters(), finishTime);
+    return metrics;
+  }
   
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java
index 95a2e36..d90ff93 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java
@@ -17,11 +17,14 @@
  */
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the normalized map/reduce requirements.
@@ -82,4 +85,9 @@ public class NormalizedResourceEvent implements HistoryEvent {
     tEvent.addInfo("TASK_TYPE", getTaskType());
     return tEvent;
   }
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
index 6087c7a..5ddd839 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -29,6 +31,7 @@ import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record successful completion of a reduce attempt
@@ -238,12 +241,17 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent {
     tEvent.addInfo("STATE", getState());
     tEvent.addInfo("SHUFFLE_FINISH_TIME", getShuffleFinishTime());
     tEvent.addInfo("SORT_FINISH_TIME", getSortFinishTime());
-    tEvent.addInfo("COUNTERS_GROUPS",
-        JobHistoryEventUtils.countersToJSON(getCounters()));
     tEvent.addInfo("HOSTNAME", getHostname());
     tEvent.addInfo("PORT", getPort());
     tEvent.addInfo("RACK_NAME", getRackName());
     return tEvent;
   }
 
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> metrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getCounters(), finishTime);
+    return metrics;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
index c7c4387..30d2b09 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -28,6 +30,7 @@ import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record successful task completion
@@ -149,10 +152,14 @@ public class TaskAttemptFinishedEvent  implements HistoryEvent {
     tEvent.addInfo("FINISH_TIME", getFinishTime());
     tEvent.addInfo("STATUS", getTaskStatus());
     tEvent.addInfo("STATE", getState());
-    tEvent.addInfo("COUNTERS_GROUPS",
-        JobHistoryEventUtils.countersToJSON(getCounters()));
     tEvent.addInfo("HOSTNAME", getHostname());
     return tEvent;
   }
 
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> metrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getCounters(), finishTime);
+    return metrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
index 61c5178..99213fd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
@@ -25,10 +28,9 @@ import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 
 /**
  * Event to record start of a task attempt
@@ -151,4 +153,9 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
     return tEvent;
   }
 
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
index 0bb1358..8826c96 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
@@ -18,8 +18,12 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapred.ProgressSplitsBlock;
 import org.apache.hadoop.mapred.TaskStatus;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
@@ -28,10 +32,7 @@ import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
-
-import org.apache.hadoop.mapred.ProgressSplitsBlock;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record unsuccessful (Killed/Failed) completion of task attempts
@@ -267,9 +268,13 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
     tEvent.addInfo("SHUFFLE_FINISH_TIME", getFinishTime());
     tEvent.addInfo("SORT_FINISH_TIME", getFinishTime());
     tEvent.addInfo("MAP_FINISH_TIME", getFinishTime());
-    tEvent.addInfo("COUNTERS_GROUPS",
-        JobHistoryEventUtils.countersToJSON(getCounters()));
     return tEvent;
   }
 
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> metrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getCounters(), finishTime);
+    return metrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
index 5e82dea..9a29b81 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapred.TaskStatus;
@@ -28,8 +31,7 @@ import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the failure of a task
@@ -151,9 +153,13 @@ public class TaskFailedEvent implements HistoryEvent {
     tEvent.addInfo("ERROR", getError());
     tEvent.addInfo("FAILED_ATTEMPT_ID",
         getFailedAttemptID() == null ? "" : getFailedAttemptID().toString());
-    tEvent.addInfo("COUNTERS_GROUPS", 
-        JobHistoryEventUtils.countersToJSON(getCounters()));
     return tEvent;
   }
 
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> metrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getCounters(), finishTime);
+    return metrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
index e359e32..fd003cc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -29,6 +31,7 @@ import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.util.JobHistoryEventUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the successful completion of a task
@@ -124,8 +127,6 @@ public class TaskFinishedEvent implements HistoryEvent {
     TimelineEvent tEvent = new TimelineEvent();
     tEvent.setId(StringUtils.toUpperCase(getEventType().name()));
     tEvent.addInfo("TASK_TYPE", getTaskType().toString());
-    tEvent.addInfo("COUNTERS_GROUPS",
-        JobHistoryEventUtils.countersToJSON(getCounters()));
     tEvent.addInfo("FINISH_TIME", getFinishTime());
     tEvent.addInfo("STATUS", TaskStatus.State.SUCCEEDED.toString());
     tEvent.addInfo("SUCCESSFUL_TASK_ATTEMPT_ID",
@@ -133,5 +134,11 @@ public class TaskFinishedEvent implements HistoryEvent {
             getSuccessfulTaskAttemptId().toString());
     return tEvent;
   }
-  
+
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    Set<TimelineMetric> jobMetrics = JobHistoryEventUtils
+        .countersToTimelineMetric(getCounters(), finishTime);
+    return jobMetrics;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
index d1b97bf..6792ce7 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
+import java.util.Set;
+
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -25,6 +27,7 @@ import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record the start of a task
@@ -84,4 +87,9 @@ public class TaskStartedEvent implements HistoryEvent {
     return tEvent;
   }
 
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java
index b9a389c..0722207 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java
@@ -18,15 +18,15 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
-import java.io.IOException;
+import java.util.Set;
 
+import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
-
-import org.apache.avro.util.Utf8;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 
 /**
  * Event to record updates to a task
@@ -71,4 +71,9 @@ public class TaskUpdatedEvent implements HistoryEvent {
     return tEvent;
   }
 
+  @Override
+  public Set<TimelineMetric> getTimelineMetrics() {
+    return null;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java
index e60380b..225d517 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java
@@ -17,10 +17,13 @@
 */
 package org.apache.hadoop.mapreduce.util;
 
+import java.util.HashSet;
+import java.util.Set;
+
 import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.CounterGroup;
 import org.apache.hadoop.mapreduce.Counters;
-
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.node.ArrayNode;
@@ -53,4 +56,20 @@ public final class JobHistoryEventUtils {
     return nodes;
   }
 
+  public static Set<TimelineMetric> countersToTimelineMetric(Counters counters,
+      long timestamp) {
+    Set<TimelineMetric> entityMetrics = new HashSet<TimelineMetric>();
+    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(timestamp, c.getValue());
+        entityMetrics.add(metric);
+      }
+    }
+    return entityMetrics;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/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 300b4fb..5915a43 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
@@ -18,7 +18,12 @@
 
 package org.apache.hadoop.mapred;
 
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileReader;
 import java.io.IOException;
 import java.util.EnumSet;
 import java.util.List;
@@ -38,11 +43,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.timeline.TimelineStore;
 import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
+import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.junit.Assert;
@@ -200,7 +207,7 @@ public class TestMRTimelineEventHandling {
       Assert.assertEquals(apps.size(), 1);
       ApplicationReport appReport = apps.get(0);
       firstAppId = appReport.getApplicationId();
-
+      UtilsForTests.waitForAppFinished(job, cluster);
       checkNewTimelineEvent(firstAppId, appReport);
 
       LOG.info("Run 2nd job which should be failed.");
@@ -213,6 +220,7 @@ public class TestMRTimelineEventHandling {
 
       appReport = apps.get(0).getApplicationId().equals(firstAppId) ?
           apps.get(0) : apps.get(1);
+
       checkNewTimelineEvent(firstAppId, appReport);
 
     } finally {
@@ -262,6 +270,27 @@ public class TestMRTimelineEventHandling {
     Assert.assertTrue("jobEventFilePath: " + jobEventFilePath +
         " does not exist.",
         jobEventFile.exists());
+    verifyMetricsWhenEvent(jobEventFile, EventType.JOB_FINISHED.name());
+
+    // for this test, we expect MR job metrics are published in YARN_APPLICATION
+    String outputAppDir = basePath + "/YARN_APPLICATION/";
+    entityFolder = new File(outputAppDir);
+    Assert.assertTrue(
+        "Job output directory: " + outputAppDir +
+        " does not exist.",
+        entityFolder.isDirectory());
+
+    // check for job event file
+    String appEventFileName = appId.toString()
+        + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
+
+    String appEventFilePath = outputAppDir + appEventFileName;
+    File appEventFile = new File(appEventFilePath);
+    Assert.assertTrue(
+        "appEventFilePath: " + appEventFilePath +
+        " does not exist.",
+        appEventFile.exists());
+    verifyMetricsWhenEvent(appEventFile, null);
 
     // check for task event file
     String outputDirTask = basePath + "/MAPREDUCE_TASK/";
@@ -278,6 +307,7 @@ public class TestMRTimelineEventHandling {
     Assert.assertTrue("taskEventFileName: " + taskEventFilePath +
         " does not exist.",
         taskEventFile.exists());
+    verifyMetricsWhenEvent(taskEventFile, EventType.TASK_FINISHED.name());
     
     // check for task attempt event file
     String outputDirTaskAttempt = basePath + "/MAPREDUCE_TASK_ATTEMPT/";
@@ -294,6 +324,48 @@ public class TestMRTimelineEventHandling {
     File taskAttemptEventFile = new File(taskAttemptEventFilePath);
     Assert.assertTrue("taskAttemptEventFileName: " + taskAttemptEventFilePath +
         " does not exist.", taskAttemptEventFile.exists());
+    verifyMetricsWhenEvent(taskAttemptEventFile,
+        EventType.MAP_ATTEMPT_FINISHED.name());
+  }
+
+  private void verifyMetricsWhenEvent(File entityFile, String eventId)
+      throws IOException {
+    BufferedReader reader = null;
+    String strLine;
+    try {
+      reader = new BufferedReader(new FileReader(entityFile));
+      boolean jobMetricsFoundForAppEntity = false;
+      while ((strLine = reader.readLine()) != null) {
+        if (strLine.trim().length() > 0) {
+          org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
+              FileSystemTimelineReaderImpl.getTimelineRecordFromJSON(
+                  strLine.trim(),
+                  org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity.class);
+          if (eventId == null) {
+            // Job metrics are published without any events for
+            // ApplicationEntity. There is also possibility that some other
+            // ApplicationEntity is published without events, hence loop all
+            if (entity.getEvents().size() == 0) {
+              jobMetricsFoundForAppEntity = entity.getMetrics().size() > 0;
+              if (jobMetricsFoundForAppEntity) {
+                return;
+              }
+            }
+          } else {
+            for (TimelineEvent event : entity.getEvents()) {
+              if (event.getId().equals(eventId)) {
+                assertTrue(entity.getMetrics().size() > 0);
+                return;
+              }
+            }
+          }
+        }
+      }
+      fail("Expected event : " + eventId + " not found in the file "
+          + entityFile);
+    } finally {
+      reader.close();
+    }
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5f5c98e5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
index 972391c..cb494db 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
@@ -18,9 +18,9 @@
 
 package org.apache.hadoop.mapred;
 
+import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.DataOutputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -31,8 +31,10 @@ import java.util.Enumeration;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -42,20 +44,24 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.SequenceFile.CompressionType;
 import org.apache.hadoop.mapred.SortValidator.RecordStatsChecker.NonSplitableSequenceFileInputFormat;
 import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityReducer;
 import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 
-import org.apache.commons.logging.Log;
+import com.google.common.base.Supplier;
 
 /** 
  * Utilities used in unit test.
@@ -607,6 +613,29 @@ public class UtilsForTests {
     return job;
   }
 
+  public static void waitForAppFinished(RunningJob job, MiniMRYarnCluster cluster)
+ throws IOException {
+    ApplicationId appId = ApplicationId.newInstance(
+        Long.parseLong(job.getID().getJtIdentifier()), job.getID().getId());
+    ConcurrentMap<ApplicationId, RMApp> rmApps =
+        cluster.getResourceManager().getRMContext().getRMApps();
+    if (!rmApps.containsKey(appId)) {
+      throw new IOException("Job not found");
+    }
+    final RMApp rmApp = rmApps.get(appId);
+    try {
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          return RMAppImpl.isAppInFinalState(rmApp);
+        }
+      }, 1000, 1000 * 180);
+    } catch (TimeoutException | InterruptedException e1) {
+      throw new IOException("Yarn application with " + appId + " didn't finish "
+          + "did not reach finale State", e1);
+    }
+  }
+
   // Run a job that will be succeeded and wait until it completes
   public static RunningJob runJobSucceed(JobConf conf, Path inDir, Path outDir)
          throws IOException {


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


[34/50] [abbrv] hadoop git commit: YARN-5015. entire time series is returned for YARN container system metrics (CPU and memory) (Varun Saxena via sjlee)

Posted by vr...@apache.org.
YARN-5015. entire time series is returned for YARN container system metrics (CPU and memory) (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/fc78a937
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fc78a937
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fc78a937

Branch: refs/heads/YARN-2928
Commit: fc78a937d80ef01712382c563a7a0c3e53da6a24
Parents: c998a52
Author: Sangjin Lee <sj...@apache.org>
Authored: Tue May 31 13:09:59 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:12 2016 -0700

----------------------------------------------------------------------
 ...stTimelineReaderWebServicesHBaseStorage.java | 211 +++++++++++--
 .../storage/TestHBaseTimelineStorage.java       | 315 ++++++++++++++-----
 .../storage/flow/TestHBaseStorageFlowRun.java   |  26 +-
 .../reader/TimelineDataToRetrieve.java          |  32 +-
 .../reader/TimelineEntityFilters.java           |   5 +-
 .../reader/TimelineReaderWebServices.java       | 274 ++++++++++++----
 .../reader/TimelineReaderWebServicesUtils.java  |  17 +-
 .../storage/reader/ApplicationEntityReader.java |   3 +-
 .../storage/reader/FlowRunEntityReader.java     |  12 +
 .../storage/reader/GenericEntityReader.java     |   4 +-
 .../storage/reader/TimelineEntityReader.java    |   5 +-
 .../TestFileSystemTimelineReaderImpl.java       |  12 +-
 12 files changed, 711 insertions(+), 205 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc78a937/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
index ca80ed5..f9f4607 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -21,6 +21,7 @@ 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 static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.lang.reflect.UndeclaredThrowableException;
@@ -112,13 +113,14 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     TimelineMetric m1 = new TimelineMetric();
     m1.setId("MAP_SLOT_MILLIS");
     Map<Long, Number> metricValues =
-        ImmutableMap.of(ts - 100000, (Number)2, ts - 80000, 40);
+        ImmutableMap.of(ts - 100000, (Number)2, ts - 90000, 7, ts - 80000, 40);
     m1.setType(Type.TIME_SERIES);
     m1.setValues(metricValues);
     metrics.add(m1);
     m1 = new TimelineMetric();
     m1.setId("MAP1_SLOT_MILLIS");
-    metricValues = ImmutableMap.of(ts - 100000, (Number)2, ts - 80000, 40);
+    metricValues =
+        ImmutableMap.of(ts - 100000, (Number)2, ts - 90000, 9, ts - 80000, 40);
     m1.setType(Type.TIME_SERIES);
     m1.setValues(metricValues);
     metrics.add(m1);
@@ -460,6 +462,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     assertNotNull(resp);
     assertTrue("Response from server should have been " + status,
         resp.getClientResponseStatus().equals(status));
+    System.out.println("Response is: " + resp.getEntity(String.class));
   }
 
   @Test
@@ -615,12 +618,18 @@ public class TestTimelineReaderWebServicesHBaseStorage {
             (entity.getStartTime() == 1425016501034L) &&
             (entity.getMetrics().size() == 1)));
       }
+
+      // fields as CONFIGS will lead to a HTTP 400 as it makes no sense for
+      // flow runs.
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
+          "fields=CONFIGS");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
     } finally {
       client.destroy();
     }
   }
 
-
   @Test
   public void testGetFlowRunsMetricsToRetrieve() throws Exception {
     Client client = createClient();
@@ -1024,15 +1033,12 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertNotNull(entity);
       assertEquals("application_1111111111_1111", entity.getId());
       assertEquals(3, 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);
-      TimelineMetric m3 = newMetric(TimelineMetric.Type.TIME_SERIES,
-          "MAP1_SLOT_MILLIS", ts - 100000, 2L);
-      m3.addValue(ts - 80000, 40L);
+      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, 40L);
+      TimelineMetric m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+          "MAP1_SLOT_MILLIS", ts - 80000, 40L);
       for (TimelineMetric metric : entity.getMetrics()) {
         assertTrue(verifyMetrics(metric, m1, m2, m3));
       }
@@ -1045,9 +1051,8 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertNotNull(entity);
       assertEquals("application_1111111111_2222", entity.getId());
       assertEquals(1, entity.getMetrics().size());
-      TimelineMetric m4 = newMetric(TimelineMetric.Type.TIME_SERIES,
-         "MAP_SLOT_MILLIS", ts - 100000, 5L);
-      m4.addValue(ts - 80000, 101L);
+      TimelineMetric m4 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+         "MAP_SLOT_MILLIS", ts - 80000, 101L);
       for (TimelineMetric metric : entity.getMetrics()) {
         assertTrue(verifyMetrics(metric, m4));
       }
@@ -1067,15 +1072,35 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       TimelineEntity entity = resp.getEntity(TimelineEntity.class);
       assertNotNull(entity);
       assertEquals("application_1111111111_1111", entity.getId());
+      assertEquals(1, entity.getConfigs().size());
+      assertEquals(3, 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, 40L);
+      TimelineMetric m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+          "MAP1_SLOT_MILLIS", ts - 80000, 40L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m1, m2, m3));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111?" +
+          "fields=ALL&metricslimit=10");
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("application_1111111111_1111", entity.getId());
+      assertEquals(1, entity.getConfigs().size());
       assertEquals(3, entity.getMetrics().size());
-      TimelineMetric m1 = newMetric(TimelineMetric.Type.TIME_SERIES,
-          "HDFS_BYTES_READ", ts - 100000, 31L);
+      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 = newMetric(TimelineMetric.Type.TIME_SERIES, "MAP_SLOT_MILLIS",
+          ts - 100000, 2L);
       m2.addValue(ts - 80000, 40L);
-      TimelineMetric m3 = newMetric(TimelineMetric.Type.TIME_SERIES,
-          "MAP1_SLOT_MILLIS", ts - 100000, 2L);
+      m3 = newMetric(TimelineMetric.Type.TIME_SERIES, "MAP1_SLOT_MILLIS",
+          ts - 100000, 2L);
       m3.addValue(ts - 80000, 40L);
       for (TimelineMetric metric : entity.getMetrics()) {
         assertTrue(verifyMetrics(metric, m1, m2, m3));
@@ -1229,11 +1254,6 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         }
       }
       assertEquals(2, metricCnt);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?metricstoretrieve=!(MAP1_,HDFS_");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
     } finally {
       client.destroy();
     }
@@ -1550,6 +1570,35 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         assertTrue(entity.getId().equals("entity2"));
         for (TimelineMetric metric : entity.getMetrics()) {
           assertTrue(metric.getId().startsWith("MAP1"));
+          assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
+        }
+      }
+      assertEquals(2, metricCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
+          "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
+          "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)&metricstoretrieve=" +
+          "!(HDFS)&metricslimit=10");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        assertTrue(entity.getId().equals("entity2"));
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP1"));
+          if (metric.getId().equals("MAP1_SLOT_MILLIS")) {
+            assertEquals(2, metric.getValues().size());
+            assertEquals(TimelineMetric.Type.TIME_SERIES, metric.getType());
+          } else if (metric.getId().equals("MAP11_SLOT_MILLIS")) {
+            assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
+          } else {
+            fail("Unexpected metric id");
+          }
         }
       }
       assertEquals(2, metricCnt);
@@ -1794,6 +1843,23 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertEquals(1, entity.getMetrics().size());
       for (TimelineMetric  metric : entity.getMetrics()) {
         assertTrue(metric.getId().startsWith("MAP11_"));
+        assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
+        assertEquals(1, metric.getValues().size());
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1/entity2?metricstoretrieve=!(MAP1_,HDFS_)&" +
+          "metricslimit=5");
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("entity2", entity.getId());
+      assertEquals("type1", entity.getType());
+      assertEquals(1, entity.getMetrics().size());
+      for (TimelineMetric  metric : entity.getMetrics()) {
+        assertTrue(metric.getId().startsWith("MAP11_"));
+        assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
       }
     } finally {
       client.destroy();
@@ -1818,6 +1884,29 @@ public class TestTimelineReaderWebServicesHBaseStorage {
             entity.getMetrics().size() == 3) ||
             (entity.getId().equals("application_1111111111_2222") &&
             entity.getMetrics().size() == 1));
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
+          assertEquals(1, metric.getValues().size());
+        }
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+              "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" +
+              "1002345678919/apps?fields=ALL&metricslimit=2");
+      resp = getResponse(client, uri);
+      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() == 3) ||
+            (entity.getId().equals("application_1111111111_2222") &&
+            entity.getMetrics().size() == 1));
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getValues().size() <= 2);
+          assertEquals(TimelineMetric.Type.TIME_SERIES, metric.getType());
+        }
       }
 
       // Query without specifying cluster ID.
@@ -1855,11 +1944,75 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       for (TimelineEntity entity : entities) {
         assertTrue("Unexpected app in result",
             (entity.getId().equals("application_1111111111_1111") &&
-            entity.getMetrics().size() == 3) ||
+            entity.getConfigs().size() == 1 &&
+            entity.getConfigs().equals(ImmutableMap.of("cfg2", "value1"))) ||
             (entity.getId().equals("application_1111111111_2222") &&
-            entity.getMetrics().size() == 1) ||
+            entity.getConfigs().size() == 1 &&
+            entity.getConfigs().equals(ImmutableMap.of("cfg1", "value1"))) ||
             (entity.getId().equals("application_1111111111_2224") &&
-            entity.getMetrics().size() == 1));
+            entity.getConfigs().size() == 0));
+        for (TimelineMetric metric : entity.getMetrics()) {
+          if (entity.getId().equals("application_1111111111_1111")) {
+            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, 40L);
+            TimelineMetric m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+                "MAP1_SLOT_MILLIS", ts - 80000, 40L);
+            assertTrue(verifyMetrics(metric, m1, m2, m3));
+          } else if (entity.getId().equals("application_1111111111_2222")) {
+            TimelineMetric m1 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+                "MAP_SLOT_MILLIS", ts - 80000, 101L);
+            assertTrue(verifyMetrics(metric, m1));
+          } else if (entity.getId().equals("application_1111111111_2224")) {
+            TimelineMetric m1 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+                "MAP_SLOT_MILLIS", ts - 80000, 101L);
+            assertTrue(verifyMetrics(metric, m1));
+          }
+        }
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" +
+          "fields=ALL&metricslimit=6");
+      resp = getResponse(client, uri);
+      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.getConfigs().size() == 1 &&
+            entity.getConfigs().equals(ImmutableMap.of("cfg2", "value1"))) ||
+            (entity.getId().equals("application_1111111111_2222") &&
+            entity.getConfigs().size() == 1 &&
+            entity.getConfigs().equals(ImmutableMap.of("cfg1", "value1"))) ||
+            (entity.getId().equals("application_1111111111_2224") &&
+            entity.getConfigs().size() == 0));
+        for (TimelineMetric metric : entity.getMetrics()) {
+          if (entity.getId().equals("application_1111111111_1111")) {
+            TimelineMetric m1 = newMetric(TimelineMetric.Type.TIME_SERIES,
+                "HDFS_BYTES_READ", ts - 80000, 57L);
+            m1.addValue(ts - 100000, 31L);
+            TimelineMetric m2 = newMetric(TimelineMetric.Type.TIME_SERIES,
+                "MAP_SLOT_MILLIS", ts - 80000, 40L);
+            m2.addValue(ts - 100000, 2L);
+            TimelineMetric m3 = newMetric(TimelineMetric.Type.TIME_SERIES,
+                "MAP1_SLOT_MILLIS", ts - 80000, 40L);
+            m3.addValue(ts - 100000, 2L);
+            assertTrue(verifyMetrics(metric, m1, m2, m3));
+          } else if (entity.getId().equals("application_1111111111_2222")) {
+            TimelineMetric m1 = newMetric(TimelineMetric.Type.TIME_SERIES,
+                "MAP_SLOT_MILLIS", ts - 80000, 101L);
+            m1.addValue(ts - 100000, 5L);
+            assertTrue(verifyMetrics(metric, m1));
+          } else if (entity.getId().equals("application_1111111111_2224")) {
+            TimelineMetric m1 = newMetric(TimelineMetric.Type.TIME_SERIES,
+                "MAP_SLOT_MILLIS", ts - 80000, 101L);
+            m1.addValue(ts - 100000, 5L);
+            assertTrue(verifyMetrics(metric, m1));
+          }
+        }
       }
 
       // Query without specifying cluster ID.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc78a937/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/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-hbase-tests/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-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index bcf2d2c..c002ca0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -612,7 +612,7 @@ public class TestHBaseTimelineStorage {
     aggMetric.setId("MEM_USAGE");
     Map<Long, Number> aggMetricValues = new HashMap<Long, Number>();
     ts = System.currentTimeMillis();
-    aggMetricValues.put(ts - 120000, 102400000);
+    aggMetricValues.put(ts - 120000, 102400000L);
     aggMetric.setType(Type.SINGLE_VALUE);
     aggMetric.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
     aggMetric.setValues(aggMetricValues);
@@ -721,12 +721,14 @@ public class TestHBaseTimelineStorage {
       NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
       matchMetrics(metricValues, metricMap);
 
-      // read the timeline entity using the reader this time
+      // read the timeline entity using the reader this time. In metrics limit
+      // specify Integer MAX_VALUE. A TIME_SERIES will be returned(if more than
+      // one value exists for a metric).
       TimelineEntity e1 = reader.getEntity(
           new TimelineReaderContext(cluster, user, flow, runid, appId,
           entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(
-          null, null, EnumSet.of(TimelineReader.Field.ALL)));
+          new TimelineDataToRetrieve(null, null,
+          EnumSet.of(TimelineReader.Field.ALL), Integer.MAX_VALUE));
       assertNotNull(e1);
 
       // verify attributes
@@ -753,12 +755,69 @@ public class TestHBaseTimelineStorage {
         assertTrue(metric2.getId().equals("MAP_SLOT_MILLIS") ||
             metric2.getId().equals("MEM_USAGE"));
         if (metric2.getId().equals("MAP_SLOT_MILLIS")) {
+          assertEquals(6, metricValues2.size());
           matchMetrics(metricValues, metricValues2);
         }
         if (metric2.getId().equals("MEM_USAGE")) {
+          assertEquals(1, metricValues2.size());
           matchMetrics(aggMetricValues, metricValues2);
         }
       }
+
+      // In metrics limit specify a value of 3. No more than 3 values for a
+      // metric will be returned.
+      e1 = reader.getEntity(new TimelineReaderContext(cluster, user, flow,
+          runid, appId, entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null,
+          EnumSet.of(TimelineReader.Field.ALL), 3));
+      assertNotNull(e1);
+      assertEquals(appId, e1.getId());
+      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
+          e1.getType());
+      assertEquals(conf, e1.getConfigs());
+      metrics2 = e1.getMetrics();
+      assertEquals(2, metrics2.size());
+      for (TimelineMetric metric2 : metrics2) {
+        Map<Long, Number> metricValues2 = metric2.getValues();
+        assertTrue(metricValues2.size() <= 3);
+        assertTrue(metric2.getId().equals("MAP_SLOT_MILLIS") ||
+            metric2.getId().equals("MEM_USAGE"));
+      }
+
+      // Check if single value(latest value) instead of time series is returned
+      // if metricslimit is not set(null), irrespective of number of metric
+      // values.
+      e1 = reader.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, appId,
+         entity.getType(), entity.getId()), new TimelineDataToRetrieve(
+         null, null, EnumSet.of(TimelineReader.Field.ALL), null));
+      assertNotNull(e1);
+      assertEquals(appId, e1.getId());
+      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
+          e1.getType());
+      assertEquals(cTime, e1.getCreatedTime());
+      assertEquals(infoMap, e1.getInfo());
+      assertEquals(isRelatedTo, e1.getIsRelatedToEntities());
+      assertEquals(relatesTo, e1.getRelatesToEntities());
+      assertEquals(conf, e1.getConfigs());
+      assertEquals(2, e1.getMetrics().size());
+      for (TimelineMetric metric : e1.getMetrics()) {
+        assertEquals(1, metric.getValues().size());
+        assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
+        assertTrue(metric.getId().equals("MAP_SLOT_MILLIS") ||
+            metric.getId().equals("MEM_USAGE"));
+        assertEquals(1, metric.getValues().size());
+        if (metric.getId().equals("MAP_SLOT_MILLIS")) {
+          assertTrue(metric.getValues().containsKey(ts - 20000));
+          assertEquals(metricValues.get(ts - 20000),
+              metric.getValues().get(ts - 20000));
+        }
+        if (metric.getId().equals("MEM_USAGE")) {
+          assertTrue(metric.getValues().containsKey(ts - 120000));
+          assertEquals(aggMetricValues.get(ts - 120000),
+              metric.getValues().get(ts - 120000));
+        }
+      }
     } finally {
       if (hbi != null) {
         hbi.stop();
@@ -839,8 +898,8 @@ public class TestHBaseTimelineStorage {
       String flow = "some_flow_name";
       String flowVersion = "AB7822C10F1111";
       long runid = 1002345678919L;
-      String appName =
-          ApplicationId.newInstance(System.currentTimeMillis(), 1).toString();
+      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
+          9000000L, 1).toString();
       hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
       hbi.stop();
 
@@ -931,12 +990,14 @@ public class TestHBaseTimelineStorage {
       TimelineEntity e1 = reader.getEntity(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL),
+          Integer.MAX_VALUE));
       Set<TimelineEntity> es1 = reader.getEntities(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), null),
           new TimelineEntityFilters(),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL),
+          Integer.MAX_VALUE));
       assertNotNull(e1);
       assertEquals(1, es1.size());
 
@@ -962,6 +1023,25 @@ public class TestHBaseTimelineStorage {
         Map<Long, Number> metricValues2 = metric2.getValues();
         matchMetrics(metricValues, metricValues2);
       }
+
+      e1 = reader.getEntity(new TimelineReaderContext(cluster, user, flow,
+          runid, appName, entity.getType(), entity.getId()),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
+      assertNotNull(e1);
+      assertEquals(id, e1.getId());
+      assertEquals(type, e1.getType());
+      assertEquals(cTime, e1.getCreatedTime());
+      assertEquals(infoMap, e1.getInfo());
+      assertEquals(isRelatedTo, e1.getIsRelatedToEntities());
+      assertEquals(relatesTo, e1.getRelatesToEntities());
+      assertEquals(conf, e1.getConfigs());
+      for (TimelineMetric metric : e1.getMetrics()) {
+        assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
+        assertEquals(1, metric.getValues().size());
+        assertTrue(metric.getValues().containsKey(ts - 20000));
+        assertEquals(metricValues.get(ts - 20000),
+            metric.getValues().get(ts - 20000));
+      }
     } finally {
       if (hbi != null) {
         hbi.stop();
@@ -1067,11 +1147,11 @@ public class TestHBaseTimelineStorage {
       TimelineEntity e1 = reader.getEntity(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
       TimelineEntity e2 = reader.getEntity(
           new TimelineReaderContext(cluster, user, null, null, appName,
           entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
       assertNotNull(e1);
       assertNotNull(e2);
       assertEquals(e1, e2);
@@ -1125,8 +1205,8 @@ public class TestHBaseTimelineStorage {
       String flow = "other_flow_name";
       String flowVersion = "1111F01C2287BA";
       long runid = 1009876543218L;
-      String appName =
-          ApplicationId.newInstance(System.currentTimeMillis(), 1).toString();
+      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
+          9000000L, 1).toString();
       byte[] startRow =
           EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
       hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
@@ -1173,12 +1253,12 @@ public class TestHBaseTimelineStorage {
       TimelineEntity e1 = reader.getEntity(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
       Set<TimelineEntity> es1 = reader.getEntities(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), null),
           new TimelineEntityFilters(),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
       assertNotNull(e1);
       assertEquals(1, es1.size());
 
@@ -1235,7 +1315,7 @@ public class TestHBaseTimelineStorage {
       TimelineEntity e1 = reader.getEntity(
           new TimelineReaderContext(cluster, user, flow, runid, appName,
           entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
       assertNotNull(e1);
       // check the events
       NavigableSet<TimelineEvent> events = e1.getEvents();
@@ -1325,7 +1405,7 @@ public class TestHBaseTimelineStorage {
     TimelineEntity entity = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111","world", "hello"),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertNotNull(entity);
     assertEquals(3, entity.getConfigs().size());
     assertEquals(1, entity.getIsRelatedToEntities().size());
@@ -1333,7 +1413,7 @@ public class TestHBaseTimelineStorage {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111","world",
         null), new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(3, entities.size());
     int cfgCnt = 0;
     int metricCnt = 0;
@@ -1457,7 +1537,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         null, ef),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(1, entities.size());
     int eventCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -1583,7 +1663,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, irt, null, null, null,
         null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     int isRelatedToCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -1732,7 +1812,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, rt, null, null, null, null,
         null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     int relatesToCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -1932,7 +2012,7 @@ public class TestHBaseTimelineStorage {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111","world", "hello"),
         new TimelineDataToRetrieve(
-        null, null, EnumSet.of(Field.INFO, Field.CONFIGS)));
+        null, null, EnumSet.of(Field.INFO, Field.CONFIGS), null));
     assertNotNull(e1);
     assertEquals(3, e1.getConfigs().size());
     assertEquals(0, e1.getIsRelatedToEntities().size());
@@ -1941,7 +2021,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(),
         new TimelineDataToRetrieve(
-        null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS)));
+        null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS), null));
     assertEquals(3, es1.size());
     int metricsCnt = 0;
     int isRelatedToCnt = 0;
@@ -1964,14 +2044,14 @@ public class TestHBaseTimelineStorage {
     TimelineEntity e1 = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111","world", "hello"),
-        new TimelineDataToRetrieve(list, null, null));
+        new TimelineDataToRetrieve(list, null, null, null));
     assertNotNull(e1);
     assertEquals(1, e1.getConfigs().size());
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(list, null, null));
+        new TimelineDataToRetrieve(list, null, null, null));
     int cfgCnt = 0;
     for (TimelineEntity entity : es1) {
       cfgCnt += entity.getConfigs().size();
@@ -2002,7 +2082,8 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
     assertEquals(2, entities.size());
     int cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -2015,7 +2096,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -2031,7 +2112,8 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList1, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
     assertEquals(1, entities.size());
     cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -2049,7 +2131,8 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList2, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
     assertEquals(0, entities.size());
 
     TimelineFilterList confFilterList3 = new TimelineFilterList(
@@ -2060,7 +2143,8 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList3, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
     assertEquals(0, entities.size());
 
     TimelineFilterList confFilterList4 = new TimelineFilterList(
@@ -2071,7 +2155,8 @@ public class TestHBaseTimelineStorage {
             1002345678919L, "application_1231111111_1111","world", null),
             new TimelineEntityFilters(null, null, null, null, null, null,
             confFilterList4, null, null),
-            new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+            new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+            null));
     assertEquals(0, entities.size());
 
     TimelineFilterList confFilterList5 = new TimelineFilterList(
@@ -2082,7 +2167,8 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList5, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
     assertEquals(3, entities.size());
   }
 
@@ -2099,7 +2185,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList, null, null),
-        new TimelineDataToRetrieve(list, null, null));
+        new TimelineDataToRetrieve(list, null, null, null));
     assertEquals(1, entities.size());
     int cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -2130,7 +2216,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList1, null, null),
-        new TimelineDataToRetrieve(confsToRetrieve, null, null));
+        new TimelineDataToRetrieve(confsToRetrieve, null, null, null));
     assertEquals(2, entities.size());
     cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -2151,14 +2237,14 @@ public class TestHBaseTimelineStorage {
     TimelineEntity e1 = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111","world", "hello"),
-        new TimelineDataToRetrieve(null, list, null));
+        new TimelineDataToRetrieve(null, list, null, null));
     assertNotNull(e1);
     assertEquals(1, e1.getMetrics().size());
     Set<TimelineEntity> es1 = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(null, list, null));
+        new TimelineDataToRetrieve(null, list, null, null));
     int metricCnt = 0;
     for (TimelineEntity entity : es1) {
       metricCnt += entity.getMetrics().size();
@@ -2187,7 +2273,8 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
     assertEquals(2, entities.size());
     int metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -2200,7 +2287,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -2218,7 +2305,8 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList1, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
     assertEquals(1, entities.size());
     metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -2236,7 +2324,8 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList2, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
     assertEquals(0, entities.size());
 
     TimelineFilterList metricFilterList3 = new TimelineFilterList(
@@ -2247,7 +2336,8 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList3, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
     assertEquals(0, entities.size());
 
     TimelineFilterList metricFilterList4 = new TimelineFilterList(
@@ -2258,7 +2348,8 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList4, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
     assertEquals(0, entities.size());
 
     TimelineFilterList metricFilterList5 = new TimelineFilterList(
@@ -2269,7 +2360,8 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList5, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
     assertEquals(3, entities.size());
   }
 
@@ -2286,7 +2378,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList, null),
-        new TimelineDataToRetrieve(null, list, null));
+        new TimelineDataToRetrieve(null, list, null, null));
     assertEquals(1, entities.size());
     int metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -2316,17 +2408,38 @@ public class TestHBaseTimelineStorage {
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList1, null),
         new TimelineDataToRetrieve(
-        null, metricsToRetrieve, EnumSet.of(Field.METRICS)));
+        null, metricsToRetrieve, EnumSet.of(Field.METRICS), null));
+    assertEquals(2, entities.size());
+    metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertEquals(TimelineMetric.Type.SINGLE_VALUE, metric.getType());
+        assertEquals(1, metric.getValues().size());
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
+    }
+    assertEquals(2, metricCnt);
+
+    entities = reader.getEntities(new TimelineReaderContext("cluster1", "user1",
+        "some_flow_name", 1002345678919L, "application_1231111111_1111","world",
+        null), new TimelineEntityFilters(null, null, null, null, null, null,
+        null, metricFilterList1, null), new TimelineDataToRetrieve(null,
+        metricsToRetrieve, EnumSet.of(Field.METRICS), Integer.MAX_VALUE));
     assertEquals(2, entities.size());
     metricCnt = 0;
+    int metricValCnt = 0;
     for (TimelineEntity entity : entities) {
       metricCnt += entity.getMetrics().size();
       for (TimelineMetric metric : entity.getMetrics()) {
+        metricValCnt += metric.getValues().size();
         assertTrue("Metric Id returned should start with MAP1_",
             metric.getId().startsWith("MAP1_"));
       }
     }
     assertEquals(2, metricCnt);
+    assertEquals(7, metricValCnt);
   }
 
   @Test
@@ -2348,7 +2461,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList,
         null, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(2, entities.size());
     int infoCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -2364,7 +2477,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList1,
         null, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(1, entities.size());
     infoCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -2382,7 +2495,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList2,
         null, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(0, entities.size());
 
     TimelineFilterList infoFilterList3 = new TimelineFilterList(
@@ -2393,7 +2506,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList3,
         null, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(0, entities.size());
 
     TimelineFilterList infoFilterList4 = new TimelineFilterList(
@@ -2404,7 +2517,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList4,
         null, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(0, entities.size());
 
     TimelineFilterList infoFilterList5 = new TimelineFilterList(
@@ -2415,7 +2528,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList5,
         null, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(3, entities.size());
   }
 
@@ -2425,7 +2538,7 @@ public class TestHBaseTimelineStorage {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1111111111_2222",
         TimelineEntityType.YARN_APPLICATION.toString(), null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertNotNull(entity);
     assertEquals(3, entity.getConfigs().size());
     assertEquals(1, entity.getIsRelatedToEntities().size());
@@ -2434,7 +2547,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
         new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(3, entities.size());
     int cfgCnt = 0;
     int metricCnt = 0;
@@ -2546,7 +2659,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, "application_1111111111_2222",
         TimelineEntityType.YARN_APPLICATION.toString(), null),
         new TimelineDataToRetrieve(
-        null, null, EnumSet.of(Field.INFO, Field.CONFIGS)));
+        null, null, EnumSet.of(Field.INFO, Field.CONFIGS), null));
     assertNotNull(e1);
     assertEquals(3, e1.getConfigs().size());
     assertEquals(0, e1.getIsRelatedToEntities().size());
@@ -2556,7 +2669,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(),
         new TimelineDataToRetrieve(
-        null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS)));
+        null, null, EnumSet.of(Field.IS_RELATED_TO, Field.METRICS), null));
     assertEquals(3, es1.size());
     int metricsCnt = 0;
     int isRelatedToCnt = 0;
@@ -2586,7 +2699,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, irt, null, null, null,
         null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     int isRelatedToCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -2745,7 +2858,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, rt, null, null, null, null,
         null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     int relatesToCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -2989,7 +3102,8 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
     assertEquals(2, entities.size());
     int cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -3003,7 +3117,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -3020,7 +3134,8 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList1, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
     assertEquals(1, entities.size());
     cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -3039,7 +3154,8 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList2, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
     assertEquals(0, entities.size());
 
     TimelineFilterList confFilterList3 = new TimelineFilterList(
@@ -3051,7 +3167,8 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList3, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
     assertEquals(0, entities.size());
 
     TimelineFilterList confFilterList4 = new TimelineFilterList(
@@ -3063,7 +3180,8 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList4, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
     assertEquals(0, entities.size());
 
     TimelineFilterList confFilterList5 = new TimelineFilterList(
@@ -3075,7 +3193,8 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList5, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS),
+        null));
     assertEquals(3, entities.size());
   }
 
@@ -3092,7 +3211,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         null, ef),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(1, entities.size());
     int eventCnt = 0;
     for (TimelineEntity timelineEntity : entities) {
@@ -3218,7 +3337,7 @@ public class TestHBaseTimelineStorage {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1111111111_2222",
         TimelineEntityType.YARN_APPLICATION.toString(), null),
-        new TimelineDataToRetrieve(list, null, null));
+        new TimelineDataToRetrieve(list, null, null, null));
     assertNotNull(e1);
     assertEquals(1, e1.getConfigs().size());
     Set<TimelineEntity> es1 = reader.getEntities(
@@ -3226,7 +3345,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null) ,
         new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(list, null, null));
+        new TimelineDataToRetrieve(list, null, null, null));
     int cfgCnt = 0;
     for (TimelineEntity entity : es1) {
       cfgCnt += entity.getConfigs().size();
@@ -3252,7 +3371,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList, null, null),
-        new TimelineDataToRetrieve(list, null, null));
+        new TimelineDataToRetrieve(list, null, null, null));
     assertEquals(1, entities.size());
     int cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -3285,7 +3404,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null,
         confFilterList1, null, null),
-        new TimelineDataToRetrieve(confsToRetrieve, null, null));
+        new TimelineDataToRetrieve(confsToRetrieve, null, null, null));
     assertEquals(2, entities.size());
     cfgCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -3316,7 +3435,8 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
     assertEquals(2, entities.size());
     int metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -3330,7 +3450,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL), null));
     assertEquals(2, entities.size());
     metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -3349,7 +3469,8 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList1, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
     assertEquals(1, entities.size());
     metricCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -3368,7 +3489,8 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList2, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
     assertEquals(0, entities.size());
 
     TimelineFilterList metricFilterList3 = new TimelineFilterList(
@@ -3380,7 +3502,8 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList3, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
     assertEquals(0, entities.size());
 
     TimelineFilterList metricFilterList4 = new TimelineFilterList(
@@ -3392,7 +3515,8 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList4, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
     assertEquals(0, entities.size());
 
     TimelineFilterList metricFilterList5 = new TimelineFilterList(
@@ -3404,7 +3528,8 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList5, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS),
+        null));
     assertEquals(3, entities.size());
   }
 
@@ -3417,7 +3542,7 @@ public class TestHBaseTimelineStorage {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1111111111_2222",
         TimelineEntityType.YARN_APPLICATION.toString(), null),
-        new TimelineDataToRetrieve(null, list, null));
+        new TimelineDataToRetrieve(null, list, null, null));
     assertNotNull(e1);
     assertEquals(1, e1.getMetrics().size());
     Set<TimelineEntity> es1 = reader.getEntities(
@@ -3425,7 +3550,7 @@ public class TestHBaseTimelineStorage {
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
         new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(null, list, null));
+        new TimelineDataToRetrieve(null, list, null, null));
     int metricCnt = 0;
     for (TimelineEntity entity : es1) {
       metricCnt += entity.getMetrics().size();
@@ -3451,7 +3576,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList, null),
-        new TimelineDataToRetrieve(null, list, null));
+        new TimelineDataToRetrieve(null, list, null, null));
     int metricCnt = 0;
     assertEquals(1, entities.size());
     for (TimelineEntity entity : entities) {
@@ -3477,17 +3602,37 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
         metricFilterList1, null),
-        new TimelineDataToRetrieve(null, metricsToRetrieve, null));
+        new TimelineDataToRetrieve(null, metricsToRetrieve, null, null));
+    metricCnt = 0;
+    assertEquals(2, entities.size());
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
+    }
+    assertEquals(2, metricCnt);
+
+    entities = reader.getEntities(new TimelineReaderContext("cluster1", "user1",
+        "some_flow_name", 1002345678919L, null,
+        TimelineEntityType.YARN_APPLICATION.toString(), null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList1, null), new TimelineDataToRetrieve(null,
+        metricsToRetrieve, EnumSet.of(Field.METRICS), Integer.MAX_VALUE));
     metricCnt = 0;
+    int metricValCnt = 0;
     assertEquals(2, entities.size());
     for (TimelineEntity entity : entities) {
       metricCnt += entity.getMetrics().size();
       for (TimelineMetric metric : entity.getMetrics()) {
+        metricValCnt += metric.getValues().size();
         assertTrue("Metric Id returned should start with MAP1_",
             metric.getId().startsWith("MAP1_"));
       }
     }
     assertEquals(2, metricCnt);
+    assertEquals(7, metricValCnt);
   }
 
   @Test
@@ -3510,7 +3655,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList,
         null, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(2, entities.size());
     int infoCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -3527,7 +3672,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList1,
         null, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(1, entities.size());
     infoCnt = 0;
     for (TimelineEntity entity : entities) {
@@ -3546,7 +3691,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList2,
         null, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(0, entities.size());
 
     TimelineFilterList infoFilterList3 = new TimelineFilterList(
@@ -3558,7 +3703,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList3,
         null, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(0, entities.size());
 
     TimelineFilterList infoFilterList4 = new TimelineFilterList(
@@ -3570,7 +3715,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList4,
         null, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(0, entities.size());
 
     TimelineFilterList infoFilterList5 = new TimelineFilterList(
@@ -3582,7 +3727,7 @@ public class TestHBaseTimelineStorage {
         null),
         new TimelineEntityFilters(null, null, null, null, null, infoFilterList5,
         null, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO), null));
     assertEquals(3, entities.size());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc78a937/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
index 801d43c..d0f98a5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
@@ -422,7 +422,7 @@ public class TestHBaseStorageFlowRun {
       TimelineEntity entity = hbr.getEntity(
           new TimelineReaderContext(cluster, user, flow, 1002345678919L, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineDataToRetrieve(null, metricsToRetrieve, null));
+          new TimelineDataToRetrieve(null, metricsToRetrieve, null, null));
       assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
       Set<TimelineMetric> metrics = entity.getMetrics();
       assertEquals(1, metrics.size());
@@ -447,7 +447,7 @@ public class TestHBaseStorageFlowRun {
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
           new TimelineEntityFilters(),
-          new TimelineDataToRetrieve(null, metricsToRetrieve, null));
+          new TimelineDataToRetrieve(null, metricsToRetrieve, null, null));
       assertEquals(2, entities.size());
       int metricCnt = 0;
       for (TimelineEntity timelineEntity : entities) {
@@ -513,8 +513,8 @@ public class TestHBaseStorageFlowRun {
       entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, runid, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineEntityFilters(),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+          new TimelineEntityFilters(), new TimelineDataToRetrieve(null, null,
+          EnumSet.of(Field.METRICS), null));
       assertEquals(1, entities.size());
       for (TimelineEntity timelineEntity : entities) {
         Set<TimelineMetric> timelineMetrics = timelineEntity.getMetrics();
@@ -766,8 +766,8 @@ public class TestHBaseStorageFlowRun {
           new TimelineReaderContext(cluster, user, flow, null,
           null, TimelineEntityType.YARN_FLOW_RUN.toString(), null),
           new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList, null),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+          metricFilterList, null), new TimelineDataToRetrieve(null, null,
+          EnumSet.of(Field.METRICS), null));
       assertEquals(2, entities.size());
       int metricCnt = 0;
       for (TimelineEntity entity : entities) {
@@ -783,8 +783,8 @@ public class TestHBaseStorageFlowRun {
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
           new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList1, null),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+          metricFilterList1, null), new TimelineDataToRetrieve(null, null,
+          EnumSet.of(Field.METRICS), null));
       assertEquals(1, entities.size());
       metricCnt = 0;
       for (TimelineEntity entity : entities) {
@@ -799,8 +799,8 @@ public class TestHBaseStorageFlowRun {
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
           new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList2, null),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+          metricFilterList2, null), new TimelineDataToRetrieve(null, null,
+          EnumSet.of(Field.METRICS), null));
       assertEquals(0, entities.size());
 
       TimelineFilterList metricFilterList3 = new TimelineFilterList(
@@ -809,8 +809,8 @@ public class TestHBaseStorageFlowRun {
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
           new TimelineEntityFilters(null, null, null, null, null, null, null,
-          metricFilterList3, null),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+          metricFilterList3, null), new TimelineDataToRetrieve(null, null,
+          EnumSet.of(Field.METRICS), null));
       assertEquals(0, entities.size());
 
       TimelineFilterList list3 = new TimelineFilterList();
@@ -832,7 +832,7 @@ public class TestHBaseStorageFlowRun {
           new TimelineEntityFilters(null, null, null, null, null, null, null,
           metricFilterList4, null),
           new TimelineDataToRetrieve(null, metricsToRetrieve,
-          EnumSet.of(Field.ALL)));
+          EnumSet.of(Field.ALL), null));
       assertEquals(2, entities.size());
       metricCnt = 0;
       for (TimelineEntity entity : entities) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc78a937/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java
index 0cc83d7..325050a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineDataToRetrieve.java
@@ -53,6 +53,10 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Fiel
  * object to retrieve, see {@link Field}. If null, retrieves 3 fields,
  * namely entity id, entity type and entity created time. All fields will
  * be returned if {@link Field#ALL} is specified.</li>
+ * <li><b>metricsLimit</b> - If fieldsToRetrieve contains METRICS/ALL or
+ * metricsToRetrieve is specified, this limit defines an upper limit to the
+ * number of metrics to return. This parameter is ignored if METRICS are not to
+ * be fetched.</li>
  * </ul>
  */
 @Private
@@ -61,16 +65,28 @@ public class TimelineDataToRetrieve {
   private TimelineFilterList confsToRetrieve;
   private TimelineFilterList metricsToRetrieve;
   private EnumSet<Field> fieldsToRetrieve;
+  private Integer metricsLimit;
+
+  /**
+   * Default limit of number of metrics to return.
+   */
+  public static final Integer DEFAULT_METRICS_LIMIT = 1;
 
   public TimelineDataToRetrieve() {
-    this(null, null, null);
+    this(null, null, null, null);
   }
 
   public TimelineDataToRetrieve(TimelineFilterList confs,
-      TimelineFilterList metrics, EnumSet<Field> fields) {
+      TimelineFilterList metrics, EnumSet<Field> fields,
+      Integer limitForMetrics) {
     this.confsToRetrieve = confs;
     this.metricsToRetrieve = metrics;
     this.fieldsToRetrieve = fields;
+    if (limitForMetrics == null || limitForMetrics < 1) {
+      this.metricsLimit = DEFAULT_METRICS_LIMIT;
+    } else {
+      this.metricsLimit = limitForMetrics;
+    }
 
     if (this.fieldsToRetrieve == null) {
       this.fieldsToRetrieve = EnumSet.noneOf(Field.class);
@@ -116,4 +132,16 @@ public class TimelineDataToRetrieve {
       fieldsToRetrieve.add(Field.METRICS);
     }
   }
+
+  public Integer getMetricsLimit() {
+    return metricsLimit;
+  }
+
+  public void setMetricsLimit(Integer limit) {
+    if (limit == null || limit < 1) {
+      this.metricsLimit = DEFAULT_METRICS_LIMIT;
+    } else {
+      this.metricsLimit = limit;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fc78a937/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
index 4821d31..8f2b725 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
@@ -32,8 +32,9 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyVa
  * filters restrict the number of entities to return.<br>
  * Filters contain the following :<br>
  * <ul>
- * <li><b>limit</b> - A limit on the number of entities to return. If null
- * or {@literal <=0}, defaults to {@link #DEFAULT_LIMIT}.</li>
+ * <li><b>limit</b> - A limit on the number of entities to return. If null or
+ * {@literal < 0}, defaults to {@link #DEFAULT_LIMIT}. The maximum possible
+ * value for limit can be {@link Long#MAX_VALUE}.</li>
  * <li><b>createdTimeBegin</b> - Matched entities should not be created
  * before this timestamp. If null or {@literal <=0}, defaults to 0.</li>
  * <li><b>createdTimeEnd</b> - Matched entities should not be created after


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


[21/50] [abbrv] hadoop git commit: YARN-5045. hbase unit tests fail due to dependency issues. (Sangjin Lee via varunsaxena)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/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-hbase-tests/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-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixOfflineAggregationWriterImpl.java
new file mode 100644
index 0000000..58d5e61
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestPhoenixOfflineAggregationWriterImpl.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;
+
+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);
+
+    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, 0L, 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/ccdec4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java
new file mode 100644
index 0000000..3b8036d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.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.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;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Generates the data/entities for the FlowRun and FlowActivity Tables
+ */
+class TestFlowDataGenerator {
+
+  private static final String metric1 = "MAP_SLOT_MILLIS";
+  private static final String metric2 = "HDFS_BYTES_READ";
+  public static final long END_TS_INCR = 10000L;
+
+  static TimelineEntity getEntityMetricsApp1(long insertTs, Configuration c1) {
+    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 = insertTs;
+
+    for (int k=1; k< 100 ; k++) {
+    metricValues.put(ts - k*200000, 20L);
+    }
+    metricValues.put(ts - 80000, 40L);
+    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();
+    for (int k=1; k< 100 ; k++) {
+      metricValues.put(ts - k*100000, 31L);
+    }
+
+    metricValues.put(ts - 80000, 57L);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues);
+    metrics.add(m2);
+
+    entity.addMetrics(metrics);
+    return entity;
+  }
+
+
+  static TimelineEntity getEntityMetricsApp1Complete(long insertTs, Configuration c1) {
+    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 = insertTs;
+
+    metricValues.put(ts - 80000, 40L);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+
+    TimelineMetric m2 = new TimelineMetric();
+    m2.setId(metric2);
+    metricValues = new HashMap<Long, Number>();
+    ts = insertTs;
+    metricValues.put(ts - 80000, 57L);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues);
+    metrics.add(m2);
+
+    entity.addMetrics(metrics);
+
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+    event.setTimestamp(insertTs);
+    event.addInfo("done", "insertTs=" + insertTs);
+    entity.addEvent(event);
+    return entity;
+  }
+
+
+  static TimelineEntity getEntityMetricsApp1(long insertTs) {
+    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 = insertTs;
+    metricValues.put(ts - 100000, 2L);
+    metricValues.put(ts - 80000, 40L);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+
+    TimelineMetric m2 = new TimelineMetric();
+    m2.setId(metric2);
+    metricValues = new HashMap<Long, Number>();
+    ts = insertTs;
+    metricValues.put(ts - 100000, 31L);
+    metricValues.put(ts - 80000, 57L);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues);
+    metrics.add(m2);
+
+    entity.addMetrics(metrics);
+    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 getEntityMetricsApp2(long insertTs) {
+    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 = insertTs;
+    metricValues.put(ts - 100000, 5L);
+    metricValues.put(ts - 80000, 101L);
+    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 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 getEntity1() {
+    TimelineEntity entity = new TimelineEntity();
+    String id = "flowRunHello";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    long cTime = 1425026901000L;
+    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 - 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);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    entity.addMetrics(metrics);
+
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    event.setTimestamp(cTime);
+    String expKey = "foo_event";
+    Object expVal = "test";
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+
+    event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+    long expTs = cTime + 21600000;// start time + 6hrs
+    event.setTimestamp(expTs);
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+
+    return entity;
+  }
+
+  static TimelineEntity getAFullEntity(long ts, long endTs) {
+    TimelineEntity entity = new TimelineEntity();
+    String id = "flowRunFullEntity";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    entity.setCreatedTime(ts);
+    // add metrics
+    Set<TimelineMetric> metrics = new HashSet<>();
+    TimelineMetric m1 = new TimelineMetric();
+    m1.setId(metric1);
+    Map<Long, Number> metricValues = new HashMap<Long, Number>();
+    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);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    TimelineMetric m2 = new TimelineMetric();
+    m2.setId(metric2);
+    metricValues = new HashMap<Long, Number>();
+    metricValues.put(ts - 900000, 31L);
+    metricValues.put(ts - 30000, 57L);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues);
+    metrics.add(m2);
+    entity.addMetrics(metrics);
+
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    event.setTimestamp(ts);
+    String expKey = "foo_event";
+    Object expVal = "test";
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+
+    event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+    long expTs = ts + 21600000;// start time + 6hrs
+    event.setTimestamp(expTs);
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+
+    return entity;
+  }
+
+  static TimelineEntity getEntityGreaterStartTime(long startTs) {
+    TimelineEntity entity = new TimelineEntity();
+    entity.setCreatedTime(startTs);
+    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);
+    event.setTimestamp(startTs);
+    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(long startTs) {
+    TimelineEntity entity = new TimelineEntity();
+    String id = "flowRunHelloMInStartTime";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    entity.setCreatedTime(startTs);
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    event.setTimestamp(startTs);
+    entity.addEvent(event);
+    return entity;
+  }
+
+  static TimelineEntity getMinFlushEntity(long startTs) {
+    TimelineEntity entity = new TimelineEntity();
+    String id = "flowRunHelloFlushEntityMin";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    entity.setCreatedTime(startTs);
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    event.setTimestamp(startTs);
+    entity.addEvent(event);
+    return entity;
+  }
+
+  static TimelineEntity getMaxFlushEntity(long startTs) {
+    TimelineEntity entity = new TimelineEntity();
+    String id = "flowRunHelloFlushEntityMax";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    entity.setCreatedTime(startTs);
+
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+    event.setTimestamp(startTs + END_TS_INCR);
+    entity.addEvent(event);
+    return entity;
+  }
+
+  static TimelineEntity getFlowApp1(long appCreatedTime) {
+    TimelineEntity entity = new TimelineEntity();
+    String id = "flowActivity_test";
+    String type = TimelineEntityType.YARN_APPLICATION.toString();
+    entity.setId(id);
+    entity.setType(type);
+    entity.setCreatedTime(appCreatedTime);
+
+    TimelineEvent event = new TimelineEvent();
+    event.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    event.setTimestamp(appCreatedTime);
+    String expKey = "foo_event";
+    Object expVal = "test";
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+
+    return entity;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
new file mode 100644
index 0000000..6b23b6c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
@@ -0,0 +1,469 @@
+/**
+ * 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 static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.Map;
+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.TableName;
+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.FlowActivityEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.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.common.ColumnHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+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 minStartTs = 1424995200300L;
+    long greaterStartTs = 1424995200300L + 864000L;
+    long endTs = 1424995200300L + 86000000L;;
+    TimelineEntity entityMinStartTime = TestFlowDataGenerator
+        .getEntityMinStartTime(minStartTs);
+
+    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(greaterStartTs);
+      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, minStartTs, 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.getFlowName());
+    long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(minStartTs);
+    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(
+          new TimelineReaderContext(cluster, null, null, null, null,
+          TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
+          new TimelineEntityFilters(10L, null, null, null, null, null,
+          null, null, null),
+          new TimelineDataToRetrieve());
+      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();
+    }
+  }
+
+  /**
+   * 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();
+    long appCreatedTime = 1425016501000L;
+    TimelineEntity entityApp1 =
+        TestFlowDataGenerator.getFlowApp1(appCreatedTime);
+    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,
+        appCreatedTime);
+
+    // use the reader to verify the data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+
+      Set<TimelineEntity> entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
+          new TimelineEntityFilters(10L, null, null, null, null, null,
+          null, null, null),
+          new TimelineDataToRetrieve());
+      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, long appCreatedTime)
+          throws IOException {
+    Scan s = new Scan();
+    s.addFamily(FlowActivityColumnFamily.INFO.getBytes());
+    byte[] startRow =
+        FlowActivityRowKey.getRowKey(cluster, appCreatedTime, user, flow);
+    s.setStartRow(startRow);
+    String clusterStop = cluster + "1";
+    byte[] stopRow =
+        FlowActivityRowKey.getRowKey(clusterStop, appCreatedTime, 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.getFlowName());
+      long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(appCreatedTime);
+      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();
+    long appCreatedTime = 1425016501000L;
+    TimelineEntity entityApp1 =
+        TestFlowDataGenerator.getFlowApp1(appCreatedTime);
+    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, appCreatedTime);
+
+    // use the timeline reader to verify data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+
+      Set<TimelineEntity> entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, null, null, null, null,
+          TimelineEntityType.YARN_FLOW_ACTIVITY.toString(), null),
+          new TimelineEntityFilters(10L, null, null, null, null, null,
+          null, null, null),
+          new TimelineDataToRetrieve());
+      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 =
+            TimelineStorageUtils.getTopOfTheDayTimestamp(appCreatedTime);
+        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,
+      long appCreatedTime)
+      throws IOException {
+    Scan s = new Scan();
+    s.addFamily(FlowActivityColumnFamily.INFO.getBytes());
+    byte[] startRow =
+        FlowActivityRowKey.getRowKey(cluster, appCreatedTime, user, flow);
+    s.setStartRow(startRow);
+    String clusterStop = cluster + "1";
+    byte[] stopRow =
+        FlowActivityRowKey.getRowKey(clusterStop, appCreatedTime, 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.getFlowName());
+      long dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(appCreatedTime);
+      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/ccdec4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
new file mode 100644
index 0000000..801d43c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
@@ -0,0 +1,851 @@
+/**
+ * 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.assertFalse;
+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.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+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.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+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.TimelineMetric;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
+import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+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.TimelineFilterList.Operator;
+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.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
+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);
+  }
+
+  @Test
+  public void checkCoProcessorOff() throws IOException, InterruptedException {
+    Configuration hbaseConf = util.getConfiguration();
+    TableName table = TableName.valueOf(hbaseConf.get(
+        FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+    Connection conn = null;
+    conn = ConnectionFactory.createConnection(hbaseConf);
+    Admin admin = conn.getAdmin();
+    if (admin == null) {
+      throw new IOException("Can't check tables since admin is null");
+    }
+    if (admin.tableExists(table)) {
+      // check the regions.
+      // check in flow run table
+      util.waitUntilAllRegionsAssigned(table);
+      HRegionServer server = util.getRSForFirstRegionInTable(table);
+      List<HRegion> regions = server.getOnlineRegions(table);
+      for (HRegion region : regions) {
+        assertTrue(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
+            hbaseConf));
+      }
+    }
+
+    table = TableName.valueOf(hbaseConf.get(
+        FlowActivityTable.TABLE_NAME_CONF_NAME,
+        FlowActivityTable.DEFAULT_TABLE_NAME));
+    if (admin.tableExists(table)) {
+      // check the regions.
+      // check in flow activity table
+      util.waitUntilAllRegionsAssigned(table);
+      HRegionServer server = util.getRSForFirstRegionInTable(table);
+      List<HRegion> regions = server.getOnlineRegions(table);
+      for (HRegion region : regions) {
+        assertFalse(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
+            hbaseConf));
+      }
+    }
+
+    table = TableName.valueOf(hbaseConf.get(
+        EntityTable.TABLE_NAME_CONF_NAME,
+        EntityTable.DEFAULT_TABLE_NAME));
+    if (admin.tableExists(table)) {
+      // check the regions.
+      // check in entity run table
+      util.waitUntilAllRegionsAssigned(table);
+      HRegionServer server = util.getRSForFirstRegionInTable(table);
+      List<HRegion> regions = server.getOnlineRegions(table);
+      for (HRegion region : regions) {
+        assertFalse(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
+            hbaseConf));
+      }
+    }
+  }
+
+  /**
+   * 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 minStartTs = 1425026900000L;
+    long greaterStartTs = 30000000000000L;
+    long endTs = 1439750690000L;
+    TimelineEntity entityMinStartTime = TestFlowDataGenerator
+        .getEntityMinStartTime(minStartTs);
+
+    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(greaterStartTs);
+      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 = Bytes.toLong(values.get(
+        FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes()));
+    assertEquals(minStartTs, starttime);
+    assertEquals(endTs, Bytes.toLong(values
+        .get(FlowRunColumn.MAX_END_TIME.getColumnQualifierBytes())));
+
+    // 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(
+          new TimelineReaderContext(cluster, user, flow, runid, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineDataToRetrieve());
+      assertTrue(TimelineEntityType.YARN_FLOW_RUN.matches(entity.getType()));
+      FlowRunEntity flowRun = (FlowRunEntity)entity;
+      assertEquals(minStartTs, flowRun.getStartTime());
+      assertEquals(endTs, flowRun.getMaxEndTime());
+    } finally {
+      hbr.close();
+    }
+  }
+
+  /**
+   * 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(System.currentTimeMillis());
+    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(System.currentTimeMillis());
+      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();
+      TimelineEntity entity = hbr.getEntity(
+          new TimelineReaderContext(cluster, user, flow, runid, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineDataToRetrieve());
+      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(141L, value);
+          break;
+        case metric2:
+          assertEquals(57L, value);
+          break;
+        default:
+          fail("unrecognized metric: " + id);
+        }
+      }
+    } finally {
+      hbr.close();
+    }
+  }
+
+  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(141L, Bytes.toLong(values.get(q)));
+
+      // check metric2
+      assertEquals(3, values.size());
+      q = ColumnHelper.getColumnQualifier(
+          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric2);
+      assertTrue(values.containsKey(q));
+      assertEquals(57L, Bytes.toLong(values.get(q)));
+    }
+    assertEquals(1, rowCount);
+  }
+
+  @Test
+  public void testWriteFlowRunMetricsPrefix() throws Exception {
+    String cluster = "testWriteFlowRunMetricsPrefix_cluster1";
+    String user = "testWriteFlowRunMetricsPrefix_user1";
+    String flow = "testWriteFlowRunMetricsPrefix_flow_name";
+    String flowVersion = "CF7022C10F1354";
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entityApp1 = TestFlowDataGenerator
+        .getEntityMetricsApp1(System.currentTimeMillis());
+    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, 1002345678919L, appName, te);
+      // write another application with same metric to this flow
+      te = new TimelineEntities();
+      TimelineEntity entityApp2 = TestFlowDataGenerator
+          .getEntityMetricsApp2(System.currentTimeMillis());
+      te.addEntity(entityApp2);
+      appName = "application_11111111111111_2222";
+      hbi.write(cluster, user, flow, flowVersion, 1002345678918L, appName, te);
+      hbi.flush();
+    } finally {
+      hbi.close();
+    }
+
+    // use the timeline reader to verify data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+      TimelineFilterList metricsToRetrieve = new TimelineFilterList(
+          Operator.OR, new TimelinePrefixFilter(TimelineCompareOp.EQUAL,
+              metric1.substring(0, metric1.indexOf("_") + 1)));
+      TimelineEntity entity = hbr.getEntity(
+          new TimelineReaderContext(cluster, user, flow, 1002345678919L, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineDataToRetrieve(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(40L, value);
+          break;
+        default:
+          fail("unrecognized metric: " + id);
+        }
+      }
+
+      Set<TimelineEntity> entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(),
+          new TimelineDataToRetrieve(null, metricsToRetrieve, null));
+      assertEquals(2, entities.size());
+      int metricCnt = 0;
+      for (TimelineEntity timelineEntity : entities) {
+        metricCnt += timelineEntity.getMetrics().size();
+      }
+      assertEquals(2, metricCnt);
+    } finally {
+      hbr.close();
+    }
+  }
+
+  @Test
+  public void testWriteFlowRunsMetricFields() throws Exception {
+    String cluster = "testWriteFlowRunsMetricFields_cluster1";
+    String user = "testWriteFlowRunsMetricFields_user1";
+    String flow = "testWriteFlowRunsMetricFields_flow_name";
+    String flowVersion = "CF7022C10F1354";
+    long runid = 1002345678919L;
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entityApp1 = TestFlowDataGenerator
+        .getEntityMetricsApp1(System.currentTimeMillis());
+    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(System.currentTimeMillis());
+      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(
+          new TimelineReaderContext(cluster, user, flow, runid, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(),
+          new TimelineDataToRetrieve());
+      assertEquals(1, entities.size());
+      for (TimelineEntity timelineEntity : entities) {
+        assertEquals(0, timelineEntity.getMetrics().size());
+      }
+
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, runid, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+      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();
+    }
+  }
+
+  @Test
+  public void testWriteFlowRunFlush() throws Exception {
+    String cluster = "atestFlushFlowRun_cluster1";
+    String user = "atestFlushFlowRun__user1";
+    String flow = "atestFlushFlowRun_flow_name";
+    String flowVersion = "AF1021C19F1351";
+    long runid = 1449526652000L;
+
+    int start = 10;
+    int count = 20000;
+    int appIdSuffix = 1;
+    HBaseTimelineWriterImpl hbi = null;
+    long insertTs = 1449796654827L - count;
+    long minTS = insertTs + 1;
+    long startTs = insertTs;
+    Configuration c1 = util.getConfiguration();
+    TimelineEntities te1 = null;
+    TimelineEntity entityApp1 = null;
+    TimelineEntity entityApp2 = null;
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+
+      for (int i = start; i < count; i++) {
+        String appName = "application_1060350000000_" + appIdSuffix;
+        insertTs++;
+        te1 = new TimelineEntities();
+        entityApp1 = TestFlowDataGenerator.getMinFlushEntity(insertTs);
+        te1.addEntity(entityApp1);
+        entityApp2 = TestFlowDataGenerator.getMaxFlushEntity(insertTs);
+        te1.addEntity(entityApp2);
+        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+        Thread.sleep(1);
+
+        appName = "application_1001199480000_7" + appIdSuffix;
+        insertTs++;
+        appIdSuffix++;
+        te1 = new TimelineEntities();
+        entityApp1 = TestFlowDataGenerator.getMinFlushEntity(insertTs);
+        te1.addEntity(entityApp1);
+        entityApp2 = TestFlowDataGenerator.getMaxFlushEntity(insertTs);
+        te1.addEntity(entityApp2);
+
+        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+        if (i % 1000 == 0) {
+          hbi.flush();
+          checkMinMaxFlush(c1, minTS, startTs, count, cluster, user, flow,
+              runid, false);
+        }
+      }
+    } finally {
+      hbi.flush();
+      hbi.close();
+      checkMinMaxFlush(c1, minTS, startTs, count, cluster, user, flow, runid,
+          true);
+    }
+  }
+
+  private void checkMinMaxFlush(Configuration c1, long minTS, long startTs,
+      int count, String cluster, String user, String flow, long runid,
+      boolean checkMax) throws IOException {
+    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());
+    int start = 10;
+    assertEquals(2, r1.size());
+    long starttime = Bytes.toLong(values
+        .get(FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes()));
+    assertEquals(minTS, starttime);
+    if (checkMax) {
+      assertEquals(startTs + 2 * (count - start)
+          + TestFlowDataGenerator.END_TS_INCR,
+          Bytes.toLong(values
+          .get(FlowRunColumn.MAX_END_TIME.getColumnQualifierBytes())));
+    }
+  }
+
+  @Test
+  public void testFilterFlowRunsByCreatedTime() throws Exception {
+    String cluster = "cluster2";
+    String user = "user2";
+    String flow = "flow_name2";
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1(
+        System.currentTimeMillis());
+    entityApp1.setCreatedTime(1425016501000L);
+    te.addEntity(entityApp1);
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678919L,
+          "application_11111111111111_1111", te);
+      // write another application with same metric to this flow
+      te = new TimelineEntities();
+      TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2(
+          System.currentTimeMillis());
+      entityApp2.setCreatedTime(1425016502000L);
+      te.addEntity(entityApp2);
+      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678918L,
+          "application_11111111111111_2222", te);
+      hbi.flush();
+    } finally {
+      hbi.close();
+    }
+
+    // use the timeline reader to verify data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+
+      Set<TimelineEntity> entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow,
+          null, null, TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, 1425016501000L, 1425016502001L, null,
+          null, null, null, null, null), new TimelineDataToRetrieve());
+      assertEquals(2, entities.size());
+      for (TimelineEntity entity : entities) {
+        if (!entity.getId().equals("user2@flow_name2/1002345678918") &&
+            !entity.getId().equals("user2@flow_name2/1002345678919")) {
+          fail("Entities with flow runs 1002345678918 and 1002345678919" +
+              "should be present.");
+        }
+      }
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, 1425016501050L, null, null, null,
+          null, null, null, null), new TimelineDataToRetrieve());
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        if (!entity.getId().equals("user2@flow_name2/1002345678918")) {
+          fail("Entity with flow run 1002345678918 should be present.");
+        }
+      }
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, null, 1425016501050L, null, null,
+          null, null, null, null), new TimelineDataToRetrieve());
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        if (!entity.getId().equals("user2@flow_name2/1002345678919")) {
+          fail("Entity with flow run 1002345678919 should be present.");
+        }
+      }
+    } finally {
+      hbr.close();
+    }
+  }
+
+  @Test
+  public void testMetricFilters() throws Exception {
+    String cluster = "cluster1";
+    String user = "user1";
+    String flow = "flow_name1";
+
+    TimelineEntities te = new TimelineEntities();
+    TimelineEntity entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1(
+        System.currentTimeMillis());
+    te.addEntity(entityApp1);
+
+    HBaseTimelineWriterImpl hbi = null;
+    Configuration c1 = util.getConfiguration();
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678919L,
+          "application_11111111111111_1111", te);
+      // write another application with same metric to this flow
+      te = new TimelineEntities();
+      TimelineEntity entityApp2 = TestFlowDataGenerator.getEntityMetricsApp2(
+          System.currentTimeMillis());
+      te.addEntity(entityApp2);
+      hbi.write(cluster, user, flow, "CF7022C10F1354", 1002345678918L,
+          "application_11111111111111_2222", te);
+      hbi.flush();
+    } finally {
+      hbi.close();
+    }
+
+    // use the timeline reader to verify data
+    HBaseTimelineReaderImpl hbr = null;
+    try {
+      hbr = new HBaseTimelineReaderImpl();
+      hbr.init(c1);
+      hbr.start();
+
+      TimelineFilterList list1 = new TimelineFilterList();
+      list1.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.GREATER_OR_EQUAL, metric1, 101));
+      TimelineFilterList list2 = new TimelineFilterList();
+      list2.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.LESS_THAN, metric1, 43));
+      list2.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.EQUAL, metric2, 57));
+      TimelineFilterList metricFilterList =
+          new TimelineFilterList(Operator.OR, list1, list2);
+      Set<TimelineEntity> entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null,
+          null, TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, null, null, null, null, null, null,
+          metricFilterList, null),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+      assertEquals(2, entities.size());
+      int metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+      }
+      assertEquals(3, metricCnt);
+
+      TimelineFilterList metricFilterList1 = new TimelineFilterList(
+          new TimelineCompareFilter(
+          TimelineCompareOp.LESS_OR_EQUAL, metric1, 127),
+          new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, metric2, 30));
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, null, null, null, null, null, null,
+          metricFilterList1, null),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+      assertEquals(1, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+      }
+      assertEquals(2, metricCnt);
+
+      TimelineFilterList metricFilterList2 = new TimelineFilterList(
+          new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, metric1, 32),
+          new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, metric2, 57));
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, null, null, null, null, null, null,
+          metricFilterList2, null),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+      assertEquals(0, entities.size());
+
+      TimelineFilterList metricFilterList3 = new TimelineFilterList(
+          new TimelineCompareFilter(TimelineCompareOp.EQUAL, "s_metric", 32));
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, null, null, null, null, null, null,
+          metricFilterList3, null),
+          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+      assertEquals(0, entities.size());
+
+      TimelineFilterList list3 = new TimelineFilterList();
+      list3.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.GREATER_OR_EQUAL, metric1, 101));
+      TimelineFilterList list4 = new TimelineFilterList();
+      list4.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.LESS_THAN, metric1, 43));
+      list4.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.EQUAL, metric2, 57));
+      TimelineFilterList metricFilterList4 =
+          new TimelineFilterList(Operator.OR, list3, list4);
+      TimelineFilterList metricsToRetrieve = new TimelineFilterList(Operator.OR,
+          new TimelinePrefixFilter(TimelineCompareOp.EQUAL,
+          metric2.substring(0, metric2.indexOf("_") + 1)));
+      entities = hbr.getEntities(
+          new TimelineReaderContext(cluster, user, flow, null, null,
+          TimelineEntityType.YARN_FLOW_RUN.toString(), null),
+          new TimelineEntityFilters(null, null, null, null, null, null, null,
+          metricFilterList4, null),
+          new TimelineDataToRetrieve(null, metricsToRetrieve,
+          EnumSet.of(Field.ALL)));
+      assertEquals(2, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+      }
+      assertEquals(1, metricCnt);
+    } finally {
+      hbr.close();
+    }
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    util.shutdownMiniCluster();
+  }
+}


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


[36/50] [abbrv] hadoop git commit: YARN-5109. timestamps are stored unencoded causing parse errors (Varun Saxena via sjlee)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java
----------------------------------------------------------------------
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/EventColumnNameConverter.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/EventColumnNameConverter.java
new file mode 100644
index 0000000..32ef1c3
--- /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/EventColumnNameConverter.java
@@ -0,0 +1,105 @@
+/**
+ * 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.hbase.util.Bytes;
+
+/**
+ * Encodes and decodes event column names for application and entity tables.
+ * The event column name is of the form : eventId=timestamp=infokey.
+ * If info is not associated with the event, event column name is of the form :
+ * eventId=timestamp=
+ * Event timestamp is long and rest are strings.
+ * Column prefixes are not part of the eventcolumn name passed for encoding. It
+ * is added later, if required in the associated ColumnPrefix implementations.
+ */
+public final class EventColumnNameConverter
+    implements KeyConverter<EventColumnName> {
+  private static final EventColumnNameConverter INSTANCE =
+      new EventColumnNameConverter();
+
+  public static EventColumnNameConverter getInstance() {
+    return INSTANCE;
+  }
+
+  private EventColumnNameConverter() {
+  }
+
+  // eventId=timestamp=infokey are of types String, Long String
+  // Strings are variable in size (i.e. end whenever separator is encountered).
+  // This is used while decoding and helps in determining where to split.
+  private static final int[] SEGMENT_SIZES = {
+      Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE };
+
+  /*
+   * (non-Javadoc)
+   *
+   * Encodes EventColumnName into a byte array with each component/field in
+   * EventColumnName separated by Separator#VALUES. This leads to an event
+   * column name of the form eventId=timestamp=infokey.
+   * If timestamp in passed EventColumnName object is null (eventId is not null)
+   * this returns a column prefix of the form eventId= and if infokey in
+   * EventColumnName is null (other 2 components are not null), this returns a
+   * column name of the form eventId=timestamp=
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #encode(java.lang.Object)
+   */
+  @Override
+  public byte[] encode(EventColumnName key) {
+    byte[] first = Separator.encode(key.getId(), Separator.SPACE, Separator.TAB,
+        Separator.VALUES);
+    if (key.getTimestamp() == null) {
+      return Separator.VALUES.join(first, Separator.EMPTY_BYTES);
+    }
+    byte[] second = Bytes.toBytes(
+        TimelineStorageUtils.invertLong(key.getTimestamp()));
+    if (key.getInfoKey() == null) {
+      return Separator.VALUES.join(first, second, Separator.EMPTY_BYTES);
+    }
+    return Separator.VALUES.join(first, second, Separator.encode(
+        key.getInfoKey(), Separator.SPACE, Separator.TAB, Separator.VALUES));
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * Decodes an event column name of the form eventId=timestamp= or
+   * eventId=timestamp=infoKey represented in byte format and converts it into
+   * an EventColumnName object.
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #decode(byte[])
+   */
+  @Override
+  public EventColumnName decode(byte[] bytes) {
+    byte[][] components = Separator.VALUES.split(bytes, SEGMENT_SIZES);
+    if (components.length != 3) {
+      throw new IllegalArgumentException("the column name is not valid");
+    }
+    String id = Separator.decode(Bytes.toString(components[0]),
+        Separator.VALUES, Separator.TAB, Separator.SPACE);
+    Long ts = TimelineStorageUtils.invertLong(Bytes.toLong(components[1]));
+    String infoKey = components[2].length == 0 ? null :
+        Separator.decode(Bytes.toString(components[2]),
+            Separator.VALUES, Separator.TAB, Separator.SPACE);
+    return new EventColumnName(id, ts, infoKey);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/KeyConverter.java
----------------------------------------------------------------------
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/KeyConverter.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/KeyConverter.java
new file mode 100644
index 0000000..4229e81
--- /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/KeyConverter.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.timelineservice.storage.common;
+
+/**
+ * Interface which has to be implemented for encoding and decoding row keys and
+ * columns.
+ */
+public interface KeyConverter<T> {
+  /**
+   * Encodes a key as a byte array.
+   *
+   * @param key key to be encoded.
+   * @return a byte array.
+   */
+  byte[] encode(T key);
+
+  /**
+   * Decodes a byte array and returns a key of type T.
+   *
+   * @param bytes byte representation
+   * @return an object(key) of type T which has been constructed after decoding
+   * the bytes.
+   */
+  T decode(byte[] bytes);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java
----------------------------------------------------------------------
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/LongKeyConverter.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/LongKeyConverter.java
new file mode 100644
index 0000000..3954145
--- /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/LongKeyConverter.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 java.io.IOException;
+/**
+ * Encodes and decodes column names / row keys which are long.
+ */
+public final class LongKeyConverter implements KeyConverter<Long> {
+  private static final LongKeyConverter INSTANCE = new LongKeyConverter();
+
+  public static LongKeyConverter getInstance() {
+    return INSTANCE;
+  }
+
+  private LongKeyConverter() {
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #encode(java.lang.Object)
+   */
+  @Override
+  public byte[] encode(Long key) {
+    try {
+      // IOException will not be thrown here as we are explicitly passing
+      // Long.
+      return LongConverter.getInstance().encodeValue(key);
+    } catch (IOException e) {
+      return null;
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #decode(byte[])
+   */
+  @Override
+  public Long decode(byte[] bytes) {
+    try {
+      return (Long) LongConverter.getInstance().decodeValue(bytes);
+    } catch (IOException e) {
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 a81c717..8a178db 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
@@ -45,7 +45,13 @@ public enum Separator {
    * getting a + for a space, which may already occur in strings, so we don't
    * want that.
    */
-  SPACE(" ", "%2$");
+  SPACE(" ", "%2$"),
+
+  /**
+   * separator in values, often used to avoid having these in qualifiers and
+   * names.
+   */
+  TAB("\t", "%3$");
 
   /**
    * The string value of this separator.
@@ -67,7 +73,22 @@ public enum Separator {
    */
   private final String quotedValue;
 
-  private static final byte[] EMPTY_BYTES = new byte[0];
+  /**
+   * Indicator for variable size of an individual segment in a split. The
+   * segment ends wherever separator is encountered.
+   * Typically used for string.
+   * Also used to indicate that there is no fixed number of splits which need to
+   * be returned. If split limit is specified as this, all possible splits are
+   * returned.
+   */
+  public static final int VARIABLE_SIZE = 0;
+
+
+  /** empty string. */
+  public static final String EMPTY_STRING = "";
+
+  /** empty bytes. */
+  public static final byte[] EMPTY_BYTES = new byte[0];
 
   /**
    * @param value of the separator to use. Cannot be null or empty string.
@@ -222,7 +243,6 @@ public enum Separator {
         System.arraycopy(this.bytes, 0, buf, offset, this.value.length());
         offset += this.value.length();
       }
-
     }
     return buf;
   }
@@ -307,7 +327,25 @@ public enum Separator {
    * @return source split by this separator.
    */
   public byte[][] split(byte[] source, int limit) {
-    return TimelineStorageUtils.split(source, this.bytes, limit);
+    return split(source, this.bytes, limit);
+  }
+
+  /**
+   * Splits the source array into multiple array segments using this separator.
+   * The sizes indicate the sizes of the relative components/segments.
+   * In case one of the segments contains this separator before the specified
+   * size is reached, the separator will be considered part of that segment and
+   * we will continue till size is reached.
+   * Variable length strings cannot contain this separator and are indiced with
+   * a size of {@value #VARIABLE_SIZE}. Such strings are encoded for this
+   * separator and decoded after the results from split is returned.
+   *
+   * @param source byte array to be split.
+   * @param sizes sizes of relative components/segments.
+   * @return source split by this separator as per the sizes specified..
+   */
+  public byte[][] split(byte[] source, int[] sizes) {
+    return split(source, this.bytes, sizes);
   }
 
   /**
@@ -315,10 +353,158 @@ public enum 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
+   * @param source byte array to be split
    * @return source split by this separator.
    */
   public byte[][] split(byte[] source) {
-    return TimelineStorageUtils.split(source, this.bytes);
+    return split(source, this.bytes);
+  }
+
+  /**
+   * 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.
+   * The sizes indicate the sizes of the relative components/segments.
+   * In case one of the segments contains this separator before the specified
+   * size is reached, the separator will be considered part of that segment and
+   * we will continue till size is reached.
+   * Variable length strings cannot contain this separator and are indiced with
+   * a size of {@value #VARIABLE_SIZE}. Such strings are encoded for this
+   * separator and decoded after the results from split is returned.
+   *
+   * @param source the source data
+   * @param separator the separator pattern to look for
+   * @param sizes indicate the sizes of the relative components/segments.
+   * @return a list of ranges.
+   */
+  private static List<Range> splitRanges(byte[] source, byte[] separator,
+      int[] sizes) {
+    List<Range> segments = new ArrayList<Range>();
+    if (source == null || separator == null) {
+      return segments;
+    }
+    // VARIABLE_SIZE here indicates that there is no limit to number of segments
+    // to return.
+    int limit = VARIABLE_SIZE;
+    if (sizes != null && sizes.length > 0) {
+      limit = sizes.length;
+    }
+    int start = 0;
+    int currentSegment = 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 > VARIABLE_SIZE) {
+        if (segments.size() >= (limit - 1)) {
+          // everything else goes in one final segment
+          break;
+        }
+        if (sizes != null) {
+          int currentSegExpectedSize = sizes[currentSegment];
+          if (currentSegExpectedSize > VARIABLE_SIZE) {
+            int currentSegSize = i - start;
+            if (currentSegSize < currentSegExpectedSize) {
+              // Segment not yet complete. More bytes to parse.
+              continue itersource;
+            } else if (currentSegSize > currentSegExpectedSize) {
+              // Segment is not as per size.
+              throw new IllegalArgumentException(
+                  "Segments not separated as per expected sizes");
+            }
+          }
+        }
+      }
+      segments.add(new Range(start, i));
+      start = i + separator.length;
+      // i will be incremented again in outer for loop
+      i += separator.length - 1;
+      currentSegment++;
+    }
+    // add in remaining to a final range
+    if (start <= source.length) {
+      if (sizes != null) {
+        // Check if final segment is as per size specified.
+        if (sizes[currentSegment] > VARIABLE_SIZE &&
+            source.length - start > sizes[currentSegment]) {
+          // Segment is not as per size.
+          throw new IllegalArgumentException(
+              "Segments not separated as per expected sizes");
+        }
+      }
+      segments.add(new Range(start, source.length));
+    }
+    return segments;
+  }
+
+  /**
+   * Splits based on segments calculated based on limit/sizes specified for the
+   * separator.
+   *
+   * @param source byte array to be split.
+   * @param segments specifies the range for each segment.
+   * @return a byte[][] split as per the segment ranges.
+   */
+  private static byte[][] split(byte[] source, List<Range> segments) {
+    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;
+  }
+
+  /**
+   * Splits the source array into multiple array segments using the given
+   * separator based on the sizes. This will naturally produce copied byte
+   * arrays for each of the split segments.
+   *
+   * @param source source array.
+   * @param separator separator represented as a byte array.
+   * @param sizes sizes of relative components/segments.
+   * @return byte[][] after splitting the source.
+   */
+  private static byte[][] split(byte[] source, byte[] separator, int[] sizes) {
+    List<Range> segments = splitRanges(source, separator, sizes);
+    return split(source, segments);
+  }
+
+  /**
+   * Splits the source array into multiple array segments using the given
+   * separator. This will naturally produce copied byte arrays for each of the
+   * split segments.
+   *
+   * @param source Source array.
+   * @param separator Separator represented as a byte array.
+   * @return byte[][] after splitting the source.
+   */
+  private static byte[][] split(byte[] source, byte[] separator) {
+    return split(source, separator, (int[]) null);
+  }
+
+  /**
+   * 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.
+   *
+   * @param source Source array.
+   * @param separator Separator represented as a byte array.
+   * @param limit a non-positive value indicates no limit on number of segments.
+   * @return byte[][] after splitting the input source.
+   */
+  private static byte[][] split(byte[] source, byte[] separator, int limit) {
+    int[] sizes = null;
+    if (limit > VARIABLE_SIZE) {
+      sizes = new int[limit];
+    }
+    List<Range> segments = splitRanges(source, separator, sizes);
+    return split(source, segments);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java
----------------------------------------------------------------------
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/StringKeyConverter.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/StringKeyConverter.java
new file mode 100644
index 0000000..b0f6d55
--- /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/StringKeyConverter.java
@@ -0,0 +1,59 @@
+/**
+ * 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;
+
+/**
+ * Encodes and decodes column names / row keys which are merely strings.
+ * Column prefixes are not part of the column name passed for encoding. It is
+ * added later, if required in the associated ColumnPrefix implementations.
+ */
+public final class StringKeyConverter implements KeyConverter<String> {
+  private static final StringKeyConverter INSTANCE = new StringKeyConverter();
+
+  public static StringKeyConverter getInstance() {
+    return INSTANCE;
+  }
+
+  private StringKeyConverter() {
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #encode(java.lang.Object)
+   */
+  @Override
+  public byte[] encode(String key) {
+    return Separator.encode(key, Separator.SPACE, Separator.TAB);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #decode(byte[])
+   */
+  @Override
+  public String decode(byte[] bytes) {
+    return Separator.decode(bytes, Separator.TAB, Separator.SPACE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 18f975a..d52a5d7 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
@@ -18,7 +18,6 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage.common;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -40,7 +39,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.client.Result;
 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;
@@ -48,18 +46,17 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter.TimelineFilterType;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 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.server.timelineservice.storage.flow.FlowRunTable;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 
 /**
  * A bunch of utility functions used across TimelineReader and TimelineWriter.
@@ -72,109 +69,10 @@ public final class TimelineStorageUtils {
 
   private static final Log LOG = LogFactory.getLog(TimelineStorageUtils.class);
 
-  /** 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 Source array.
-   * @param separator Separator represented as a byte array.
-   * @return byte[][] 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 Source array.
-   * @param separator Separator represented as a byte array.
-   * @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.
-   *
-   * @param source Source array.
-   * @param separator Separator represented as a byte array.
-   * @return a list of ranges.
-   */
-  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
-   * @return a list of ranges.
-   */
-  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).
@@ -200,53 +98,6 @@ public final class TimelineStorageUtils {
     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.
@@ -810,7 +661,8 @@ public final class TimelineStorageUtils {
       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);
+    Map<String, Object> columns =
+        prefix.readResults(result, StringKeyConverter.getInstance());
     for (Map.Entry<String, Object> column : columns.entrySet()) {
       for (String id : Separator.VALUES.splitEncoded(
           column.getValue().toString())) {
@@ -837,7 +689,8 @@ public final class TimelineStorageUtils {
       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);
+    Map<String, Object> columns =
+        prefix.readResults(result, StringKeyConverter.getInstance());
     if (isConfig) {
       for (Map.Entry<String, Object> column : columns.entrySet()) {
         entity.addConfig(column.getKey(), column.getValue().toString());
@@ -861,30 +714,24 @@ public final class TimelineStorageUtils {
   public static <T> void readEvents(TimelineEntity entity, Result result,
       ColumnPrefix<T> prefix) throws IOException {
     Map<String, TimelineEvent> eventsMap = new HashMap<>();
-    Map<?, Object> eventsResult =
-        prefix.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;
+    Map<EventColumnName, Object> eventsResult =
+        prefix.readResults(result, EventColumnNameConverter.getInstance());
+    for (Map.Entry<EventColumnName, Object>
+             eventResult : eventsResult.entrySet()) {
+      EventColumnName eventColumnName = eventResult.getKey();
+      String key = eventColumnName.getId() +
+          Long.toString(eventColumnName.getTimestamp());
+      // Retrieve previously seen event to add to it
+      TimelineEvent event = eventsMap.get(key);
+      if (event == null) {
+        // First time we're seeing this event, add it to the eventsMap
+        event = new TimelineEvent();
+        event.setId(eventColumnName.getId());
+        event.setTimestamp(eventColumnName.getTimestamp());
+        eventsMap.put(key, event);
+      }
+      if (eventColumnName.getInfoKey() != null) {
+        event.addInfo(eventColumnName.getInfoKey(), eventResult.getValue());
       }
     }
     Set<TimelineEvent> eventsSet = new HashSet<>(eventsMap.values());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 de2b29d..02a4bb3 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
@@ -27,9 +27,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.ColumnPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
@@ -78,7 +79,6 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    */
   private final String columnPrefix;
   private final byte[] columnPrefixBytes;
-  private final boolean compoundColQual;
 
   /**
    * Private constructor, meant to be used by the enum definition.
@@ -122,7 +122,6 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
       this.columnPrefixBytes =
           Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
     }
-    this.compoundColQual = compondColQual;
   }
 
   /**
@@ -154,14 +153,6 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
     return column.getValueConverter();
   }
 
-  public byte[] getCompoundColQualBytes(String qualifier,
-      byte[]...components) {
-    if (!compoundColQual) {
-      return ColumnHelper.getColumnQualifier(null, qualifier);
-    }
-    return ColumnHelper.getCompoundColumnQualifierBytes(qualifier, components);
-  }
-
   /*
    * (non-Javadoc)
    *
@@ -233,26 +224,12 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    *
    * @see
    * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
-   * #readResults(org.apache.hadoop.hbase.client.Result)
+   * #readResults(org.apache.hadoop.hbase.client.Result,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
    */
-  public Map<String, Object> readResults(Result result) throws IOException {
-    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 if there is any exception encountered while reading
-   *     result.
-   */
-  public Map<?, Object> readResultsHavingCompoundColumnQualifiers(Result result)
-          throws IOException {
-    return column.readResultsHavingCompoundColumnQualifiers(result,
-        columnPrefixBytes);
+  public <K> Map<K, Object> readResults(Result result,
+      KeyConverter<K> keyConverter) throws IOException {
+    return column.readResults(result, columnPrefixBytes, keyConverter);
   }
 
   /*
@@ -260,11 +237,14 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    *
    * @see
    * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
-   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result)
+   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
    */
-  public <V> NavigableMap<String, NavigableMap<Long, V>>
-      readResultsWithTimestamps(Result result) throws IOException {
-    return column.readResultsWithTimestamps(result, columnPrefixBytes);
+  public <K, V> NavigableMap<K, NavigableMap<Long, V>>
+      readResultsWithTimestamps(Result result, KeyConverter<K> keyConverter)
+      throws IOException {
+    return column.readResultsWithTimestamps(result, columnPrefixBytes,
+        keyConverter);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 04c633c..6d08390 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
@@ -17,10 +17,6 @@
  */
 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.TimelineStorageUtils;
-
 /**
  * Represents a rowkey for the entity table.
  */
@@ -28,13 +24,13 @@ public class EntityRowKey {
   private final String clusterId;
   private final String userId;
   private final String flowName;
-  private final long flowRunId;
+  private final Long flowRunId;
   private final String appId;
   private final String entityType;
   private final String entityId;
 
   public EntityRowKey(String clusterId, String userId, String flowName,
-      long flowRunId, String appId, String entityType, String entityId) {
+      Long flowRunId, String appId, String entityType, String entityId) {
     this.clusterId = clusterId;
     this.userId = userId;
     this.flowName = flowName;
@@ -56,7 +52,7 @@ public class EntityRowKey {
     return flowName;
   }
 
-  public long getFlowRunId() {
+  public Long getFlowRunId() {
     return flowRunId;
   }
 
@@ -85,14 +81,8 @@ public class EntityRowKey {
    */
   public static byte[] getRowKeyPrefix(String clusterId, String userId,
       String flowName, Long flowRunId, String appId) {
-    byte[] first =
-        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(userId, clusterId,
-            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));
-    byte[] third = TimelineStorageUtils.encodeAppId(appId);
-    return Separator.QUALIFIERS.join(first, second, third, new byte[0]);
+    return EntityRowKeyConverter.getInstance().encode(new EntityRowKey(
+        clusterId, userId, flowName, flowRunId, appId, null, null));
   }
 
   /**
@@ -111,16 +101,8 @@ public class EntityRowKey {
    */
   public static byte[] getRowKeyPrefix(String clusterId, String userId,
       String flowName, Long flowRunId, String appId, String entityType) {
-    byte[] first =
-        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(userId, clusterId,
-            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));
-    byte[] third = TimelineStorageUtils.encodeAppId(appId);
-    byte[] fourth =
-        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(entityType, ""));
-    return Separator.QUALIFIERS.join(first, second, third, fourth);
+    return EntityRowKeyConverter.getInstance().encode(new EntityRowKey(
+        clusterId, userId, flowName, flowRunId, appId, entityType, null));
   }
 
   /**
@@ -140,16 +122,8 @@ public class EntityRowKey {
   public static byte[] getRowKey(String clusterId, String userId,
       String flowName, Long flowRunId, String appId, String entityType,
       String entityId) {
-    byte[] first =
-        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(userId, clusterId,
-            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));
-    byte[] third = TimelineStorageUtils.encodeAppId(appId);
-    byte[] fourth =
-        Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(entityType, entityId));
-    return Separator.QUALIFIERS.join(first, second, third, fourth);
+    return EntityRowKeyConverter.getInstance().encode(new EntityRowKey(
+        clusterId, userId, flowName, flowRunId, appId, entityType, entityId));
   }
 
   /**
@@ -159,27 +133,6 @@ public class EntityRowKey {
    * @return An <cite>EntityRowKey</cite> 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 flowName =
-        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[2]));
-    long flowRunId =
-        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
-    String appId = TimelineStorageUtils.decodeAppId(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, flowName, flowRunId, appId,
-        entityType, entityId);
+    return EntityRowKeyConverter.getInstance().decode(rowKey);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyConverter.java
----------------------------------------------------------------------
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/EntityRowKeyConverter.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/EntityRowKeyConverter.java
new file mode 100644
index 0000000..43c0569
--- /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/entity/EntityRowKeyConverter.java
@@ -0,0 +1,143 @@
+/**
+ * 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.entity;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+
+/**
+ * Encodes and decodes row key for entity table.
+ * The row key is of the form :
+ * userName!clusterId!flowName!flowRunId!appId!entityType!entityId.
+ * flowRunId is a long, appId is encoded/decoded using
+ * {@link AppIdKeyConverter} and rest are strings.
+ */
+public final class EntityRowKeyConverter implements KeyConverter<EntityRowKey> {
+  private static final EntityRowKeyConverter INSTANCE =
+      new EntityRowKeyConverter();
+
+  public static EntityRowKeyConverter getInstance() {
+    return INSTANCE;
+  }
+
+  private EntityRowKeyConverter() {
+  }
+
+  // Entity row key is of the form
+  // userName!clusterId!flowName!flowRunId!appId!entityType!entityId with each
+  // segment separated by !. The sizes below indicate sizes of each one of these
+  // segements in sequence. clusterId, userName, flowName, entityType and
+  // entityId are strings. flowrunId is a long hence 8 bytes in size. app id is
+  // represented as 12 bytes with cluster timestamp part of appid being 8 bytes
+  // (long) and seq id being 4 bytes(int).
+  // Strings are variable in size (i.e. end whenever separator is encountered).
+  // This is used while decoding and helps in determining where to split.
+  private static final int[] SEGMENT_SIZES = {
+      Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+      Bytes.SIZEOF_LONG, AppIdKeyConverter.getKeySize(),
+      Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE };
+
+  /*
+   * (non-Javadoc)
+   *
+   * Encodes EntityRowKey object into a byte array with each component/field in
+   * EntityRowKey separated by Separator#QUALIFIERS. This leads to an entity
+   * table row key of the form
+   * userName!clusterId!flowName!flowRunId!appId!entityType!entityId
+   * If entityType in passed EntityRowKey object is null (and the fields
+   * preceding it i.e. clusterId, userId and flowName, flowRunId and appId are
+   * not null), this returns a row key prefix of the form
+   * userName!clusterId!flowName!flowRunId!appId! and if entityId in
+   * EntityRowKey is null (other 6 components are not null), this returns a row
+   * key prefix of the form
+   * userName!clusterId!flowName!flowRunId!appId!entityType!
+   * flowRunId is inverted while encoding as it helps maintain a descending
+   * order for row keys in entity table.
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #encode(java.lang.Object)
+   */
+  @Override
+  public byte[] encode(EntityRowKey rowKey) {
+    byte[] user = Separator.encode(rowKey.getUserId(),
+        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
+    byte[] cluster = Separator.encode(rowKey.getClusterId(),
+        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
+    byte[] flow = Separator.encode(rowKey.getFlowName(),
+        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
+    byte[] first = Separator.QUALIFIERS.join(user, cluster, flow);
+    // Note that flowRunId is a long, so we can't encode them all at the same
+    // time.
+    byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(
+        rowKey.getFlowRunId()));
+    byte[] third = AppIdKeyConverter.getInstance().encode(rowKey.getAppId());
+    if (rowKey.getEntityType() == null) {
+      return Separator.QUALIFIERS.join(
+          first, second, third, Separator.EMPTY_BYTES);
+    }
+    byte[] entityType = Separator.encode(rowKey.getEntityType(),
+        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
+    byte[] entityId = rowKey.getEntityId() == null ? Separator.EMPTY_BYTES :
+        Separator.encode(rowKey.getEntityId(), Separator.SPACE, Separator.TAB,
+        Separator.QUALIFIERS);
+    byte[] fourth = Separator.QUALIFIERS.join(entityType, entityId);
+    return Separator.QUALIFIERS.join(first, second, third, fourth);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * Decodes an application row key of the form
+   * userName!clusterId!flowName!flowRunId!appId!entityType!entityId represented
+   * in byte format and converts it into an EntityRowKey object. flowRunId is
+   * inverted while decoding as it was inverted while encoding.
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #decode(byte[])
+   */
+  @Override
+  public EntityRowKey decode(byte[] rowKey) {
+    byte[][] rowKeyComponents =
+        Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
+    if (rowKeyComponents.length != 7) {
+      throw new IllegalArgumentException("the row key is not valid for " +
+          "an entity");
+    }
+    String userId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[1]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    String flowName = Separator.decode(Bytes.toString(rowKeyComponents[2]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    Long flowRunId =
+        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
+    String appId = AppIdKeyConverter.getInstance().decode(rowKeyComponents[4]);
+    String entityType = Separator.decode(Bytes.toString(rowKeyComponents[5]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    String entityId =Separator.decode(Bytes.toString(rowKeyComponents[6]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    return new EntityRowKey(clusterId, userId, flowName, flowRunId, appId,
+        entityType, entityId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 188c2fe..71c3d90 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
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
@@ -51,7 +52,6 @@ public enum FlowActivityColumnPrefix
    */
   private final String columnPrefix;
   private final byte[] columnPrefixBytes;
-  private final boolean compoundColQual;
 
   private final AggregationOperation aggOp;
 
@@ -83,7 +83,6 @@ public enum FlowActivityColumnPrefix
           .encode(columnPrefix));
     }
     this.aggOp = aggOp;
-    this.compoundColQual = compoundColQual;
   }
 
   /**
@@ -169,10 +168,12 @@ public enum FlowActivityColumnPrefix
    *
    * @see
    * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
-   * #readResults(org.apache.hadoop.hbase.client.Result)
+   * #readResults(org.apache.hadoop.hbase.client.Result,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
    */
-  public Map<String, Object> readResults(Result result) throws IOException {
-    return column.readResults(result, columnPrefixBytes);
+  public <K> Map<K, Object> readResults(Result result,
+      KeyConverter<K> keyConverter) throws IOException {
+    return column.readResults(result, columnPrefixBytes, keyConverter);
   }
 
   /*
@@ -180,11 +181,14 @@ public enum FlowActivityColumnPrefix
    *
    * @see
    * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
-   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result)
+   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
    */
-  public <T> NavigableMap<String, NavigableMap<Long, T>>
-      readResultsWithTimestamps(Result result) throws IOException {
-    return column.readResultsWithTimestamps(result, columnPrefixBytes);
+  public <K, V> NavigableMap<K, NavigableMap<Long, V>>
+      readResultsWithTimestamps(Result result, KeyConverter<K> keyConverter)
+      throws IOException {
+    return column.readResultsWithTimestamps(result, columnPrefixBytes,
+        keyConverter);
   }
 
   /**
@@ -270,20 +274,4 @@ public enum FlowActivityColumnPrefix
     column.store(rowKey, tableMutator, columnQualifier, null, inputValue,
         combinedAttributes);
   }
-
-  @Override
-  public byte[] getCompoundColQualBytes(String qualifier,
-      byte[]...components) {
-    if (!compoundColQual) {
-      return ColumnHelper.getColumnQualifier(null, qualifier);
-    }
-    return ColumnHelper.getCompoundColumnQualifierBytes(qualifier, components);
-  }
-
-  @Override
-  public Map<?, Object> readResultsHavingCompoundColumnQualifiers(Result result)
-      throws IOException {
-    // There are no compound column qualifiers for flow activity table.
-    return null;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 2726ae2..eea38a5 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
@@ -17,8 +17,6 @@
  */
 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.TimelineStorageUtils;
 
 /**
@@ -27,11 +25,11 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStor
 public class FlowActivityRowKey {
 
   private final String clusterId;
-  private final long dayTs;
+  private final Long dayTs;
   private final String userId;
   private final String flowName;
 
-  public FlowActivityRowKey(String clusterId, long dayTs, String userId,
+  public FlowActivityRowKey(String clusterId, Long dayTs, String userId,
       String flowName) {
     this.clusterId = clusterId;
     this.dayTs = dayTs;
@@ -43,7 +41,7 @@ public class FlowActivityRowKey {
     return clusterId;
   }
 
-  public long getDayTimestamp() {
+  public Long getDayTimestamp() {
     return dayTs;
   }
 
@@ -63,7 +61,8 @@ public class FlowActivityRowKey {
    * @return byte array with the row key prefix
    */
   public static byte[] getRowKeyPrefix(String clusterId) {
-    return Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId, ""));
+    return FlowActivityRowKeyConverter.getInstance().encode(
+        new FlowActivityRowKey(clusterId, null, null, null));
   }
 
   /**
@@ -75,9 +74,8 @@ public class FlowActivityRowKey {
    * @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]);
+    return FlowActivityRowKeyConverter.getInstance().encode(
+        new FlowActivityRowKey(clusterId, dayTs, null, null));
   }
 
   /**
@@ -94,12 +92,8 @@ public class FlowActivityRowKey {
       String flowName) {
     // convert it to Day's time stamp
     eventTs = TimelineStorageUtils.getTopOfTheDayTimestamp(eventTs);
-
-    return Separator.QUALIFIERS.join(
-        Bytes.toBytes(Separator.QUALIFIERS.encode(clusterId)),
-        Bytes.toBytes(TimelineStorageUtils.invertLong(eventTs)),
-        Bytes.toBytes(Separator.QUALIFIERS.encode(userId)),
-        Bytes.toBytes(Separator.QUALIFIERS.encode(flowName)));
+    return FlowActivityRowKeyConverter.getInstance().encode(
+        new FlowActivityRowKey(clusterId, eventTs, userId, flowName));
   }
 
   /**
@@ -109,21 +103,6 @@ public class FlowActivityRowKey {
    * @return A <cite>FlowActivityRowKey</cite> 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 =
-        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[1]));
-    String userId = Separator.QUALIFIERS.decode(Bytes
-        .toString(rowKeyComponents[2]));
-    String flowName = Separator.QUALIFIERS.decode(Bytes
-        .toString(rowKeyComponents[3]));
-    return new FlowActivityRowKey(clusterId, dayTs, userId, flowName);
+    return FlowActivityRowKeyConverter.getInstance().decode(rowKey);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyConverter.java
----------------------------------------------------------------------
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/FlowActivityRowKeyConverter.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/FlowActivityRowKeyConverter.java
new file mode 100644
index 0000000..9dc4c98
--- /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/FlowActivityRowKeyConverter.java
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+
+/**
+ * Encodes and decodes row key for flow activity table.
+ * The row key is of the form : clusterId!dayTimestamp!user!flowName.
+ * dayTimestamp(top of the day timestamp) is a long and rest are strings.
+ */
+public final class FlowActivityRowKeyConverter implements
+    KeyConverter<FlowActivityRowKey> {
+  private static final FlowActivityRowKeyConverter INSTANCE =
+      new FlowActivityRowKeyConverter();
+
+  public static FlowActivityRowKeyConverter getInstance() {
+    return INSTANCE;
+  }
+
+  private FlowActivityRowKeyConverter() {
+  }
+
+  // Flow activity row key is of the form clusterId!dayTimestamp!user!flowName
+  // with each segment separated by !. The sizes below indicate sizes of each
+  // one of these segements in sequence. clusterId, user and flowName are
+  // strings. Top of the day timestamp is a long hence 8 bytes in size.
+  // Strings are variable in size (i.e. end whenever separator is encountered).
+  // This is used while decoding and helps in determining where to split.
+  private static final int[] SEGMENT_SIZES = {
+      Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE,
+      Separator.VARIABLE_SIZE };
+
+  /*
+   * (non-Javadoc)
+   *
+   * Encodes FlowActivityRowKey object into a byte array with each
+   * component/field in FlowActivityRowKey separated by Separator#QUALIFIERS.
+   * This leads to an flow activity table row key of the form
+   * clusterId!dayTimestamp!user!flowName
+   * If dayTimestamp in passed FlowActivityRowKey object is null and clusterId
+   * is not null, this returns a row key prefix as clusterId! and if userId in
+   * FlowActivityRowKey is null (and the fields preceding it i.e. clusterId and
+   * dayTimestamp are not null), this returns a row key prefix as
+   * clusterId!dayTimeStamp!
+   * dayTimestamp is inverted while encoding as it helps maintain a descending
+   * order for row keys in flow activity table.
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #encode(java.lang.Object)
+   */
+
+  @Override
+  public byte[] encode(FlowActivityRowKey rowKey) {
+    if (rowKey.getDayTimestamp() == null) {
+      return Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(),
+          Separator.SPACE, Separator.TAB, Separator.QUALIFIERS),
+              Separator.EMPTY_BYTES);
+    }
+    if (rowKey.getUserId() == null) {
+      return Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(),
+          Separator.SPACE, Separator.TAB, Separator.QUALIFIERS),
+          Bytes.toBytes(TimelineStorageUtils.invertLong(
+              rowKey.getDayTimestamp())), Separator.EMPTY_BYTES);
+    }
+    return Separator.QUALIFIERS.join(
+        Separator.encode(rowKey.getClusterId(), Separator.SPACE, Separator.TAB,
+            Separator.QUALIFIERS),
+        Bytes.toBytes(
+            TimelineStorageUtils.invertLong(rowKey.getDayTimestamp())),
+        Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
+            Separator.QUALIFIERS),
+        Separator.encode(rowKey.getFlowName(), Separator.SPACE, Separator.TAB,
+            Separator.QUALIFIERS));
+  }
+
+  @Override
+  public FlowActivityRowKey decode(byte[] rowKey) {
+    byte[][] rowKeyComponents =
+        Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
+    if (rowKeyComponents.length != 4) {
+      throw new IllegalArgumentException("the row key is not valid for "
+          + "a flow activity");
+    }
+    String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    Long dayTs =
+        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[1]));
+    String userId = Separator.decode(Bytes.toString(rowKeyComponents[2]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    String flowName = Separator.decode(Bytes.toString(rowKeyComponents[3]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    return new FlowActivityRowKey(clusterId, dayTs, userId, flowName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 77f2ab2..0f14c89 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
@@ -26,10 +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.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.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;
 
 /**
@@ -40,8 +41,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
   /**
    * To store flow run info values.
    */
-  METRIC(FlowRunColumnFamily.INFO, "m", null,
-      LongConverter.getInstance());
+  METRIC(FlowRunColumnFamily.INFO, "m", null, LongConverter.getInstance());
 
   private final ColumnHelper<FlowRunTable> column;
   private final ColumnFamily<FlowRunTable> columnFamily;
@@ -52,17 +52,14 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
    */
   private final String columnPrefix;
   private final byte[] columnPrefixBytes;
-  private final boolean compoundColQual;
 
   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.
+   * @param columnFamily that this column is stored in.
+   * @param columnPrefix for this column.
    */
   private FlowRunColumnPrefix(ColumnFamily<FlowRunTable> columnFamily,
       String columnPrefix, AggregationOperation fra, ValueConverter converter) {
@@ -79,11 +76,10 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
       this.columnPrefixBytes = null;
     } else {
       // Future-proof by ensuring the right column prefix hygiene.
-      this.columnPrefixBytes = Bytes.toBytes(Separator.SPACE
-          .encode(columnPrefix));
+      this.columnPrefixBytes =
+          Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
     }
     this.aggOp = fra;
-    this.compoundColQual = compoundColQual;
   }
 
   /**
@@ -99,14 +95,14 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
 
   @Override
   public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) {
-    return ColumnHelper.getColumnQualifier(
-        this.columnPrefixBytes, qualifierPrefix);
+    return ColumnHelper.getColumnQualifier(this.columnPrefixBytes,
+        qualifierPrefix);
   }
 
   @Override
   public byte[] getColumnPrefixBytes(String qualifierPrefix) {
-    return ColumnHelper.getColumnQualifier(
-        this.columnPrefixBytes, qualifierPrefix);
+    return ColumnHelper.getColumnQualifier(this.columnPrefixBytes,
+        qualifierPrefix);
   }
 
   @Override
@@ -139,8 +135,8 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
     }
 
     byte[] columnQualifier = getColumnPrefixBytes(qualifier);
-    Attribute[] combinedAttributes = TimelineStorageUtils.combineAttributes(
-        attributes, this.aggOp);
+    Attribute[] combinedAttributes =
+        TimelineStorageUtils.combineAttributes(attributes, this.aggOp);
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         combinedAttributes);
   }
@@ -166,8 +162,8 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
     }
 
     byte[] columnQualifier = getColumnPrefixBytes(qualifier);
-    Attribute[] combinedAttributes = TimelineStorageUtils.combineAttributes(
-        attributes, this.aggOp);
+    Attribute[] combinedAttributes =
+        TimelineStorageUtils.combineAttributes(attributes, this.aggOp);
     column.store(rowKey, tableMutator, columnQualifier, timestamp, inputValue,
         combinedAttributes);
   }
@@ -180,8 +176,8 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
    * #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);
+    byte[] columnQualifier =
+        ColumnHelper.getColumnQualifier(this.columnPrefixBytes, qualifier);
     return column.readResult(result, columnQualifier);
   }
 
@@ -190,10 +186,12 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
    *
    * @see
    * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
-   * #readResults(org.apache.hadoop.hbase.client.Result)
+   * #readResults(org.apache.hadoop.hbase.client.Result,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
    */
-  public Map<String, Object> readResults(Result result) throws IOException {
-    return column.readResults(result, columnPrefixBytes);
+  public <K> Map<K, Object> readResults(Result result,
+      KeyConverter<K> keyConverter) throws IOException {
+    return column.readResults(result, columnPrefixBytes, keyConverter);
   }
 
   /*
@@ -201,11 +199,14 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
    *
    * @see
    * org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix
-   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result)
+   * #readResultsWithTimestamps(org.apache.hadoop.hbase.client.Result,
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter)
    */
-  public <T> NavigableMap<String, NavigableMap<Long, T>>
-      readResultsWithTimestamps(Result result) throws IOException {
-    return column.readResultsWithTimestamps(result, columnPrefixBytes);
+  public <K, V> NavigableMap<K, NavigableMap<Long, V>>
+      readResultsWithTimestamps(Result result, KeyConverter<K> keyConverter)
+      throws IOException {
+    return column.readResultsWithTimestamps(result, columnPrefixBytes,
+        keyConverter);
   }
 
   /**
@@ -213,8 +214,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
    * 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
+   * @param columnPrefix Name of the column to retrieve
    * @return the corresponding {@link FlowRunColumnPrefix} or null
    */
   public static final FlowRunColumnPrefix columnFor(String columnPrefix) {
@@ -242,10 +242,8 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
    * {@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
+   * @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.
    */
@@ -267,20 +265,4 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
     // Default to null
     return null;
   }
-
-  @Override
-  public byte[] getCompoundColQualBytes(String qualifier,
-      byte[]...components) {
-    if (!compoundColQual) {
-      return ColumnHelper.getColumnQualifier(null, qualifier);
-    }
-    return ColumnHelper.getCompoundColumnQualifierBytes(qualifier, components);
-  }
-
-  @Override
-  public Map<?, Object> readResultsHavingCompoundColumnQualifiers(Result result)
-      throws IOException {
-    // There are no compound column qualifiers for flow run table.
-    return null;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 eac8f05..925242b 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
@@ -17,10 +17,6 @@
  */
 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.TimelineStorageUtils;
-
 /**
  * Represents a rowkey for the flow run table.
  */
@@ -28,10 +24,10 @@ public class FlowRunRowKey {
   private final String clusterId;
   private final String userId;
   private final String flowName;
-  private final long flowRunId;
+  private final Long flowRunId;
 
   public FlowRunRowKey(String clusterId, String userId, String flowName,
-      long flowRunId) {
+      Long flowRunId) {
     this.clusterId = clusterId;
     this.userId = userId;
     this.flowName = flowName;
@@ -50,7 +46,7 @@ public class FlowRunRowKey {
     return flowName;
   }
 
-  public long getFlowRunId() {
+  public Long getFlowRunId() {
     return flowRunId;
   }
 
@@ -65,13 +61,13 @@ public class FlowRunRowKey {
    */
   public static byte[] getRowKeyPrefix(String clusterId, String userId,
       String flowName) {
-    return Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId, userId,
-        flowName, ""));
+    return FlowRunRowKeyConverter.getInstance().encode(new FlowRunRowKey(
+        clusterId, userId, flowName, null));
   }
 
   /**
    * Constructs a row key for the entity table as follows: {
-   * clusterId!userI!flowName!Inverted Flow Run Id}.
+   * clusterId!userId!flowName!Inverted Flow Run Id}.
    *
    * @param clusterId Cluster Id.
    * @param userId User Id.
@@ -81,12 +77,8 @@ public class FlowRunRowKey {
    */
   public static byte[] getRowKey(String clusterId, String userId,
       String flowName, Long flowRunId) {
-    byte[] first = Bytes.toBytes(Separator.QUALIFIERS.joinEncoded(clusterId,
-        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));
-    return Separator.QUALIFIERS.join(first, second);
+    return FlowRunRowKeyConverter.getInstance().encode(new FlowRunRowKey(
+        clusterId, userId, flowName, flowRunId));
   }
 
   /**
@@ -96,22 +88,7 @@ public class FlowRunRowKey {
    * @return A <cite>FlowRunRowKey</cite> 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 flowName =
-        Separator.QUALIFIERS.decode(Bytes.toString(rowKeyComponents[2]));
-    long flowRunId =
-        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
-    return new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
+    return FlowRunRowKeyConverter.getInstance().decode(rowKey);
   }
 
   /**


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


[14/50] [abbrv] hadoop git commit: YARN-5102. timeline service build fails with java 8. (Sangjin Lee via varunsaxena)

Posted by vr...@apache.org.
YARN-5102. timeline service build fails with java 8. (Sangjin Lee via varunsaxena)


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

Branch: refs/heads/YARN-2928
Commit: a5daac5c89bf36cd4fd8506a3e3cb1504585287b
Parents: ccdec4a
Author: Varun Saxena <va...@apache.org>
Authored: Wed May 18 01:18:18 2016 +0530
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:08 2016 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5daac5c/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 04464e4..0a4f058 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1042,6 +1042,12 @@
         <groupId>org.apache.hbase</groupId>
         <artifactId>hbase-common</artifactId>
         <version>${hbase.version}</version>
+        <exclusions>
+          <exclusion>
+            <artifactId>jdk.tools</artifactId>
+            <groupId>jdk.tools</groupId>
+          </exclusion>
+        </exclusions>
       </dependency>
       <dependency>
         <groupId>org.apache.hbase</groupId>


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


[05/50] [abbrv] hadoop git commit: YARN-3863. Support complex filters in TimelineReader (Varun Saxena via sjlee)

Posted by vr...@apache.org.
YARN-3863. Support complex filters 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/366eb54e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/366eb54e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/366eb54e

Branch: refs/heads/YARN-2928
Commit: 366eb54e1e50e4c00c9904075db714fe65b6649c
Parents: ed3c74b
Author: Sangjin Lee <sj...@apache.org>
Authored: Mon Apr 11 21:07:32 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:05 2016 -0700

----------------------------------------------------------------------
 .../reader/TimelineEntityFilters.java           |  170 +-
 .../reader/TimelineReaderWebServicesUtils.java  |   88 +-
 .../reader/filter/TimelineCompareFilter.java    |   35 +-
 .../reader/filter/TimelineExistsFilter.java     |   62 +
 .../reader/filter/TimelineFilter.java           |   16 +-
 .../reader/filter/TimelineFilterList.java       |   14 +
 .../reader/filter/TimelineFilterUtils.java      |  206 +-
 .../reader/filter/TimelineKeyValueFilter.java   |   48 +
 .../reader/filter/TimelineKeyValuesFilter.java  |   71 +
 .../reader/filter/TimelinePrefixFilter.java     |    6 +
 .../storage/FileSystemTimelineReaderImpl.java   |   36 +-
 .../storage/HBaseTimelineWriterImpl.java        |   31 +-
 .../storage/application/ApplicationColumn.java  |   28 +-
 .../application/ApplicationColumnPrefix.java    |   37 +-
 .../storage/apptoflow/AppToFlowColumn.java      |   16 +
 .../timelineservice/storage/common/Column.java  |   17 +
 .../storage/common/ColumnHelper.java            |   16 +
 .../storage/common/ColumnPrefix.java            |   35 +
 .../common/TimelineEntityFiltersType.java       |   71 +
 .../storage/common/TimelineStorageUtils.java    |  461 +++-
 .../storage/entity/EntityColumn.java            |   28 +-
 .../storage/entity/EntityColumnPrefix.java      |   38 +-
 .../storage/flow/FlowActivityColumnPrefix.java  |   35 +
 .../storage/flow/FlowRunColumn.java             |    3 +
 .../storage/flow/FlowRunColumnPrefix.java       |   26 +
 .../storage/flow/FlowScanner.java               |    1 +
 .../storage/reader/ApplicationEntityReader.java |  426 ++--
 .../reader/FlowActivityEntityReader.java        |    7 +
 .../storage/reader/FlowRunEntityReader.java     |   97 +-
 .../storage/reader/GenericEntityReader.java     |  623 ++---
 .../storage/reader/TimelineEntityReader.java    |   71 +-
 .../reader/TestTimelineReaderWebServices.java   |   10 +-
 .../TestFileSystemTimelineReaderImpl.java       |  332 ++-
 .../storage/TestHBaseTimelineStorage.java       | 2172 +++++++++++++++++-
 .../storage/flow/TestFlowDataGenerator.java     |   16 +
 .../storage/flow/TestHBaseStorageFlowRun.java   |  267 ++-
 .../flow/TestHBaseStorageFlowRunCompaction.java |    2 +-
 .../reader/filter/TimelineExistsFilter.java     |   62 +
 .../reader/filter/TimelineKeyValueFilter.java   |   48 +
 .../reader/filter/TimelineKeyValuesFilter.java  |   71 +
 .../common/TimelineEntityFiltersType.java       |   71 +
 41 files changed, 5054 insertions(+), 816 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
index 5b2c300..4821d31 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
@@ -18,11 +18,14 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.reader;
 
-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.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
 
 /**
  * Encapsulates information regarding the filters to apply while querying. These
@@ -36,36 +39,81 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
  * <li><b>createdTimeEnd</b> - Matched entities should not be created after
  * this timestamp. If null or {@literal <=0}, defaults to
  * {@link Long#MAX_VALUE}.</li>
- * <li><b>relatesTo</b> - Matched entities should relate to given entities.
- * If null or empty, the relations are not matched.</li>
- * <li><b>isRelatedTo</b> - Matched entities should be related to given
- * entities. If null or empty, the relations are not matched.</li>
+ * <li><b>relatesTo</b> - Matched entities should or should not relate to given
+ * entities depending on what's specified in the filter. The entities in
+ * relatesTo are identified by entity type and id. This is represented as
+ * a {@link TimelineFilterList} object containing
+ * {@link TimelineKeyValuesFilter} objects, each of which contains a
+ * set of values for a key and the comparison operator (equals/not equals). The
+ * key which represents the entity type is a string and values are a set of
+ * entity identifiers (also string). As it is a filter list, relatesTo can be
+ * evaluated with logical AND/OR and we can create a hierarchy of these
+ * {@link TimelineKeyValuesFilter} objects. If null or empty, the relations are
+ * not matched.</li>
+ * <li><b>isRelatedTo</b> - Matched entities should or should not be related
+ * to given entities depending on what's specified in the filter. The entities
+ * in isRelatedTo are identified by entity type and id.  This is represented as
+ * a {@link TimelineFilterList} object containing
+ * {@link TimelineKeyValuesFilter} objects, each of which contains a
+ * set of values for a key and the comparison operator (equals/not equals). The
+ * key which represents the entity type is a string and values are a set of
+ * entity identifiers (also string). As it is a filter list, relatesTo can be
+ * evaluated with logical AND/OR and we can create a hierarchy of these
+ * {@link TimelineKeyValuesFilter} objects. If null or empty, the relations are
+ * not matched.</li>
  * <li><b>infoFilters</b> - Matched entities should have exact matches to
- * the given info represented as key-value pairs. If null or empty, the
- * filter is not applied.</li>
+ * the given info and should be either equal or not equal to given value
+ * depending on what's specified in the filter. This is represented as a
+ * {@link TimelineFilterList} object containing {@link TimelineKeyValueFilter}
+ * objects, each of which contains key-value pairs with a comparison operator
+ * (equals/not equals). The key which represents the info key is a string but
+ * value can be any object. As it is a filter list, info filters can be
+ * evaluated with logical AND/OR and we can create a hierarchy of these
+ * key-value pairs. If null or empty, the filter is not applied.</li>
  * <li><b>configFilters</b> - Matched entities should have exact matches to
- * the given configs represented as key-value pairs. If null or empty, the
- * filter is not applied.</li>
+ * the given configurations and should be either equal or not equal to given
+ * value depending on what's specified in the filter. This is represented as a
+ * {@link TimelineFilterList} object containing {@link TimelineKeyValueFilter}
+ * objects, each of which contains key-value pairs with a comparison operator
+ * (equals/not equals). Both key (which represents config name) and value (which
+ * is config value) are strings. As it is a filter list, config filters can be
+ * evaluated with logical AND/OR and we can create a hierarchy of these
+ * {@link TimelineKeyValueFilter} objects. If null or empty, the filter is not
+ * applied.</li>
  * <li><b>metricFilters</b> - Matched entities should contain the given
- * metrics. If null or empty, the filter is not applied.</li>
- * <li><b>eventFilters</b> - Matched entities should contain the given
- * events. If null or empty, the filter is not applied.</li>
+ * metrics and satisfy the specified relation with the value. This is
+ * represented as a {@link TimelineFilterList} object containing
+ * {@link TimelineCompareFilter} objects, each of which contains key-value pairs
+ * along with the specified relational/comparison operator represented by
+ * {@link TimelineCompareOp}.  The key is a string and value is integer
+ * (Short/Integer/Long). As it is a filter list, metric filters can be evaluated
+ * with logical AND/OR and we can create a hierarchy of these
+ * {@link TimelineCompareFilter} objects. If null or empty, the filter is not
+ * applied.</li>
+ * <li><b>eventFilters</b> - Matched entities should contain or not contain the
+ * given events. This is represented as a {@link TimelineFilterList} object
+ * containing {@link TimelineExistsFilter} objects, each of which contains a
+ * value which must or must not exist depending on comparison operator specified
+ * in the filter. For event filters, the value represents a event id. As it is a
+ * filter list, event filters can be evaluated with logical AND/OR and we can
+ * create a hierarchy of these {@link TimelineExistsFilter} objects. If null or
+ * empty, the filter is not applied.</li>
  * </ul>
  */
 @Private
 @Unstable
 public class TimelineEntityFilters {
-  private Long limit;
-  private Long createdTimeBegin;
-  private Long createdTimeEnd;
-  private Map<String, Set<String>> relatesTo;
-  private Map<String, Set<String>> isRelatedTo;
-  private Map<String, Object> infoFilters;
-  private Map<String, String> configFilters;
-  private Set<String>  metricFilters;
-  private Set<String> eventFilters;
-  private static final Long DEFAULT_BEGIN_TIME = 0L;
-  private static final Long DEFAULT_END_TIME = Long.MAX_VALUE;
+  private long limit;
+  private long createdTimeBegin;
+  private long createdTimeEnd;
+  private TimelineFilterList relatesTo;
+  private TimelineFilterList isRelatedTo;
+  private TimelineFilterList infoFilters;
+  private TimelineFilterList configFilters;
+  private TimelineFilterList metricFilters;
+  private TimelineFilterList eventFilters;
+  private static final long DEFAULT_BEGIN_TIME = 0L;
+  private static final long DEFAULT_END_TIME = Long.MAX_VALUE;
 
   /**
    * Default limit of number of entities to return for getEntities API.
@@ -78,23 +126,26 @@ public class TimelineEntityFilters {
 
   public TimelineEntityFilters(
       Long entityLimit, Long timeBegin, Long timeEnd,
-      Map<String, Set<String>> entityRelatesTo,
-      Map<String, Set<String>> entityIsRelatedTo,
-      Map<String, Object> entityInfoFilters,
-      Map<String, String> entityConfigFilters,
-      Set<String>  entityMetricFilters,
-      Set<String> entityEventFilters) {
-    this.limit = entityLimit;
-    if (this.limit == null || this.limit < 0) {
+      TimelineFilterList entityRelatesTo,
+      TimelineFilterList entityIsRelatedTo,
+      TimelineFilterList entityInfoFilters,
+      TimelineFilterList entityConfigFilters,
+      TimelineFilterList  entityMetricFilters,
+      TimelineFilterList entityEventFilters) {
+    if (entityLimit == null || entityLimit < 0) {
       this.limit = DEFAULT_LIMIT;
+    } else {
+      this.limit = entityLimit;
     }
-    this.createdTimeBegin = timeBegin;
-    if (this.createdTimeBegin == null || this.createdTimeBegin < 0) {
+    if (timeBegin == null || timeBegin < 0) {
       this.createdTimeBegin = DEFAULT_BEGIN_TIME;
+    } else {
+      this.createdTimeBegin = timeBegin;
     }
-    this.createdTimeEnd = timeEnd;
-    if (this.createdTimeEnd == null || this.createdTimeEnd < 0) {
+    if (timeEnd == null || timeEnd < 0) {
       this.createdTimeEnd = DEFAULT_END_TIME;
+    } else {
+      this.createdTimeEnd = timeEnd;
     }
     this.relatesTo = entityRelatesTo;
     this.isRelatedTo = entityIsRelatedTo;
@@ -104,84 +155,87 @@ public class TimelineEntityFilters {
     this.eventFilters = entityEventFilters;
   }
 
-  public Long getLimit() {
+  public long getLimit() {
     return limit;
   }
 
   public void setLimit(Long entityLimit) {
-    this.limit = entityLimit;
-    if (this.limit == null || this.limit < 0) {
+    if (entityLimit == null || entityLimit < 0) {
       this.limit = DEFAULT_LIMIT;
+    } else {
+      this.limit = entityLimit;
     }
   }
 
-  public Long getCreatedTimeBegin() {
+  public long getCreatedTimeBegin() {
     return createdTimeBegin;
   }
 
   public void setCreatedTimeBegin(Long timeBegin) {
-    this.createdTimeBegin = timeBegin;
-    if (this.createdTimeBegin == null || this.createdTimeBegin < 0) {
+    if (timeBegin == null || timeBegin < 0) {
       this.createdTimeBegin = DEFAULT_BEGIN_TIME;
+    } else {
+      this.createdTimeBegin = timeBegin;
     }
   }
 
-  public Long getCreatedTimeEnd() {
+  public long getCreatedTimeEnd() {
     return createdTimeEnd;
   }
 
   public void setCreatedTimeEnd(Long timeEnd) {
-    this.createdTimeEnd = timeEnd;
-    if (this.createdTimeEnd == null || this.createdTimeEnd < 0) {
+    if (timeEnd == null || timeEnd < 0) {
       this.createdTimeEnd = DEFAULT_END_TIME;
+    } else {
+      this.createdTimeEnd = timeEnd;
     }
   }
 
-  public Map<String, Set<String>> getRelatesTo() {
+  public TimelineFilterList getRelatesTo() {
     return relatesTo;
   }
 
-  public void setRelatesTo(Map<String, Set<String>> relations) {
+  public void setRelatesTo(TimelineFilterList relations) {
     this.relatesTo = relations;
   }
 
-  public Map<String, Set<String>> getIsRelatedTo() {
+  public TimelineFilterList getIsRelatedTo() {
     return isRelatedTo;
   }
 
-  public void setIsRelatedTo(Map<String, Set<String>> relations) {
+  public void setIsRelatedTo(TimelineFilterList relations) {
     this.isRelatedTo = relations;
   }
 
-  public Map<String, Object> getInfoFilters() {
+  public TimelineFilterList getInfoFilters() {
     return infoFilters;
   }
 
-  public void setInfoFilters(Map<String, Object> filters) {
+  public void setInfoFilters(TimelineFilterList filters) {
     this.infoFilters = filters;
   }
 
-  public Map<String, String> getConfigFilters() {
+  public TimelineFilterList getConfigFilters() {
     return configFilters;
   }
 
-  public void setConfigFilters(Map<String, String> filters) {
+  public void setConfigFilters(TimelineFilterList filters) {
     this.configFilters = filters;
   }
 
-  public Set<String> getMetricFilters() {
+  public TimelineFilterList getMetricFilters() {
     return metricFilters;
   }
 
-  public void setMetricFilters(Set<String> filters) {
+  public void setMetricFilters(TimelineFilterList filters) {
     this.metricFilters = filters;
   }
 
-  public Set<String> getEventFilters() {
+  public TimelineFilterList getEventFilters() {
     return eventFilters;
   }
 
-  public void setEventFilters(Set<String> filters) {
+  public void setEventFilters(TimelineFilterList filters) {
     this.eventFilters = filters;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
index d12f7e5..57d75db 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
@@ -20,15 +20,19 @@ 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.http.HttpServletRequest;
 
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 
 /**
@@ -87,7 +91,7 @@ final class TimelineReaderWebServicesUtils {
         parseKeyStrValuesStr(isRelatedTo, COMMA_DELIMITER, COLON_DELIMITER),
         parseKeyStrValueObj(infofilters, COMMA_DELIMITER, COLON_DELIMITER),
         parseKeyStrValueStr(conffilters, COMMA_DELIMITER, COLON_DELIMITER),
-        parseValuesStr(metricfilters, COMMA_DELIMITER),
+        parseMetricFilters(metricfilters, COMMA_DELIMITER),
         parseValuesStr(eventfilters, COMMA_DELIMITER));
   }
 
@@ -114,22 +118,26 @@ final class TimelineReaderWebServicesUtils {
    * @param delimiter string is delimited by this delimiter.
    * @return set of strings.
    */
-  static Set<String> parseValuesStr(String str, String delimiter) {
+  static TimelineFilterList parseValuesStr(String str, String delimiter) {
     if (str == null || str.isEmpty()) {
       return null;
     }
-    Set<String> strSet = new HashSet<String>();
+    TimelineFilterList filterList = new TimelineFilterList();
     String[] strs = str.split(delimiter);
     for (String aStr : strs) {
-      strSet.add(aStr.trim());
+      filterList.addFilter(new TimelineExistsFilter(TimelineCompareOp.EQUAL,
+          aStr.trim()));
     }
-    return strSet;
+    return filterList;
   }
 
-  @SuppressWarnings("unchecked")
-  private static <T> void parseKeyValues(Map<String, T> map, String str,
+  private static TimelineFilterList parseKeyValues(String str,
       String pairsDelim, String keyValuesDelim, boolean stringValue,
       boolean multipleValues) {
+    if (str == null) {
+      return null;
+    }
+    TimelineFilterList list = new TimelineFilterList();
     String[] pairs = str.split(pairsDelim);
     for (String pair : pairs) {
       if (pair == null || pair.trim().isEmpty()) {
@@ -143,23 +151,28 @@ final class TimelineReaderWebServicesUtils {
         try {
           Object value =
               GenericObjectMapper.OBJECT_READER.readValue(pairStrs[1].trim());
-          map.put(pairStrs[0].trim(), (T) value);
+          list.addFilter(new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+              pairStrs[0].trim(), value));
         } catch (IOException e) {
-          map.put(pairStrs[0].trim(), (T) pairStrs[1].trim());
+          list.addFilter(new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+              pairStrs[0].trim(), pairStrs[1].trim()));
         }
       } else {
         String key = pairStrs[0].trim();
         if (multipleValues) {
-          Set<String> values = new HashSet<String>();
+          Set<Object> values = new HashSet<Object>();
           for (int i = 1; i < pairStrs.length; i++) {
             values.add(pairStrs[i].trim());
           }
-          map.put(key, (T) values);
+          list.addFilter(new TimelineKeyValuesFilter(
+              TimelineCompareOp.EQUAL, key, values));
         } else {
-          map.put(key, (T) pairStrs[1].trim());
+          list.addFilter(new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+              key, pairStrs[1].trim()));
         }
       }
     }
+    return list;
   }
 
   /**
@@ -175,14 +188,9 @@ final class TimelineReaderWebServicesUtils {
    * @param keyValuesDelim values for a key are delimited by this delimiter.
    * @return a map of key-values with each key having a set of values.
    */
-  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;
+  static TimelineFilterList parseKeyStrValuesStr(String str, String pairsDelim,
+      String keyValuesDelim) {
+    return parseKeyValues(str, pairsDelim, keyValuesDelim, true, true);
   }
 
   /**
@@ -195,14 +203,9 @@ final class TimelineReaderWebServicesUtils {
    * @param keyValDelim key and value are delimited by this delimiter.
    * @return a map of key-value pairs with both key and value being strings.
    */
-  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;
+  static TimelineFilterList parseKeyStrValueStr(String str, String pairsDelim,
+      String keyValDelim) {
+    return parseKeyValues(str, pairsDelim, keyValDelim, true, false);
   }
 
   /**
@@ -216,14 +219,9 @@ final class TimelineReaderWebServicesUtils {
    * @return a map of key-value pairs with key being a string and value, any
    *     object.
    */
-  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;
+  static TimelineFilterList parseKeyStrValueObj(String str, String pairsDelim,
+      String keyValDelim) {
+    return parseKeyValues(str, pairsDelim, keyValDelim, false, false);
   }
 
   /**
@@ -247,6 +245,20 @@ final class TimelineReaderWebServicesUtils {
     return fieldList;
   }
 
+  static TimelineFilterList parseMetricFilters(String str,
+      String delimiter) {
+    if (str == null || str.isEmpty()) {
+      return null;
+    }
+    TimelineFilterList list = new TimelineFilterList();
+    String[] strs = str.split(delimiter);
+    for (String aStr : strs) {
+      list.addFilter(new TimelineCompareFilter(
+          TimelineCompareOp.GREATER_OR_EQUAL, aStr.trim(), 0L));
+    }
+    return list;
+  }
+
   /**
    * Interpret passed string as a long.
    * @param str Passed string.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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
index 14e7124..81902ee 100644
--- 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
@@ -29,17 +29,27 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @Unstable
 public class TimelineCompareFilter extends TimelineFilter {
 
-  private TimelineCompareOp compareOp;
-  private String key;
-  private Object value;
+  private final TimelineCompareOp compareOp;
+  private final String key;
+  private final Object value;
+  // If comparison operator is NOT_EQUAL, this flag decides if we should return
+  // the entity if key does not exist.
+  private final boolean keyMustExist;
 
-  public TimelineCompareFilter() {
-  }
-
-  public TimelineCompareFilter(TimelineCompareOp op, String key, Object val) {
+  public TimelineCompareFilter(TimelineCompareOp op, String key, Object val,
+       boolean keyMustExistFlag) {
     this.compareOp = op;
     this.key = key;
     this.value = val;
+    if (op == TimelineCompareOp.NOT_EQUAL) {
+      this.keyMustExist = keyMustExistFlag;
+    } else {
+      this.keyMustExist = true;
+    }
+  }
+
+  public TimelineCompareFilter(TimelineCompareOp op, String key, Object val) {
+    this(op, key, val, true);
   }
 
   @Override
@@ -58,4 +68,15 @@ public class TimelineCompareFilter extends TimelineFilter {
   public Object getValue() {
     return value;
   }
+
+  public boolean getKeyMustExist() {
+    return keyMustExist;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("%s (%s, %s:%s:%b)",
+        this.getClass().getSimpleName(), this.compareOp.name(),
+        this.key, this.value, this.keyMustExist);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java
----------------------------------------------------------------------
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/TimelineExistsFilter.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/TimelineExistsFilter.java
new file mode 100644
index 0000000..36d0d7b
--- /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/TimelineExistsFilter.java
@@ -0,0 +1,62 @@
+/**
+ * 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 existence of a
+ * value.
+ */
+@Private
+@Unstable
+public class TimelineExistsFilter extends TimelineFilter {
+
+  private final TimelineCompareOp compareOp;
+  private final String value;
+
+  public TimelineExistsFilter(TimelineCompareOp op, String value) {
+    this.value = value;
+    if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
+      throw new IllegalArgumentException("CompareOp for exists filter should " +
+          "be EQUAL or NOT_EQUAL");
+    }
+    this.compareOp = op;
+  }
+
+  @Override
+  public TimelineFilterType getFilterType() {
+    return TimelineFilterType.EXISTS;
+  }
+
+  public String getValue() {
+    return value;
+  }
+
+  public TimelineCompareOp getCompareOp() {
+    return compareOp;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("%s (%s %s)",
+        this.getClass().getSimpleName(), this.compareOp.name(), this.value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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
index d4b4045..5e84976 100644
--- 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
@@ -39,13 +39,25 @@ public abstract class TimelineFilter {
      */
     LIST,
     /**
-     * Filter which is used for comparison.
+     * Filter which is used for key-value comparison.
      */
     COMPARE,
     /**
+     * Filter which is used for checking key-value equality.
+     */
+    KEY_VALUE,
+    /**
+     * Filter which is used for checking key-multiple values equality.
+     */
+    KEY_VALUES,
+    /**
      * Filter which matches prefix for a config or a metric.
      */
-    PREFIX
+    PREFIX,
+    /**
+     * Filter which checks existence of a value.
+     */
+    EXISTS
   }
 
   public abstract TimelineFilterType getFilterType();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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
index 8727bd7..57b41a6 100644
--- 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
@@ -53,6 +53,14 @@ public class TimelineFilterList extends TimelineFilter {
     this(Operator.AND, filters);
   }
 
+  public TimelineFilterList() {
+    this(Operator.AND);
+  }
+
+  public TimelineFilterList(Operator op) {
+    this.operator = op;
+  }
+
   public TimelineFilterList(Operator op, TimelineFilter...filters) {
     this.operator = op;
     this.filterList = new ArrayList<TimelineFilter>(Arrays.asList(filters));
@@ -88,4 +96,10 @@ public class TimelineFilterList extends TimelineFilter {
   public void addFilter(TimelineFilter filter) {
     filterList.add(filter);
   }
+
+  @Override
+  public String toString() {
+    return String.format("TimelineFilterList %s (%d): %s",
+        this.operator, this.filterList.size(), this.filterList.toString());
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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
index f902500..8cae410 100644
--- 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
@@ -18,25 +18,40 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.reader.filter;
 
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+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.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.Column;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix;
+
 import org.apache.hadoop.hbase.filter.QualifierFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 
 /**
  * Set of utility methods used by timeline filter classes.
  */
 public final class TimelineFilterUtils {
 
+  private static final Log LOG = LogFactory.getLog(TimelineFilterUtils.class);
+
   private TimelineFilterUtils() {
   }
 
   /**
    * Returns the equivalent HBase filter list's {@link Operator}.
-   * @param op
+   *
+   * @param op timeline filter list operator.
    * @return HBase filter list's Operator.
    */
   private static Operator getHBaseOperator(TimelineFilterList.Operator op) {
@@ -52,7 +67,8 @@ public final class TimelineFilterUtils {
 
   /**
    * Returns the equivalent HBase compare filter's {@link CompareOp}.
-   * @param op
+   *
+   * @param op timeline compare op.
    * @return HBase compare filter's CompareOp.
    */
   private static CompareOp getHBaseCompareOp(
@@ -90,6 +106,159 @@ public final class TimelineFilterUtils {
   }
 
   /**
+   * Create a HBase {@link QualifierFilter} for the passed column prefix and
+   * compare op.
+   *
+   * @param <T> Describes the type of column prefix.
+   * @param compareOp compare op.
+   * @param columnPrefix column prefix.
+   * @return a column qualifier filter.
+   */
+  public static <T> Filter createHBaseQualifierFilter(CompareOp compareOp,
+      ColumnPrefix<T> columnPrefix) {
+    return new QualifierFilter(compareOp,
+        new BinaryPrefixComparator(
+            columnPrefix.getColumnPrefixBytes("")));
+  }
+
+  /**
+   * Create filters for confs or metrics to retrieve. This list includes a
+   * configs/metrics family filter and relevant filters for confs/metrics to
+   * retrieve, if present.
+   *
+   * @param <T> Describes the type of column prefix.
+   * @param confsOrMetricToRetrieve configs/metrics to retrieve.
+   * @param columnFamily config or metric column family.
+   * @param columnPrefix config or metric column prefix.
+   * @return a filter list.
+   * @throws IOException if any problem occurs while creating the filters.
+   */
+  public static <T> Filter createFilterForConfsOrMetricsToRetrieve(
+      TimelineFilterList confsOrMetricToRetrieve, ColumnFamily<T> columnFamily,
+      ColumnPrefix<T> columnPrefix) throws IOException {
+    Filter familyFilter = new FamilyFilter(CompareOp.EQUAL,
+        new BinaryComparator(columnFamily.getBytes()));
+    if (confsOrMetricToRetrieve != null &&
+        !confsOrMetricToRetrieve.getFilterList().isEmpty()) {
+      // If confsOrMetricsToRetrive are specified, create a filter list based
+      // on it and family filter.
+      FilterList filter = new FilterList(familyFilter);
+      filter.addFilter(
+          createHBaseFilterList(columnPrefix, confsOrMetricToRetrieve));
+      return filter;
+    } else {
+      // Only the family filter needs to be added.
+      return familyFilter;
+    }
+  }
+
+  /**
+   * Create 2 HBase {@link SingleColumnValueFilter} filters for the specified
+   * value range represented by start and end value and wraps them inside a
+   * filter list. Start and end value should not be null.
+   *
+   * @param <T> Describes the type of column prefix.
+   * @param column Column for which single column value filter is to be created.
+   * @param startValue Start value.
+   * @param endValue End value.
+   * @return 2 single column value filters wrapped in a filter list.
+   * @throws IOException if any problem is encountered while encoding value.
+   */
+  public static <T> FilterList createSingleColValueFiltersByRange(
+      Column<T> column, Object startValue, Object endValue) throws IOException {
+    FilterList list = new FilterList();
+    Filter singleColValFilterStart = createHBaseSingleColValueFilter(
+        column.getColumnFamilyBytes(), column.getColumnQualifierBytes(),
+        column.getValueConverter().encodeValue(startValue),
+        CompareOp.GREATER_OR_EQUAL, true);
+    list.addFilter(singleColValFilterStart);
+
+    Filter singleColValFilterEnd = createHBaseSingleColValueFilter(
+        column.getColumnFamilyBytes(), column.getColumnQualifierBytes(),
+        column.getValueConverter().encodeValue(endValue),
+        CompareOp.LESS_OR_EQUAL, true);
+    list.addFilter(singleColValFilterEnd);
+    return list;
+  }
+
+  /**
+   * Creates a HBase {@link SingleColumnValueFilter}.
+   *
+   * @param columnFamily Column Family represented as bytes.
+   * @param columnQualifier Column Qualifier represented as bytes.
+   * @param value Value.
+   * @param compareOp Compare operator.
+   * @param filterIfMissing This flag decides if we should filter the row if the
+   *     specified column is missing. This is based on the filter's keyMustExist
+   *     field.
+   * @return a {@link SingleColumnValueFilter} object
+   * @throws IOException
+   */
+  private static SingleColumnValueFilter createHBaseSingleColValueFilter(
+      byte[] columnFamily, byte[] columnQualifier, byte[] value,
+      CompareOp compareOp, boolean filterIfMissing) throws IOException {
+    SingleColumnValueFilter singleColValFilter =
+        new SingleColumnValueFilter(columnFamily, columnQualifier, compareOp,
+        new BinaryComparator(value));
+    singleColValFilter.setLatestVersionOnly(true);
+    singleColValFilter.setFilterIfMissing(filterIfMissing);
+    return singleColValFilter;
+  }
+
+  /**
+   * Create a filter list of qualifier filters based on passed set of columns.
+   *
+   * @param <T> Describes the type of column prefix.
+   * @param colPrefix Column Prefix.
+   * @param columns set of column qualifiers.
+   * @return filter list.
+   */
+  public static <T> FilterList createFiltersFromColumnQualifiers(
+      ColumnPrefix<T> colPrefix, Set<String> columns) {
+    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
+    for (String column : columns) {
+      // For columns which have compound column qualifiers (eg. events), we need
+      // to include the required separator.
+      byte[] compoundColQual =
+          colPrefix.getCompoundColQualBytes(column, (byte[])null);
+      list.addFilter(new QualifierFilter(CompareOp.EQUAL,
+          new BinaryPrefixComparator(
+              colPrefix.getColumnPrefixBytes(compoundColQual))));
+    }
+    return list;
+  }
+
+  /**
+   * Fetch columns from filter list containing exists and multivalue equality
+   * filters. This is done to fetch only required columns from back-end and
+   * then match event filters or relationships in reader.
+   *
+   * @param filterList filter list.
+   * @return set of columns.
+   */
+  public static Set<String> fetchColumnsFromFilterList(
+      TimelineFilterList filterList) {
+    Set<String> strSet = new HashSet<String>();
+    for (TimelineFilter filter : filterList.getFilterList()) {
+      switch(filter.getFilterType()) {
+      case LIST:
+        strSet.addAll(fetchColumnsFromFilterList((TimelineFilterList)filter));
+        break;
+      case KEY_VALUES:
+        strSet.add(((TimelineKeyValuesFilter)filter).getKey());
+        break;
+      case EXISTS:
+        strSet.add(((TimelineExistsFilter)filter).getValue());
+        break;
+      default:
+        LOG.info("Unexpected filter type " + filter.getFilterType());
+        break;
+      }
+    }
+    return strSet;
+  }
+
+  /**
    * Creates equivalent HBase {@link FilterList} from {@link TimelineFilterList}
    * while converting different timeline filters(of type {@link TimelineFilter})
    * into their equivalent HBase filters.
@@ -98,22 +267,45 @@ public final class TimelineFilterUtils {
    * @param colPrefix column prefix which will be used for conversion.
    * @param filterList timeline filter list which has to be converted.
    * @return A {@link FilterList} object.
+   * @throws IOException if any problem occurs while creating the filter list.
    */
   public static <T> FilterList createHBaseFilterList(ColumnPrefix<T> colPrefix,
-      TimelineFilterList filterList) {
+      TimelineFilterList filterList) throws IOException {
     FilterList list =
         new FilterList(getHBaseOperator(filterList.getOperator()));
     for (TimelineFilter filter : filterList.getFilterList()) {
       switch(filter.getFilterType()) {
       case LIST:
-        list.addFilter(
-            createHBaseFilterList(colPrefix, (TimelineFilterList)filter));
+        list.addFilter(createHBaseFilterList(colPrefix,
+            (TimelineFilterList)filter));
         break;
       case PREFIX:
-        list.addFilter(createHBaseColQualPrefixFilter(
-            colPrefix, (TimelinePrefixFilter)filter));
+        list.addFilter(createHBaseColQualPrefixFilter(colPrefix,
+            (TimelinePrefixFilter)filter));
+        break;
+      case COMPARE:
+        TimelineCompareFilter compareFilter = (TimelineCompareFilter)filter;
+        list.addFilter(
+            createHBaseSingleColValueFilter(
+                colPrefix.getColumnFamilyBytes(),
+                colPrefix.getColumnPrefixBytes(compareFilter.getKey()),
+                colPrefix.getValueConverter().
+                    encodeValue(compareFilter.getValue()),
+                getHBaseCompareOp(compareFilter.getCompareOp()),
+                compareFilter.getKeyMustExist()));
+        break;
+      case KEY_VALUE:
+        TimelineKeyValueFilter kvFilter = (TimelineKeyValueFilter)filter;
+        list.addFilter(
+            createHBaseSingleColValueFilter(
+                colPrefix.getColumnFamilyBytes(),
+                colPrefix.getColumnPrefixBytes(kvFilter.getKey()),
+                colPrefix.getValueConverter().encodeValue(kvFilter.getValue()),
+                getHBaseCompareOp(kvFilter.getCompareOp()),
+                kvFilter.getKeyMustExist()));
         break;
       default:
+        LOG.info("Unexpected filter type " + filter.getFilterType());
         break;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java
----------------------------------------------------------------------
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/TimelineKeyValueFilter.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/TimelineKeyValueFilter.java
new file mode 100644
index 0000000..58f0ee9
--- /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/TimelineKeyValueFilter.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.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
+ * being equal or not to the values in back-end store.
+ */
+@Private
+@Unstable
+public class TimelineKeyValueFilter extends TimelineCompareFilter {
+  public TimelineKeyValueFilter(TimelineCompareOp op, String key, Object val,
+      boolean keyMustExistFlag) {
+    super(op, key, val, keyMustExistFlag);
+    if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
+      throw new IllegalArgumentException("TimelineCompareOp for equality"
+          + " filter should be EQUAL or NOT_EQUAL");
+    }
+  }
+
+  public TimelineKeyValueFilter(TimelineCompareOp op, String key, Object val) {
+    this(op, key, val, true);
+  }
+
+  @Override
+  public TimelineFilterType getFilterType() {
+    return TimelineFilterType.KEY_VALUE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java
----------------------------------------------------------------------
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/TimelineKeyValuesFilter.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/TimelineKeyValuesFilter.java
new file mode 100644
index 0000000..0d34d47
--- /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/TimelineKeyValuesFilter.java
@@ -0,0 +1,71 @@
+/**
+ * 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.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Filter class which represents filter to be applied based on multiple values
+ * for a key and these values being equal or not equal to values in back-end
+ * store.
+ */
+@Private
+@Unstable
+public class TimelineKeyValuesFilter extends TimelineFilter {
+  private final TimelineCompareOp compareOp;
+  private final String key;
+  private final Set<Object> values;
+  public TimelineKeyValuesFilter(TimelineCompareOp op, String key,
+      Set<Object> values) {
+    if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
+      throw new IllegalArgumentException("TimelineCompareOp for multi value "
+          + "equality filter should be EQUAL or NOT_EQUAL");
+    }
+    this.compareOp = op;
+    this.key = key;
+    this.values = values;
+  }
+
+  @Override
+  public TimelineFilterType getFilterType() {
+    return TimelineFilterType.KEY_VALUES;
+  }
+
+  public String getKey() {
+    return key;
+  }
+
+  public Set<Object> getValues() {
+    return values;
+  }
+
+  public TimelineCompareOp getCompareOp() {
+    return compareOp;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("%s (%s, %s:%s)",
+        this.getClass().getSimpleName(), this.compareOp.name(),
+        this.key, (values == null) ? "" : values.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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
index 6233f26..f36e593 100644
--- 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
@@ -53,4 +53,10 @@ public class TimelinePrefixFilter extends TimelineFilter {
   public TimelineCompareOp getCompareOp() {
     return compareOp;
   }
+
+  @Override
+  public String toString() {
+    return String.format("%s (%s %s)",
+        this.getClass().getSimpleName(), this.compareOp.name(), this.prefix);
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 97e05dd..bdddd7e 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
@@ -296,39 +296,39 @@ public class FileSystemTimelineReaderImpl extends AbstractService
           continue;
         }
         if (filters.getRelatesTo() != null &&
-            !filters.getRelatesTo().isEmpty() &&
-            !TimelineStorageUtils.matchRelations(
-            entity.getRelatesToEntities(), filters.getRelatesTo())) {
+            !filters.getRelatesTo().getFilterList().isEmpty() &&
+            !TimelineStorageUtils.matchRelatesTo(entity,
+            filters.getRelatesTo())) {
           continue;
         }
         if (filters.getIsRelatedTo()  != null &&
-            !filters.getIsRelatedTo().isEmpty() &&
-            !TimelineStorageUtils.matchRelations(
-            entity.getIsRelatedToEntities(), filters.getIsRelatedTo())) {
+            !filters.getIsRelatedTo().getFilterList().isEmpty() &&
+            !TimelineStorageUtils.matchIsRelatedTo(entity,
+            filters.getIsRelatedTo())) {
           continue;
         }
         if (filters.getInfoFilters() != null &&
-            !filters.getInfoFilters().isEmpty() &&
-            !TimelineStorageUtils.matchFilters(
-            entity.getInfo(), filters.getInfoFilters())) {
+            !filters.getInfoFilters().getFilterList().isEmpty() &&
+            !TimelineStorageUtils.matchInfoFilters(entity,
+            filters.getInfoFilters())) {
           continue;
         }
         if (filters.getConfigFilters() != null &&
-            !filters.getConfigFilters().isEmpty() &&
-            !TimelineStorageUtils.matchFilters(
-            entity.getConfigs(), filters.getConfigFilters())) {
+            !filters.getConfigFilters().getFilterList().isEmpty() &&
+            !TimelineStorageUtils.matchConfigFilters(entity,
+            filters.getConfigFilters())) {
           continue;
         }
         if (filters.getMetricFilters() != null &&
-            !filters.getMetricFilters().isEmpty() &&
-            !TimelineStorageUtils.matchMetricFilters(
-            entity.getMetrics(), filters.getMetricFilters())) {
+            !filters.getMetricFilters().getFilterList().isEmpty() &&
+            !TimelineStorageUtils.matchMetricFilters(entity,
+            filters.getMetricFilters())) {
           continue;
         }
         if (filters.getEventFilters() != null &&
-            !filters.getEventFilters().isEmpty() &&
-            !TimelineStorageUtils.matchEventFilters(
-            entity.getEvents(), filters.getEventFilters())) {
+            !filters.getEventFilters().getFilterList().isEmpty() &&
+            !TimelineStorageUtils.matchEventFilters(entity,
+            filters.getEventFilters())) {
           continue;
         }
         TimelineEntity entityToBeReturned = createEntityToBeReturned(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 b75007d..172f982 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
@@ -407,36 +407,39 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
                   "! Using the current timestamp");
               eventTimestamp = System.currentTimeMillis();
             }
-            byte[] columnQualifierFirst =
-                Bytes.toBytes(Separator.VALUES.encode(eventId));
-            byte[] columnQualifierWithTsBytes = Separator.VALUES.
-                join(columnQualifierFirst, Bytes.toBytes(
-                    TimelineStorageUtils.invertLong(eventTimestamp)));
+            byte[] eventTs =
+                Bytes.toBytes(TimelineStorageUtils.invertLong(eventTimestamp));
             Map<String, Object> eventInfo = event.getInfo();
             if ((eventInfo == null) || (eventInfo.size() == 0)) {
-              // add separator since event key is empty
-              byte[] compoundColumnQualifierBytes =
-                  Separator.VALUES.join(columnQualifierWithTsBytes,
-                      null);
               if (isApplication) {
+                byte[] compoundColumnQualifierBytes =
+                    ApplicationColumnPrefix.EVENT.
+                        getCompoundColQualBytes(eventId, eventTs, null);
                 ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
                     compoundColumnQualifierBytes, null,
-                      TimelineStorageUtils.EMPTY_BYTES);
+                    TimelineStorageUtils.EMPTY_BYTES);
               } else {
+                byte[] compoundColumnQualifierBytes =
+                    EntityColumnPrefix.EVENT.
+                        getCompoundColQualBytes(eventId, eventTs, null);
                 EntityColumnPrefix.EVENT.store(rowKey, entityTable,
                     compoundColumnQualifierBytes, null,
-                      TimelineStorageUtils.EMPTY_BYTES);
+                    TimelineStorageUtils.EMPTY_BYTES);
               }
             } else {
               for (Map.Entry<String, Object> info : eventInfo.entrySet()) {
                 // eventId?infoKey
-                byte[] compoundColumnQualifierBytes =
-                    Separator.VALUES.join(columnQualifierWithTsBytes,
-                        Bytes.toBytes(info.getKey()));
+                byte[] infoKey = Bytes.toBytes(info.getKey());
                 if (isApplication) {
+                  byte[] compoundColumnQualifierBytes =
+                      ApplicationColumnPrefix.EVENT.
+                          getCompoundColQualBytes(eventId, eventTs, infoKey);
                   ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
                       compoundColumnQualifierBytes, null, info.getValue());
                 } else {
+                  byte[] compoundColumnQualifierBytes =
+                      EntityColumnPrefix.EVENT.
+                          getCompoundColQualBytes(eventId, eventTs, infoKey);
                   EntityColumnPrefix.EVENT.store(rowKey, entityTable,
                       compoundColumnQualifierBytes, null, info.getValue());
                 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 5734389..80fcf8c 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
@@ -24,8 +24,11 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
 /**
@@ -41,7 +44,8 @@ public enum ApplicationColumn implements Column<ApplicationTable> {
   /**
    * When the application was created.
    */
-  CREATED_TIME(ApplicationColumnFamily.INFO, "created_time"),
+  CREATED_TIME(ApplicationColumnFamily.INFO, "created_time",
+      LongConverter.getInstance()),
 
   /**
    * The version of the flow that this app belongs to.
@@ -55,12 +59,17 @@ public enum ApplicationColumn implements Column<ApplicationTable> {
 
   private ApplicationColumn(ColumnFamily<ApplicationTable> columnFamily,
       String columnQualifier) {
+    this(columnFamily, columnQualifier, GenericConverter.getInstance());
+  }
+
+  private ApplicationColumn(ColumnFamily<ApplicationTable> columnFamily,
+      String columnQualifier, ValueConverter converter) {
     this.columnFamily = columnFamily;
     this.columnQualifier = columnQualifier;
     // Future-proof by ensuring the right column prefix hygiene.
     this.columnQualifierBytes =
         Bytes.toBytes(Separator.SPACE.encode(columnQualifier));
-    this.column = new ColumnHelper<ApplicationTable>(columnFamily);
+    this.column = new ColumnHelper<ApplicationTable>(columnFamily, converter);
   }
 
   /**
@@ -81,6 +90,21 @@ public enum ApplicationColumn implements Column<ApplicationTable> {
     return column.readResult(result, columnQualifierBytes);
   }
 
+  @Override
+  public byte[] getColumnQualifierBytes() {
+    return columnQualifierBytes.clone();
+  }
+
+  @Override
+  public byte[] getColumnFamilyBytes() {
+    return columnFamily.getBytes();
+  }
+
+  @Override
+  public ValueConverter getValueConverter() {
+    return column.getValueConverter();
+  }
+
   /**
    * Retrieve an {@link ApplicationColumn} given a name, or null if there is no
    * match. The following holds true: {@code columnFor(x) == columnFor(y)} if

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 9120f3d..1dfc4db 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
@@ -56,7 +56,7 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
   /**
    * Lifecycle events for an application.
    */
-  EVENT(ApplicationColumnFamily.INFO, "e"),
+  EVENT(ApplicationColumnFamily.INFO, "e", true),
 
   /**
    * Config column stores configuration with config key as the column name.
@@ -78,6 +78,7 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
    */
   private final String columnPrefix;
   private final byte[] columnPrefixBytes;
+  private final boolean compoundColQual;
 
   /**
    * Private constructor, meant to be used by the enum definition.
@@ -87,7 +88,18 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
    */
   private ApplicationColumnPrefix(ColumnFamily<ApplicationTable> columnFamily,
       String columnPrefix) {
-    this(columnFamily, columnPrefix, GenericConverter.getInstance());
+    this(columnFamily, columnPrefix, false, GenericConverter.getInstance());
+  }
+
+  private ApplicationColumnPrefix(ColumnFamily<ApplicationTable> columnFamily,
+      String columnPrefix, boolean compoundColQual) {
+    this(columnFamily, columnPrefix, compoundColQual,
+        GenericConverter.getInstance());
+  }
+
+  private ApplicationColumnPrefix(ColumnFamily<ApplicationTable> columnFamily,
+      String columnPrefix, ValueConverter converter) {
+    this(columnFamily, columnPrefix, false, converter);
   }
 
   /**
@@ -99,7 +111,7 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
    * this column prefix.
    */
   private ApplicationColumnPrefix(ColumnFamily<ApplicationTable> columnFamily,
-      String columnPrefix, ValueConverter converter) {
+      String columnPrefix, boolean compoundColQual, ValueConverter converter) {
     column = new ColumnHelper<ApplicationTable>(columnFamily, converter);
     this.columnFamily = columnFamily;
     this.columnPrefix = columnPrefix;
@@ -110,6 +122,7 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
       this.columnPrefixBytes =
           Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
     }
+    this.compoundColQual = compoundColQual;
   }
 
   /**
@@ -131,6 +144,20 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
         this.columnPrefixBytes, qualifierPrefix);
   }
 
+  @Override
+  public byte[] getColumnFamilyBytes() {
+    return columnFamily.getBytes();
+  }
+
+  @Override
+  public byte[] getCompoundColQualBytes(String qualifier,
+      byte[]...components) {
+    if (!compoundColQual) {
+      return ColumnHelper.getColumnQualifier(null, qualifier);
+    }
+    return ColumnHelper.getCompoundColumnQualifierBytes(qualifier, components);
+  }
+
   /*
    * (non-Javadoc)
    *
@@ -196,6 +223,10 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
     return column.readResult(result, columnQualifier);
   }
 
+  public ValueConverter getValueConverter() {
+    return column.getValueConverter();
+  }
+
   /*
    * (non-Javadoc)
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 a8e1c66..ff61633 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,6 +25,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.common.ValueConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
 import java.io.IOException;
@@ -71,6 +72,11 @@ public enum AppToFlowColumn implements Column<AppToFlowTable> {
     return columnQualifier;
   }
 
+  @Override
+  public byte[] getColumnQualifierBytes() {
+    return columnQualifierBytes.clone();
+  }
+
   public void store(byte[] rowKey,
       TypedBufferedMutator<AppToFlowTable> tableMutator, Long timestamp,
       Object inputValue, Attribute... attributes) throws IOException {
@@ -78,6 +84,16 @@ public enum AppToFlowColumn implements Column<AppToFlowTable> {
         inputValue, attributes);
   }
 
+  @Override
+  public byte[] getColumnFamilyBytes() {
+    return columnFamily.getBytes();
+  }
+
+  @Override
+  public ValueConverter getValueConverter() {
+    return column.getValueConverter();
+  }
+
   public Object readResult(Result result) throws IOException {
     return column.readResult(result, columnQualifierBytes);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 1f0b48f..90f2de4 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
@@ -60,4 +60,21 @@ public interface Column<T> {
    */
   Object readResult(Result result) throws IOException;
 
+  /**
+   * Returns column family name(as bytes) associated with this column.
+   * @return a byte array encoding column family for this column qualifier.
+   */
+  byte[] getColumnFamilyBytes();
+
+  /**
+   * Get byte representation for this column qualifier.
+   * @return a byte array representing column qualifier.
+   */
+  byte[] getColumnQualifierBytes();
+
+  /**
+   * Returns value converter implementation associated with this column.
+   * @return a {@link ValueConverter} implementation.
+   */
+  ValueConverter getValueConverter();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 15bb818..4adb413 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
@@ -363,6 +363,22 @@ public class ColumnHelper<T> {
   }
 
   /**
+   * Create a compound column qualifier by combining qualifier and components.
+   *
+   * @param qualifier Column QUalifier.
+   * @param components Other components.
+   * @return a byte array representing compound column qualifier.
+   */
+  public static byte[] getCompoundColumnQualifierBytes(String qualifier,
+      byte[]...components) {
+    byte[] colQualBytes = Bytes.toBytes(Separator.VALUES.encode(qualifier));
+    for (int i = 0; i < components.length; i++) {
+      colQualBytes = Separator.VALUES.join(colQualBytes, components[i]);
+    }
+    return colQualBytes;
+  }
+
+  /**
    * @param columnPrefixBytes The byte representation for the column prefix.
    *          Should not contain {@link Separator#QUALIFIERS}.
    * @param qualifier for the remainder of the column.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 f221b31..e4b7f16 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
@@ -112,6 +112,18 @@ public interface ColumnPrefix<T> {
       readResultsWithTimestamps(Result result) throws IOException;
 
   /**
+   * @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 if any problem occurs while reading results.
+   */
+  Map<?, Object> readResultsHavingCompoundColumnQualifiers(Result result)
+      throws IOException;
+
+  /**
    * @param qualifierPrefix Column qualifier or prefix of qualifier.
    * @return a byte array encoding column prefix and qualifier/prefix passed.
    */
@@ -122,4 +134,27 @@ public interface ColumnPrefix<T> {
    * @return a byte array encoding column prefix and qualifier/prefix passed.
    */
   byte[] getColumnPrefixBytes(byte[] qualifierPrefix);
+
+  /**
+   * Returns column family name(as bytes) associated with this column prefix.
+   * @return a byte array encoding column family for this prefix.
+   */
+  byte[] getColumnFamilyBytes();
+
+  /**
+   * Returns value converter implementation associated with this column prefix.
+   * @return a {@link ValueConverter} implementation.
+   */
+  ValueConverter getValueConverter();
+
+  /**
+   * Get compound column qualifier bytes if the column qualifier is a compound
+   * qualifier. Returns the qualifier passed as bytes if the column is not a
+   * compound column qualifier.
+   *
+   * @param qualifier Column Qualifier.
+   * @param components Other components.
+   * @return byte array representing compound column qualifier.
+   */
+  byte[] getCompoundColQualBytes(String qualifier, byte[]...components);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java
----------------------------------------------------------------------
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/TimelineEntityFiltersType.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/TimelineEntityFiltersType.java
new file mode 100644
index 0000000..4099e92
--- /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/TimelineEntityFiltersType.java
@@ -0,0 +1,71 @@
+/**
+ * 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.server.timelineservice.reader.filter.TimelineFilter.TimelineFilterType;
+
+/**
+ * Used to define which filter to match.
+ */
+enum TimelineEntityFiltersType {
+  CONFIG {
+    boolean isValidFilter(TimelineFilterType filterType) {
+      return filterType == TimelineFilterType.LIST ||
+          filterType == TimelineFilterType.KEY_VALUE;
+    }
+  },
+  INFO {
+    boolean isValidFilter(TimelineFilterType filterType) {
+      return filterType == TimelineFilterType.LIST ||
+          filterType == TimelineFilterType.KEY_VALUE;
+    }
+  },
+  METRIC {
+    boolean isValidFilter(TimelineFilterType filterType) {
+      return filterType == TimelineFilterType.LIST ||
+          filterType == TimelineFilterType.COMPARE;
+    }
+  },
+  EVENT {
+    boolean isValidFilter(TimelineFilterType filterType) {
+      return filterType == TimelineFilterType.LIST ||
+          filterType == TimelineFilterType.EXISTS;
+    }
+  },
+  IS_RELATED_TO {
+    boolean isValidFilter(TimelineFilterType filterType) {
+      return filterType == TimelineFilterType.LIST ||
+          filterType == TimelineFilterType.KEY_VALUES;
+    }
+  },
+  RELATES_TO {
+    boolean isValidFilter(TimelineFilterType filterType) {
+      return filterType == TimelineFilterType.LIST ||
+          filterType == TimelineFilterType.KEY_VALUES;
+    }
+  };
+
+  /**
+   * Checks whether filter type is valid for the filter being matched.
+   *
+   * @param filterType filter type.
+   * @return true, if its a valid filter, false otherwise.
+   */
+  abstract boolean isValidFilter(TimelineFilterType filterType);
+}
\ No newline at end of file


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


[23/50] [abbrv] hadoop git commit: YARN-5045. hbase unit tests fail due to dependency issues. (Sangjin Lee via varunsaxena)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
new file mode 100644
index 0000000..ca80ed5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -0,0 +1,2008 @@
+/**
+ * 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.text.DateFormat;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+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.server.timelineservice.storage.common.TimelineStorageUtils;
+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.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.ClientResponse.Status;
+import com.sun.jersey.api.client.GenericType;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+
+public class TestTimelineReaderWebServicesHBaseStorage {
+  private int serverPort;
+  private TimelineReaderServer server;
+  private static HBaseTestingUtility util;
+  private static long ts = System.currentTimeMillis();
+  private static long dayTs =
+      TimelineStorageUtils.getTopOfTheDayTimestamp(ts);
+
+  @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 =
+        ImmutableMap.of(ts - 100000, (Number)2, ts - 80000, 40);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    m1 = new TimelineMetric();
+    m1.setId("MAP1_SLOT_MILLIS");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)2, ts - 80000, 40);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    m1 = new TimelineMetric();
+    m1.setId("HDFS_BYTES_READ");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)31, 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);
+    event.setTimestamp(cTime);
+    String expKey = "foo_event";
+    Object expVal = "test";
+    event.addInfo(expKey, expVal);
+    entity.addEvent(event);
+    TimelineEvent event11 = new TimelineEvent();
+    event11.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
+    Long expTs = 1425019501000L;
+    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(cTime);
+    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 = 1425016501037L;
+    entity3.setCreatedTime(cTime);
+    TimelineEvent event2 = new TimelineEvent();
+    event2.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+    event2.setTimestamp(cTime);
+    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(cTime);
+    event4.addInfo("foo_event", "test");
+    entity4.addEvent(event4);
+    metrics.clear();
+    m2 = new TimelineMetric();
+    m2.setId("MAP_SLOT_MILLIS");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)5L, ts - 80000, 101L);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues);
+    metrics.add(m2);
+    entity4.addMetrics(metrics);
+    te4.addEntity(entity4);
+
+    TimelineEntities te5 = new TimelineEntities();
+    TimelineEntity entity5 = new TimelineEntity();
+    entity5.setId("entity1");
+    entity5.setType("type1");
+    entity5.setCreatedTime(1425016501034L);
+    // add some config entries
+    entity5.addConfigs(ImmutableMap.of("config_param1", "value1",
+        "config_param2", "value2", "cfg_param1", "value3"));
+    entity5.addInfo(ImmutableMap.of("info1", (Object)"cluster1",
+        "info2", 2.0, "info3", 35000, "info4", 36000));
+    metrics = new HashSet<>();
+    m1 = new TimelineMetric();
+    m1.setId("MAP_SLOT_MILLIS");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)2, ts - 80000, 40);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    m1 = new TimelineMetric();
+    m1.setId("HDFS_BYTES_READ");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)31, ts - 80000, 57);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    entity5.addMetrics(metrics);
+    TimelineEvent event51 = new TimelineEvent();
+    event51.setId("event1");
+    event51.setTimestamp(cTime);
+    entity5.addEvent(event51);
+    TimelineEvent event52 = new TimelineEvent();
+    event52.setId("event2");
+    event52.setTimestamp(cTime);
+    entity5.addEvent(event52);
+    TimelineEvent event53 = new TimelineEvent();
+    event53.setId("event3");
+    event53.setTimestamp(cTime);
+    entity5.addEvent(event53);
+    TimelineEvent event54 = new TimelineEvent();
+    event54.setId("event4");
+    event54.setTimestamp(cTime);
+    entity5.addEvent(event54);
+    Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
+    isRelatedTo1.put("type2",
+        Sets.newHashSet("entity21","entity22","entity23","entity24"));
+    isRelatedTo1.put("type4", Sets.newHashSet("entity41","entity42"));
+    isRelatedTo1.put("type1", Sets.newHashSet("entity14","entity15"));
+    isRelatedTo1.put("type3",
+        Sets.newHashSet("entity31", "entity35", "entity32", "entity33"));
+    entity5.addIsRelatedToEntities(isRelatedTo1);
+    Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
+    relatesTo1.put("type2",
+        Sets.newHashSet("entity21","entity22","entity23","entity24"));
+    relatesTo1.put("type4", Sets.newHashSet("entity41","entity42"));
+    relatesTo1.put("type1", Sets.newHashSet("entity14","entity15"));
+    relatesTo1.put("type3",
+        Sets.newHashSet("entity31", "entity35", "entity32", "entity33"));
+    entity5.addRelatesToEntities(relatesTo1);
+    te5.addEntity(entity5);
+
+    TimelineEntity entity6 = new TimelineEntity();
+    entity6.setId("entity2");
+    entity6.setType("type1");
+    entity6.setCreatedTime(1425016501034L);
+    entity6.addConfigs(ImmutableMap.of("cfg_param3", "value1",
+        "configuration_param2", "value2", "config_param1", "value3"));
+    entity6.addInfo(ImmutableMap.of("info1", (Object)"cluster2",
+        "info2", 2.0, "info4", 35000));
+    metrics = new HashSet<>();
+    m1 = new TimelineMetric();
+    m1.setId("MAP1_SLOT_MILLIS");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)12, ts - 80000, 140);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    m1 = new TimelineMetric();
+    m1.setId("HDFS_BYTES_READ");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)78, ts - 80000, 157);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    m1 = new TimelineMetric();
+    m1.setId("MAP11_SLOT_MILLIS");
+    m1.setType(Type.SINGLE_VALUE);
+    m1.addValue(ts - 100000, 122);
+    metrics.add(m1);
+    entity6.addMetrics(metrics);
+    TimelineEvent event61 = new TimelineEvent();
+    event61.setId("event1");
+    event61.setTimestamp(cTime);
+    entity6.addEvent(event61);
+    TimelineEvent event62 = new TimelineEvent();
+    event62.setId("event5");
+    event62.setTimestamp(cTime);
+    entity6.addEvent(event62);
+    TimelineEvent event63 = new TimelineEvent();
+    event63.setId("event3");
+    event63.setTimestamp(cTime);
+    entity6.addEvent(event63);
+    TimelineEvent event64 = new TimelineEvent();
+    event64.setId("event6");
+    event64.setTimestamp(cTime);
+    entity6.addEvent(event64);
+    Map<String, Set<String>> isRelatedTo2 = new HashMap<String, Set<String>>();
+    isRelatedTo2.put("type2",
+        Sets.newHashSet("entity21","entity22","entity23","entity24"));
+    isRelatedTo2.put("type5", Sets.newHashSet("entity51","entity52"));
+    isRelatedTo2.put("type6", Sets.newHashSet("entity61","entity66"));
+    isRelatedTo2.put("type3", Sets.newHashSet("entity31"));
+    entity6.addIsRelatedToEntities(isRelatedTo2);
+    Map<String, Set<String>> relatesTo2 = new HashMap<String, Set<String>>();
+    relatesTo2.put("type2",
+        Sets.newHashSet("entity21","entity22","entity23","entity24"));
+    relatesTo2.put("type5", Sets.newHashSet("entity51","entity52"));
+    relatesTo2.put("type6", Sets.newHashSet("entity61","entity66"));
+    relatesTo2.put("type3", Sets.newHashSet("entity31"));
+    entity6.addRelatesToEntities(relatesTo2);
+    te5.addEntity(entity6);
+
+    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.write(cluster, user, flow, flowVersion, runid,
+          "application_1111111111_1111", te5);
+      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.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+      config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+      config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
+          "localhost:0");
+      config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
+      config.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
+          "org.apache.hadoop.yarn.server.timelineservice.storage." +
+              "HBaseTimelineReaderImpl");
+      config.setInt("hfile.format.version", 3);
+      server = new TimelineReaderServer();
+      server.init(config);
+      server.start();
+      serverPort = server.getWebServerPort();
+    } catch (Exception e) {
+      Assert.fail("Web server failed to start");
+    }
+  }
+
+  private static Client createClient() {
+    ClientConfig cfg = new DefaultClientConfig();
+    cfg.getClasses().add(YarnJacksonJaxbJsonProvider.class);
+    return new Client(new URLConnectionClientHandler(
+        new DummyURLConnectionFactory()), cfg);
+  }
+
+  private static ClientResponse getResponse(Client client, URI uri)
+      throws Exception {
+    ClientResponse resp =
+        client.resource(uri).accept(MediaType.APPLICATION_JSON)
+        .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
+    if (resp == null ||
+        resp.getClientResponseStatus() != ClientResponse.Status.OK) {
+      String msg = 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/clusters/cluster1/users/user1/flows/flow_name/runs/" +
+          "1002345678919");
+      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(3, 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);
+      TimelineMetric m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+          "MAP1_SLOT_MILLIS", ts - 80000, 40L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m1, m2, m3));
+      }
+
+      // Query without specifying cluster ID.
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/users/user1/flows/flow_name/runs/1002345678919");
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(FlowRunEntity.class);
+      assertNotNull(entity);
+      assertEquals("user1@flow_name/1002345678919", entity.getId());
+      assertEquals(3, 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);
+      m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+          "MAP1_SLOT_MILLIS", ts - 80000, 40L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m1, m2, m3));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlowRuns() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs");
+      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/" +
+          "clusters/cluster1/users/user1/flows/flow_name/runs?limit=1");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      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/clusters/cluster1/users/user1/flows/flow_name/runs?" +
+          "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/clusters/cluster1/users/user1/flows/flow_name/runs?" +
+          "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/clusters/cluster1/users/user1/flows/flow_name/runs?" +
+          "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/clusters/cluster1/users/user1/flows/flow_name/runs?" +
+          "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() == 3)) ||
+            ((entity.getId().equals("user1@flow_name/1002345678920")) &&
+            (entity.getRunId() == 1002345678920L) &&
+            (entity.getStartTime() == 1425016501034L) &&
+            (entity.getMetrics().size() == 1)));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+
+  @Test
+  public void testGetFlowRunsMetricsToRetrieve() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
+          "metricstoretrieve=MAP_,HDFS_");
+      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());
+      int metricCnt = 0;
+      for (FlowRunEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP_") ||
+              metric.getId().startsWith("HDFS_"));
+        }
+      }
+      assertEquals(3, metricCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
+          "metricstoretrieve=!(MAP_,HDFS_)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      metricCnt = 0;
+      for (FlowRunEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP1_"));
+        }
+      }
+      assertEquals(1, metricCnt);
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesByUID() throws Exception {
+    Client client = createClient();
+    try {
+      // Query all flows.
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flows");
+      ClientResponse resp = getResponse(client, uri);
+      Set<FlowActivityEntity> flowEntities =
+          resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertNotNull(flowEntities);
+      assertEquals(2, flowEntities.size());
+      List<String> listFlowUIDs = new ArrayList<String>();
+      for (FlowActivityEntity entity : flowEntities) {
+        String flowUID =
+            (String)entity.getInfo().get(TimelineReaderManager.UID_KEY);
+        listFlowUIDs.add(flowUID);
+        assertEquals(TimelineUIDConverter.FLOW_UID.encodeUID(
+            new TimelineReaderContext(entity.getCluster(), entity.getUser(),
+            entity.getFlowName(), null, null, null, null)), flowUID);
+        assertTrue((entity.getId().endsWith("@flow_name") &&
+            entity.getFlowRuns().size() == 2) ||
+            (entity.getId().endsWith("@flow_name2") &&
+            entity.getFlowRuns().size() == 1));
+      }
+
+      // Query flowruns based on UID returned in query above.
+      List<String> listFlowRunUIDs = new ArrayList<String>();
+      for (String flowUID : listFlowUIDs) {
+        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+            "timeline/flow-uid/" + flowUID + "/runs");
+        resp = getResponse(client, uri);
+        Set<FlowRunEntity> frEntities =
+            resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
+        assertNotNull(frEntities);
+        for (FlowRunEntity entity : frEntities) {
+          String flowRunUID =
+              (String)entity.getInfo().get(TimelineReaderManager.UID_KEY);
+          listFlowRunUIDs.add(flowRunUID);
+          assertEquals(TimelineUIDConverter.FLOWRUN_UID.encodeUID(
+              new TimelineReaderContext("cluster1", entity.getUser(),
+              entity.getName(), entity.getRunId(), null, null, null)),
+              flowRunUID);
+        }
+      }
+      assertEquals(3, listFlowRunUIDs.size());
+
+      // Query single flowrun based on UIDs' returned in query to get flowruns.
+      for (String flowRunUID : listFlowRunUIDs) {
+        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+            "timeline/run-uid/" + flowRunUID);
+        resp = getResponse(client, uri);
+        FlowRunEntity entity = resp.getEntity(FlowRunEntity.class);
+        assertNotNull(entity);
+      }
+
+      // Query apps based on UIDs' returned in query to get flowruns.
+      List<String> listAppUIDs = new ArrayList<String>();
+      for (String flowRunUID : listFlowRunUIDs) {
+        TimelineReaderContext context =
+            TimelineUIDConverter.FLOWRUN_UID.decodeUID(flowRunUID);
+        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+            "timeline/run-uid/" + flowRunUID + "/apps");
+        resp = getResponse(client, uri);
+        Set<TimelineEntity> appEntities =
+            resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+        assertNotNull(appEntities);
+        for (TimelineEntity entity : appEntities) {
+          String appUID =
+              (String)entity.getInfo().get(TimelineReaderManager.UID_KEY);
+          listAppUIDs.add(appUID);
+          assertEquals(TimelineUIDConverter.APPLICATION_UID.encodeUID(
+              new TimelineReaderContext(context.getClusterId(),
+              context.getUserId(), context.getFlowName(),
+              context.getFlowRunId(), entity.getId(), null, null)), appUID);
+        }
+      }
+      assertEquals(4, listAppUIDs.size());
+
+      // Query single app based on UIDs' returned in query to get apps.
+      for (String appUID : listAppUIDs) {
+        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+            "timeline/app-uid/" + appUID);
+        resp = getResponse(client, uri);
+        TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+        assertNotNull(entity);
+      }
+
+      // Query entities based on UIDs' returned in query to get apps and
+      // a specific entity type(in this case type1).
+      List<String> listEntityUIDs = new ArrayList<String>();
+      for (String appUID : listAppUIDs) {
+        TimelineReaderContext context =
+            TimelineUIDConverter.APPLICATION_UID.decodeUID(appUID);
+        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+            "timeline/app-uid/" + appUID + "/entities/type1");
+        resp = getResponse(client, uri);
+        Set<TimelineEntity> entities =
+            resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+        assertNotNull(entities);
+        for (TimelineEntity entity : entities) {
+          String entityUID =
+              (String)entity.getInfo().get(TimelineReaderManager.UID_KEY);
+          listEntityUIDs.add(entityUID);
+          assertEquals(TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(
+              new TimelineReaderContext(context.getClusterId(),
+              context.getUserId(), context.getFlowName(),
+              context.getFlowRunId(), context.getAppId(), "type1",
+              entity.getId())), entityUID);
+        }
+      }
+      assertEquals(2, listEntityUIDs.size());
+
+      // Query single entity based on UIDs' returned in query to get entities.
+      for (String entityUID : listEntityUIDs) {
+        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+            "timeline/entity-uid/" + entityUID);
+        resp = getResponse(client, uri);
+        TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+        assertNotNull(entity);
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/flow-uid/dummy:flow/runs");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/run-uid/dummy:flowrun");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      // Run Id is not a numerical value.
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/run-uid/some:dummy:flow:123v456");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/run-uid/dummy:flowrun/apps");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/app-uid/dummy:app");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/app-uid/dummy:app/entities/type1");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/entity-uid/dummy:entity");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testUIDQueryWithAndWithoutFlowContextInfo() throws Exception {
+    Client client = createClient();
+    try {
+      String appUIDWithFlowInfo =
+          "cluster1!user1!flow_name!1002345678919!application_1111111111_1111";
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/"+
+          "timeline/app-uid/" + appUIDWithFlowInfo);
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity appEntity1 = resp.getEntity(TimelineEntity.class);
+      assertNotNull(appEntity1);
+      assertEquals(
+          TimelineEntityType.YARN_APPLICATION.toString(), appEntity1.getType());
+      assertEquals("application_1111111111_1111", appEntity1.getId());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "app-uid/" + appUIDWithFlowInfo + "/entities/type1");
+      resp = getResponse(client, uri);
+      Set<TimelineEntity> entities1 =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities1);
+      assertEquals(2, entities1.size());
+      for (TimelineEntity entity : entities1) {
+        assertNotNull(entity.getInfo());
+        assertEquals(1, entity.getInfo().size());
+        String uid =
+            (String) entity.getInfo().get(TimelineReaderManager.UID_KEY);
+        assertNotNull(uid);
+        assertTrue(uid.equals(appUIDWithFlowInfo + "!type1!entity1") ||
+            uid.equals(appUIDWithFlowInfo + "!type1!entity2"));
+      }
+
+      String appUIDWithoutFlowInfo = "cluster1!application_1111111111_1111";
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+
+          "app-uid/" + appUIDWithoutFlowInfo);
+      resp = getResponse(client, uri);;
+      TimelineEntity appEntity2 = resp.getEntity(TimelineEntity.class);
+      assertNotNull(appEntity2);
+      assertEquals(
+          TimelineEntityType.YARN_APPLICATION.toString(), appEntity2.getType());
+      assertEquals("application_1111111111_1111", appEntity2.getId());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "app-uid/" + appUIDWithoutFlowInfo + "/entities/type1");
+      resp = getResponse(client, uri);
+      Set<TimelineEntity> entities2 =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities2);
+      assertEquals(2, entities2.size());
+      for (TimelineEntity entity : entities2) {
+        assertNotNull(entity.getInfo());
+        assertEquals(1, entity.getInfo().size());
+        String uid =
+            (String) entity.getInfo().get(TimelineReaderManager.UID_KEY);
+        assertNotNull(uid);
+        assertTrue(uid.equals(appUIDWithoutFlowInfo + "!type1!entity1") ||
+            uid.equals(appUIDWithoutFlowInfo + "!type1!entity2"));
+      }
+
+      String entityUIDWithFlowInfo = appUIDWithFlowInfo + "!type1!entity1";
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+
+          "entity-uid/" + entityUIDWithFlowInfo);
+      resp = getResponse(client, uri);;
+      TimelineEntity singleEntity1 = resp.getEntity(TimelineEntity.class);
+      assertNotNull(singleEntity1);
+      assertEquals("type1", singleEntity1.getType());
+      assertEquals("entity1", singleEntity1.getId());
+
+      String entityUIDWithoutFlowInfo =
+          appUIDWithoutFlowInfo + "!type1!entity1";
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+
+          "entity-uid/" + entityUIDWithoutFlowInfo);
+      resp = getResponse(client, uri);;
+      TimelineEntity singleEntity2 = resp.getEntity(TimelineEntity.class);
+      assertNotNull(singleEntity2);
+      assertEquals("type1", singleEntity2.getType());
+      assertEquals("entity1", singleEntity2.getId());
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testUIDNotProperlyEscaped() throws Exception {
+    Client client = createClient();
+    try {
+      String appUID =
+          "cluster1!user*1!flow_name!1002345678919!application_1111111111_1111";
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/"+
+          "timeline/app-uid/" + appUID);
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlows() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/flows");
+      ClientResponse resp = getResponse(client, uri);
+      Set<FlowActivityEntity> entities =
+          resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      for (FlowActivityEntity entity : entities) {
+        assertTrue((entity.getId().endsWith("@flow_name") &&
+            entity.getFlowRuns().size() == 2) ||
+            (entity.getId().endsWith("@flow_name2") &&
+            entity.getFlowRuns().size() == 1));
+      }
+
+      // 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/clusters/cluster1/flows?limit=1");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+
+      long firstFlowActivity =
+          TimelineStorageUtils.getTopOfTheDayTimestamp(1425016501000L);
+
+      DateFormat fmt = TimelineReaderWebServices.DATE_FORMAT.get();
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/flows?daterange="
+          + fmt.format(firstFlowActivity) + "-"
+          + fmt.format(dayTs));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      for (FlowActivityEntity entity : entities) {
+        assertTrue((entity.getId().endsWith("@flow_name") &&
+            entity.getFlowRuns().size() == 2) ||
+            (entity.getId().endsWith("@flow_name2") &&
+            entity.getFlowRuns().size() == 1));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/flows?daterange=" +
+          fmt.format(dayTs + (4*86400000L)));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/flows?daterange=-" +
+          fmt.format(dayTs));
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/flows?daterange=" +
+           fmt.format(firstFlowActivity) + "-");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/flows?daterange=20150711:20150714");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/flows?daterange=20150714-20150711");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/flows?daterange=2015071129-20150712");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/flows?daterange=20150711-2015071243");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetApp() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111?" +
+          "userid=user1&fields=ALL&flowname=flow_name&flowrunid=1002345678919");
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("application_1111111111_1111", entity.getId());
+      assertEquals(3, 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);
+      TimelineMetric m3 = newMetric(TimelineMetric.Type.TIME_SERIES,
+          "MAP1_SLOT_MILLIS", ts - 100000, 2L);
+      m3.addValue(ts - 80000, 40L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m1, m2, m3));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+              "timeline/apps/application_1111111111_2222?userid=user1" +
+              "&fields=metrics&flowname=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 m4 = newMetric(TimelineMetric.Type.TIME_SERIES,
+         "MAP_SLOT_MILLIS", ts - 100000, 5L);
+      m4.addValue(ts - 80000, 101L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m4));
+      }
+    } finally {
+        client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetAppWithoutFlowInfo() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111?" +
+          "fields=ALL");
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("application_1111111111_1111", entity.getId());
+      assertEquals(3, 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);
+      TimelineMetric m3 = newMetric(TimelineMetric.Type.TIME_SERIES,
+          "MAP1_SLOT_MILLIS", ts - 100000, 2L);
+      m3.addValue(ts - 80000, 40L);
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue(verifyMetrics(metric, m1, m2, m3));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntityWithoutFlowInfo() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/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/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/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();
+    }
+  }
+
+  /**
+   * Tests if specific configs and metrics are retrieve for getEntities call.
+   */
+  @Test
+  public void testGetEntitiesDataToRetrieve() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?confstoretrieve=cfg_");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      int cfgCnt = 0;
+      for (TimelineEntity entity : entities) {
+        cfgCnt += entity.getConfigs().size();
+        for (String configKey : entity.getConfigs().keySet()) {
+          assertTrue(configKey.startsWith("cfg_"));
+        }
+      }
+      assertEquals(2, cfgCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?confstoretrieve=cfg_,config_");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      cfgCnt = 0;
+      for (TimelineEntity entity : entities) {
+        cfgCnt += entity.getConfigs().size();
+        for (String configKey : entity.getConfigs().keySet()) {
+          assertTrue(configKey.startsWith("cfg_") ||
+              configKey.startsWith("config_"));
+        }
+      }
+      assertEquals(5, cfgCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?confstoretrieve=!(cfg_,config_)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      cfgCnt = 0;
+      for (TimelineEntity entity : entities) {
+        cfgCnt += entity.getConfigs().size();
+        for (String configKey : entity.getConfigs().keySet()) {
+          assertTrue(configKey.startsWith("configuration_"));
+        }
+      }
+      assertEquals(1, cfgCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricstoretrieve=MAP_");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      int metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP_"));
+        }
+      }
+      assertEquals(1, metricCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricstoretrieve=MAP1_,HDFS_");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP1_") ||
+              metric.getId().startsWith("HDFS_"));
+        }
+      }
+      assertEquals(3, metricCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricstoretrieve=!(MAP1_,HDFS_)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP_") ||
+              metric.getId().startsWith("MAP11_"));
+        }
+      }
+      assertEquals(2, metricCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricstoretrieve=!(MAP1_,HDFS_");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesConfigFilters() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=config_param1%20eq%20value1%20OR%20" +
+          "config_param1%20eq%20value3");
+      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"));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=config_param1%20eq%20value1%20AND" +
+          "%20configuration_param2%20eq%20value2");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+
+      // conffilters=(config_param1 eq value1 AND configuration_param2 eq
+      // value2) OR (config_param1 eq value3 AND cfg_param3 eq value1)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" +
+          "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" +
+          "%20value3%20AND%20cfg_param3%20eq%20value1)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      int cfgCnt = 0;
+      for (TimelineEntity entity : entities) {
+        cfgCnt += entity.getConfigs().size();
+        assertTrue(entity.getId().equals("entity2"));
+      }
+      assertEquals(0, cfgCnt);
+
+      // conffilters=(config_param1 eq value1 AND configuration_param2 eq
+      // value2) OR (config_param1 eq value3 AND cfg_param3 eq value1)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" +
+          "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" +
+          "%20value3%20AND%20cfg_param3%20eq%20value1)&fields=CONFIGS");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      cfgCnt = 0;
+      for (TimelineEntity entity : entities) {
+        cfgCnt += entity.getConfigs().size();
+        assertTrue(entity.getId().equals("entity2"));
+      }
+      assertEquals(3, cfgCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" +
+          "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" +
+          "%20value3%20AND%20cfg_param3%20eq%20value1)&confstoretrieve=cfg_," +
+          "configuration_");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      cfgCnt = 0;
+      for (TimelineEntity entity : entities) {
+        cfgCnt += entity.getConfigs().size();
+        assertTrue(entity.getId().equals("entity2"));
+        for (String configKey : entity.getConfigs().keySet()) {
+          assertTrue(configKey.startsWith("cfg_") ||
+              configKey.startsWith("configuration_"));
+        }
+      }
+      assertEquals(2, cfgCnt);
+
+      // Test for behavior when compare op is ne(not equals) vs ene
+      // (exists and not equals). configuration_param2 does not exist for
+      // entity1. For ne, both entity1 and entity2 will be returned. For ene,
+      // only entity2 will be returned as we are checking for existence too.
+      // conffilters=configuration_param2 ne value3
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=configuration_param2%20ne%20value3");
+      resp = getResponse(client, uri);
+      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"));
+      }
+      // conffilters=configuration_param2 ene value3
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=configuration_param2%20ene%20value3");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity2"));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesInfoFilters() throws Exception {
+    Client client = createClient();
+    try {
+      // infofilters=info1 eq cluster1 OR info1 eq cluster2
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=info1%20eq%20cluster1%20OR%20info1%20eq" +
+          "%20cluster2");
+      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"));
+      }
+
+      // infofilters=info1 eq cluster1 AND info4 eq 35000
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=info1%20eq%20cluster1%20AND%20info4%20" +
+          "eq%2035000");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+
+      // infofilters=info4 eq 35000 OR info4 eq 36000
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=info4%20eq%2035000%20OR%20info4%20eq" +
+          "%2036000");
+      resp = getResponse(client, uri);
+      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"));
+      }
+
+      // infofilters=(info1 eq cluster1 AND info4 eq 35000) OR
+      // (info1 eq cluster2 AND info2 eq 2.0)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=(info1%20eq%20cluster1%20AND%20info4%20" +
+          "eq%2035000)%20OR%20(info1%20eq%20cluster2%20AND%20info2%20eq%202.0)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      int infoCnt = 0;
+      for (TimelineEntity entity : entities) {
+        infoCnt += entity.getInfo().size();
+        assertTrue(entity.getId().equals("entity2"));
+      }
+      // Includes UID in info field even if fields not specified as INFO.
+      assertEquals(1, infoCnt);
+
+      // infofilters=(info1 eq cluster1 AND info4 eq 35000) OR
+      // (info1 eq cluster2 AND info2 eq 2.0)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=(info1%20eq%20cluster1%20AND%20info4%20" +
+          "eq%2035000)%20OR%20(info1%20eq%20cluster2%20AND%20info2%20eq%20" +
+          "2.0)&fields=INFO");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      infoCnt = 0;
+      for (TimelineEntity entity : entities) {
+        infoCnt += entity.getInfo().size();
+        assertTrue(entity.getId().equals("entity2"));
+      }
+      // Includes UID in info field.
+      assertEquals(4, infoCnt);
+
+      // Test for behavior when compare op is ne(not equals) vs ene
+      // (exists and not equals). info3 does not exist for entity2. For ne,
+      // both entity1 and entity2 will be returned. For ene, only entity2 will
+      // be returned as we are checking for existence too.
+      // infofilters=info3 ne 39000
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=info3%20ne%2039000");
+      resp = getResponse(client, uri);
+      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"));
+      }
+      // infofilters=info3 ene 39000
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=info3%20ene%2039000");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity1"));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesMetricFilters() throws Exception {
+    Client client = createClient();
+    try {
+      // metricfilters=HDFS_BYTES_READ lt 60 OR HDFS_BYTES_READ eq 157
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=HDFS_BYTES_READ%20lt%2060%20OR%20" +
+          "HDFS_BYTES_READ%20eq%20157");
+      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"));
+      }
+
+      // metricfilters=HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=HDFS_BYTES_READ%20lt%2060%20AND%20" +
+          "MAP_SLOT_MILLIS%20gt%2040");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+
+      // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR
+      // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
+          "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
+          "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      int metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        assertTrue(entity.getId().equals("entity2"));
+      }
+      assertEquals(0, metricCnt);
+
+      // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR
+      // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
+          "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
+          "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)&fields=METRICS");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        assertTrue(entity.getId().equals("entity2"));
+      }
+      assertEquals(3, metricCnt);
+
+      // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR
+      // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
+          "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
+          "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)&metricstoretrieve=" +
+          "!(HDFS)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        assertTrue(entity.getId().equals("entity2"));
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP1"));
+        }
+      }
+      assertEquals(2, metricCnt);
+
+      // Test for behavior when compare op is ne(not equals) vs ene
+      // (exists and not equals). MAP11_SLOT_MILLIS does not exist for
+      // entity1. For ne, both entity1 and entity2 will be returned. For ene,
+      // only entity2 will be returned as we are checking for existence too.
+      // metricfilters=MAP11_SLOT_MILLIS ne 100
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=MAP11_SLOT_MILLIS%20ne%20100");
+      resp = getResponse(client, uri);
+      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"));
+      }
+      // metricfilters=MAP11_SLOT_MILLIS ene 100
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=MAP11_SLOT_MILLIS%20ene%20100");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity2"));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesEventFilters() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?eventfilters=event1,event3");
+      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"));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?eventfilters=!(event1,event3)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+
+      // eventfilters=!(event1,event3) OR event5,event6
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?eventfilters=!(event1,event3)%20OR%20event5,event6");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity2"));
+      }
+
+      //  eventfilters=(!(event1,event3) OR event5,event6) OR
+      // (event1,event2 AND (event3,event4))
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?eventfilters=(!(event1,event3)%20OR%20event5," +
+          "event6)%20OR%20(event1,event2%20AND%20(event3,event4))");
+      resp = getResponse(client, uri);
+      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 testGetEntitiesRelationFilters() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?isrelatedto=type3:entity31,type2:entity21:entity22");
+      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"));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
+          "?isrelatedto=!(type3:entity31,type2:entity21:entity22)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+
+      // isrelatedto=!(type3:entity31,type2:entity21:entity22)OR type5:entity51,
+      // type6:entity61:entity66
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
+          "?isrelatedto=!(type3:entity31,type2:entity21:entity22)%20OR%20" +
+          "type5:entity51,type6:entity61:entity66");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity2"));
+      }
+
+      // isrelatedto=(!(type3:entity31,type2:entity21:entity22)OR type5:
+      // entity51,type6:entity61:entity66) OR (type1:entity14,type2:entity21:
+      // entity22 AND (type3:entity32:entity35,type4:entity42))
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
+          "?isrelatedto=(!(type3:entity31,type2:entity21:entity22)%20OR%20" +
+          "type5:entity51,type6:entity61:entity66)%20OR%20(type1:entity14," +
+          "type2:entity21:entity22%20AND%20(type3:entity32:entity35,"+
+          "type4:entity42))");
+      resp = getResponse(client, uri);
+      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"));
+      }
+
+      // relatesto=!(type3:entity31,type2:entity21:entity22)OR type5:entity51,
+      // type6:entity61:entity66
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
+          "?relatesto=!%20(type3:entity31,type2:entity21:entity22%20)%20OR%20" +
+          "type5:entity51,type6:entity61:entity66");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity2"));
+      }
+
+      // relatesto=(!(type3:entity31,type2:entity21:entity22)OR type5:entity51,
+      // type6:entity61:entity66) OR (type1:entity14,type2:entity21:entity22 AND
+      // (type3:entity32:entity35 , type4:entity42))
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
+          "?relatesto=(!(%20type3:entity31,type2:entity21:entity22)%20OR%20" +
+          "type5:entity51,type6:entity61:entity66%20)%20OR%20(type1:entity14," +
+          "type2:entity21:entity22%20AND%20(type3:entity32:entity35%20,%20"+
+          "type4:entity42))");
+      resp = getResponse(client, uri);
+      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();
+    }
+  }
+
+  /**
+   * Tests if specific configs and metrics are retrieve for getEntity call.
+   */
+  @Test
+  public void testGetEntityDataToRetrieve() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1/entity2?confstoretrieve=cfg_,configuration_");
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("entity2", entity.getId());
+      assertEquals("type1", entity.getType());
+      assertEquals(2, entity.getConfigs().size());
+      for (String configKey : entity.getConfigs().keySet()) {
+        assertTrue(configKey.startsWith("configuration_") ||
+            configKey.startsWith("cfg_"));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1/entity2?confstoretrieve=!(cfg_,configuration_)");
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("entity2", entity.getId());
+      assertEquals("type1", entity.getType());
+      assertEquals(1, entity.getConfigs().size());
+      for (String configKey : entity.getConfigs().keySet()) {
+        assertTrue(configKey.startsWith("config_"));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1/entity2?metricstoretrieve=MAP1_,HDFS_");
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("entity2", entity.getId());
+      assertEquals("type1", entity.getType());
+      assertEquals(2, entity.getMetrics().size());
+      for (TimelineMetric  metric : entity.getMetrics()) {
+        assertTrue(metric.getId().startsWith("MAP1_") ||
+            metric.getId().startsWith("HDFS_"));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1/entity2?metricstoretrieve=!(MAP1_,HDFS_)");
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("entity2", entity.getId());
+      assertEquals("type1", entity.getType());
+      assertEquals(1, entity.getMetrics().size());
+      for (TimelineMetric  metric : entity.getMetrics()) {
+        assertTrue(metric.getId().startsWith("MAP11_"));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetFlowRunApps() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" +
+          "1002345678919/apps?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() == 3) ||
+            (entity.getId().equals("application_1111111111_2222") &&
+            entity.getMetrics().size() == 1));
+      }
+
+      // Query without specifying cluster ID.
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/users/user1/flows/flow_name/runs/1002345678919/apps");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/users/user1/flows/flow_name/runs/1002345678919/" +
+          "apps?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/clusters/cluster1/users/user1/flows/flow_name/apps?" +
+          "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() == 3) ||
+            (entity.getId().equals("application_1111111111_2222") &&
+            entity.getMetrics().size() == 1) ||
+            (entity.getId().equals("application_1111111111_2224") &&
+            entity.getMetrics().size() == 1));
+      }
+
+      // Query without specifying cluster ID.
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/users/user1/flows/flow_name/apps");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(3, entities.size());
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/users/user1/flows/flow_name/apps?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/clusters/cluster1/users/user1/flows/flow_name/apps?" +
+          "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/clusters/cluster1/users/user1/flows/flow_name/apps?" +
+          "metricfilters=HDFS_BYTES_READ%20ge%200");
+      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/clusters/cluster1/users/user1/flows/flow_name/apps?" +
+          "conffilters=cfg1%20eq%20value1");
+      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/clusters/cluster1/users/user1/flows/flow_name/runs/" +
+          "1002345678929");
+      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/clusters/cluster2/flows");
+      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/clusters/cluster1/apps/application_1111111111_1378");
+      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/clusters/cluster2/users/user1/flows/flow_name/runs/" +
+          "1002345678919/apps");
+      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/clusters/cluster2/users/user1/flows/flow_name55/apps");
+      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;
+    }
+  }
+}


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


[12/50] [abbrv] hadoop git commit: YARN-3150. Documenting the timeline service v2. (Sangjin Lee and Vrushali C via gtcarrera9)

Posted by vr...@apache.org.
YARN-3150. Documenting the timeline service v2. (Sangjin Lee and Vrushali C via gtcarrera9)


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

Branch: refs/heads/YARN-2928
Commit: 89e4a94cdaa446ca26503f6aad0e6505621f43de
Parents: 4caa146
Author: Li Lu <gt...@apache.org>
Authored: Sat Apr 30 15:02:12 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:07 2016 -0700

----------------------------------------------------------------------
 .../hadoop/yarn/conf/YarnConfiguration.java     |   8 +-
 .../src/main/resources/yarn-default.xml         |   6 +-
 .../metrics/TimelineServiceV2Publisher.java     |  16 +-
 .../TestSystemMetricsPublisherForV2.java        |  16 +-
 .../src/site/markdown/TimelineServer.md         |   2 +-
 .../src/site/markdown/TimelineServiceV2.md      | 576 +++++++++++++++++++
 .../src/site/resources/images/timeline_v2.jpg   | Bin 0 -> 45112 bytes
 7 files changed, 600 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/89e4a94c/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 3a8fb49..dc38631 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
@@ -507,13 +507,13 @@ public class YarnConfiguration extends Configuration {
   public static final boolean DEFAULT_SYSTEM_METRICS_PUBLISHER_ENABLED = false;
 
   /**
-   * The setting that controls whether yarn container metrics is published to
-   * the timeline server or not by RM. This configuration setting is for ATS
+   * The setting that controls whether yarn container events are published to
+   * the timeline service or not by RM. This configuration setting is for ATS
    * V2
    */
-  public static final String RM_PUBLISH_CONTAINER_METRICS_ENABLED = YARN_PREFIX
+  public static final String RM_PUBLISH_CONTAINER_EVENTS_ENABLED = YARN_PREFIX
       + "rm.system-metrics-publisher.emit-container-events";
-  public static final boolean DEFAULT_RM_PUBLISH_CONTAINER_METRICS_ENABLED =
+  public static final boolean DEFAULT_RM_PUBLISH_CONTAINER_EVENTS_ENABLED =
       false;
 
   public static final String RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89e4a94c/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 86f284a..6faf2a2 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
@@ -799,9 +799,9 @@
   </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>
+    <description>The setting that controls whether yarn container events are
+    published to the timeline service 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>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89e4a94c/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 83558f2..955570f 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
@@ -69,7 +69,7 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
   private static final Log LOG =
       LogFactory.getLog(TimelineServiceV2Publisher.class);
   private RMTimelineCollectorManager rmTimelineCollectorManager;
-  private boolean publishContainerMetrics;
+  private boolean publishContainerEvents;
 
   public TimelineServiceV2Publisher(RMContext rmContext) {
     super("TimelineserviceV2Publisher");
@@ -81,14 +81,14 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
     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);
+    publishContainerEvents = getConfig().getBoolean(
+        YarnConfiguration.RM_PUBLISH_CONTAINER_EVENTS_ENABLED,
+        YarnConfiguration.DEFAULT_RM_PUBLISH_CONTAINER_EVENTS_ENABLED);
   }
 
   @VisibleForTesting
-  boolean isPublishContainerMetrics() {
-    return publishContainerMetrics;
+  boolean isPublishContainerEvents() {
+    return publishContainerEvents;
   }
 
   @SuppressWarnings("unchecked")
@@ -305,7 +305,7 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
   @SuppressWarnings("unchecked")
   @Override
   public void containerCreated(RMContainer container, long createdTime) {
-    if (publishContainerMetrics) {
+    if (publishContainerEvents) {
       TimelineEntity entity = createContainerEntity(container.getContainerId());
       entity.setCreatedTime(createdTime);
 
@@ -340,7 +340,7 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
   @SuppressWarnings("unchecked")
   @Override
   public void containerFinished(RMContainer container, long finishedTime) {
-    if (publishContainerMetrics) {
+    if (publishContainerEvents) {
       TimelineEntity entity = createContainerEntity(container.getContainerId());
 
       TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89e4a94c/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 7a018ca..13c67f8 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
@@ -140,7 +140,7 @@ public class TestSystemMetricsPublisherForV2 {
     conf.setBoolean(YarnConfiguration.SYSTEM_METRICS_PUBLISHER_ENABLED, true);
     conf.setInt(
         YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE, 2);
-    conf.setBoolean(YarnConfiguration.RM_PUBLISH_CONTAINER_METRICS_ENABLED,
+    conf.setBoolean(YarnConfiguration.RM_PUBLISH_CONTAINER_EVENTS_ENABLED,
         true);
     try {
       conf.set(FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_DIR_ROOT,
@@ -160,12 +160,12 @@ public class TestSystemMetricsPublisherForV2 {
         new TimelineServiceV2Publisher(mock(RMContext.class));
     try {
       Configuration conf = getTimelineV2Conf();
-      conf.setBoolean(YarnConfiguration.RM_PUBLISH_CONTAINER_METRICS_ENABLED,
-          YarnConfiguration.DEFAULT_RM_PUBLISH_CONTAINER_METRICS_ENABLED);
+      conf.setBoolean(YarnConfiguration.RM_PUBLISH_CONTAINER_EVENTS_ENABLED,
+          YarnConfiguration.DEFAULT_RM_PUBLISH_CONTAINER_EVENTS_ENABLED);
       metricsPublisher.init(conf);
       assertFalse(
-          "Default configuration should not publish container Metrics from RM",
-          metricsPublisher.isPublishContainerMetrics());
+          "Default configuration should not publish container events from RM",
+          metricsPublisher.isPublishContainerEvents());
 
       metricsPublisher.stop();
 
@@ -174,10 +174,10 @@ public class TestSystemMetricsPublisherForV2 {
       metricsPublisher.init(conf);
       assertTrue("Expected to have registered event handlers and set ready to "
           + "publish events after init",
-          metricsPublisher.isPublishContainerMetrics());
+          metricsPublisher.isPublishContainerEvents());
       metricsPublisher.start();
-      assertTrue("Expected to publish container Metrics from RM",
-          metricsPublisher.isPublishContainerMetrics());
+      assertTrue("Expected to publish container events from RM",
+          metricsPublisher.isPublishContainerEvents());
     } finally {
       metricsPublisher.stop();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89e4a94c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
index f20bd2c..f09909b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServer.md
@@ -88,7 +88,7 @@ Current status
 Future Plans
 
 1. Future releases will introduce a next generation timeline service
-which is scalable and reliable, "Timeline Server v2".
+which is scalable and reliable, ["Timeline Service v2"](./TimelineServiceV2.html).
 1. The expanded features of this service *may not* be available to
 applications using the Timeline Server v1 REST API. That includes extended
 data structures as well as the ability of the client to failover between Timeline Server instances.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89e4a94c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
new file mode 100644
index 0000000..90db556
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
@@ -0,0 +1,576 @@
+<!---
+  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. See accompanying LICENSE file.
+-->
+
+The YARN Timeline Service v.2
+========================
+
+* [Overview](#Overview)
+    * [Introduction](#Introduction)
+    * [Architecture](#Architecture)
+    * [Current Status](#Current_Status)
+* [Deployment](#Deployment)
+    * [Configurations](#Configurations)
+    * [Enabling the Timeline Service v.2](#Enabling_Timeline_Service_v2)
+* [Publishing of application specific data](#Publishing_of_application_specific_data)
+* [Timeline Service v.2 REST API](#Timeline_Service_REST_API_v2)
+
+#<a name="Overview"></a>Overview
+
+### <a name="Introduction"></a>Introduction
+
+YARN Timeline Service v.2 is the next major iteration of Timeline Server, following v.1 and v.1.5.
+V.2 is created to address two major challenges of v.1.
+
+#### Scalability
+V.1 is limited to a single instance of writer/reader and storage, and does not scale well beyond
+small clusters. V.2 uses a more scalable distributed writer architecture and a scalable backend
+storage.
+
+YARN Timeline Service v.2 separates the collection (writes) of data from serving (reads) of data.
+It uses distributed collectors, essentially one collector for each YARN application. The readers
+are separate instances that are dedicated to serving queries via REST API.
+
+YARN Timeline Service v.2 chooses Apache HBase as the primary backing storage, as Apache HBase
+scales well to a large size while maintaining good response times for reads and writes.
+
+#### Usability improvements
+In many cases, users are interested in information at the level of "flows" or logical groups of
+YARN applications. It is much more common to launch a set or series of YARN applications to
+complete a logic application. Timeline Service v.2 supports the notion of flows explicitly. In
+addition, it supports aggregating metrics at the flow level.
+
+Also, information such as configuration and metrics is treated and supported as a first-class
+citizen.
+
+###<a name="Architecture"></a>Architecture
+
+YARN Timeline Service v.2 uses a set of collectors (writers) to write data to the backend storage.
+The collectors are distributed and co-located with the application masters to which they are
+dedicated. All data that belong to that application are sent to the application level timeline
+collectors with the exception of the resource manager timeline collector.
+
+For a given application, the application master can write data for the application to the
+co-located timeline collectors (which is an NM auxiliary service in this release). In addition,
+node managers of other nodes that are running the containers for the application also write data
+to the timeline collector on the node that is running the application master.
+
+The resource manager also maintains its own timeline collector. It emits only YARN-generic
+lifecycle events to keep its volume of writes reasonable.
+
+The timeline readers are separate daemons separate from the timeline collectors, and they are
+dedicated to serving queries via REST API.
+
+The following diagram illustrates the design at a high level.
+
+![Timeline Service v.2  architecture](./images/timeline_v2.jpg)
+
+### <a name="Current_Status"></a>Current Status and Future Plans
+
+YARN Timeline Service v.2 is currently in alpha. It is very much work in progress, and many things
+can and will change rapidly. Users should enable Timeline Service v.2 only on a test or
+experimental cluster to test the feature.
+
+A complete end-to-end flow of writes and reads should be functional, with Apache HBase as the
+backend. You should be able to start generating data. When enabled, all YARN-generic events are
+published as well as YARN system metrics such as CPU and memory. Furthermore, some applications
+including Distributed Shell and MapReduce write per-framework data to YARN Timeline Service v.2.
+
+The REST API comes with a good number of useful and flexible query patterns (see below for more
+information).
+
+Although the basic mode of accessing data is via REST, it also comes with a basic web UI based on
+the proposed new YARN UI framework. Currently there is no support for command line access, however.
+
+The collectors (writers) are currently embedded in the node managers as auxiliary services. The
+resource manager also has its dedicated in-process collector. The reader is currently a single
+instance. Currently, it is not possible to write to Timeline Service outside the context of a YARN
+application (i.e. no off-cluster client).
+
+When YARN Timeline Service v.2 is disabled, one should expect no functional or performance impact
+on any other existing functionality.
+
+The work to make it production-ready continues. Some key items include
+
+* More robust storage fault tolerance
+* Security
+* Support for off-cluster clients
+* More complete and integrated web UI
+* Better support for long-running apps
+* Offline (time-based periodic) aggregation for flows, users, and queues for reporting and
+analysis
+* Timeline collectors as separate instances from node managers
+* Clustering of the readers
+* Migration and compatibility with v.1
+
+
+#<a name="Deployment"></a>Deployment
+
+###<a name="Configurations"></a>Configurations
+
+New configuration parameters that are introduced with v.2 are marked bold.
+
+#### Basic configuration
+
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.enabled` | Indicate to clients whether Timeline service is enabled or not. If enabled, the `TimelineClient` library used by applications will post entities and events to the Timeline server. Defaults to `false`. |
+| `yarn.timeline-service.version` | 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 (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, the results may vary. Defaults to `1.0f`. |
+| **`yarn.timeline-service.writer.class`** | The class for the backend storage writer. Defaults to a filesystem storage writer, therefore it should be overridden. |
+| **`yarn.timeline-service.reader.class`** | The class for the backend storage reader. Defaults to a filesystem storage reader, therefore it should be overridden. |
+| **`yarn.system-metrics-publisher.enabled`** | The setting that controls whether yarn system metrics is published on the Timeline service or not by RM And NM. Defaults to `false`. |
+| **`yarn.rm.system-metrics-publisher.emit-container-events`** | The setting that controls whether yarn container metrics is published to the timeline server or not by RM. This configuration setting is for ATS V2. Defaults to `false`. |
+
+#### Advanced configuration
+
+| Configuration Property | Description |
+|:---- |:---- |
+| `yarn.timeline-service.hostname` | The hostname of the Timeline service web application. Defaults to `0.0.0.0` |
+| `yarn.timeline-service.address` | Address for the Timeline server to start the RPC server. Defaults to `${yarn.timeline-service.hostname}:10200`. |
+| `yarn.timeline-service.webapp.address` | The http address of the Timeline service web application. Defaults to `${yarn.timeline-service.hostname}:8188`. |
+| `yarn.timeline-service.webapp.https.address` | The https address of the Timeline service web application. Defaults to `${yarn.timeline-service.hostname}:8190`. |
+| **`yarn.timeline-service.writer.flush-interval-seconds`** | The setting that controls how often the timeline collector flushes the timeline writer. Defaults to `60`. |
+| **`yarn.timeline-service.app-collector.linger-period.ms`** | Time period till which the application collector will be alive in NM, after the  application master container finishes. Defaults to `1000` (1 second). |
+| **`yarn.timeline-service.timeline-client.number-of-async-entities-to-merge`** | Time line V2 client tries to merge these many number of async entities (if available) and then call the REST ATS V2 API to submit. Defaults to `10`. |
+| **`yarn.timeline-service.coprocessor.app-final-value-retention-milliseconds`** | The setting that controls how long the final value of a metric of a completed app is retained before merging into the flow sum. Defaults to `259200000` (3 days). |
+
+
+
+### <a name="Enabling_Timeline_Service_v2"></a>Enabling the Timeline Service v.2
+
+#### Preparing Apache HBase cluster for storage
+The first part is to set up or pick an Apache HBase cluster to use as the storage cluster. Once
+you have an HBase cluster ready to use for this purpose, perform the following steps.
+
+First, add the timeline service jar to the HBase classpath in all HBase machines in the cluster. It
+is needed for the coprocessor as well as the schema creator. For example,
+
+    cp hadoop-yarn-server-timelineservice-3.0.0-SNAPSHOT.jar /usr/hbase/lib/
+
+Then, enable the coprocessor that handles the aggregation. To enable it, add the following entry in
+region servers' `hbase-site.xml` file (generally located in the `conf` directory) as follows:
+
+```
+<property>
+  <name>hbase.coprocessor.region.classes</name>
+  <value>org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunCoprocessor</value>
+</property>
+```
+
+Restart the region servers and the master to pick up the timeline service jar as well as the config
+change. In this version, the coprocessor is loaded statically (i.e. system coprocessor) as opposed
+to a dynamically (table coprocessor).
+
+Finally, run the schema creator tool to create the necessary tables:
+
+    bin/hbase org.apache.hadoop.yarn.server.timelineservice.storage.TimelineSchemaCreator
+
+The `TimelineSchemaCreator` tool supports a few options that may come handy especially when you
+are testing. For example, you can use `-skipExistingTable` (`-s` for short) to skip existing tables
+and continue to create other tables rather than failing the schema creation.
+
+#### Enabling Timeline Service v.2
+Following are the basic configurations to start Timeline service v.2:
+
+```
+<property>
+  <name>yarn.timeline-service.version</name>
+  <value>2.0f</value>
+</property>
+
+<property>
+  <name>yarn.timeline-service.enabled</name>
+  <value>true</value>
+</property>
+
+<property>
+  <name>yarn.timeline-service.writer.class</name>
+  <value>org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl</value>
+</property>
+
+<property>
+  <name>yarn.timeline-service.reader.class</name>
+  <value>org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl</value>
+</property>
+
+<property>
+  <name>yarn.nodemanager.aux-services</name>
+  <value>mapreduce_shuffle,timeline_collector</value>
+</property>
+
+<property>
+  <name>yarn.nodemanager.aux-services.timeline_collector.class</name>
+  <value>org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService</value>
+</property>
+
+<property>
+  <description>The setting that controls whether yarn system metrics is
+  published on the Timeline service or not by RM And NM.</description>
+  <name>yarn.system-metrics-publisher.enabled</name>
+  <value>true</value>
+</property>
+
+<property>
+  <description>The setting that controls whether yarn container events are
+  published to the timeline service or not by RM. This configuration setting
+  is for ATS V2.</description>
+  <name>yarn.rm.system-metrics-publisher.emit-container-events</name>
+  <value>true</value>
+</property>
+```
+
+In addition, you may want to set the YARN cluster name to a reasonably unique name in case you
+are using multiple clusters to store data in the same Apache HBase storage:
+
+```
+<property>
+  <name>yarn.resourcemanager.cluster-id</name>
+  <value>my_research_test_cluster</value>
+</property>
+```
+
+Also, add the `hbase-site.xml` configuration file to the client Hadoop cluster configuration so
+that it can write data to the Apache HBase cluster you are using.
+
+#### Running Timeline Service v.2
+Restart the resource manager as well as the node managers to pick up the new configuration. The
+collectors start within the resource manager and the node managers in an embedded manner.
+
+The Timeline Service reader is a separate YARN daemon, and it can be started using the following
+syntax:
+
+    $ yarn-daemon.sh start timelinereader
+
+#### Enabling MapReduce to write to Timeline Service v.2
+To write MapReduce framework data to Timeline Service v.2, enable the following configuration in
+`mapred-site.xml`:
+
+```
+<property>
+  <name>mapreduce.job.emit-timeline-data</name>
+  <value>true</value>
+</property>
+```
+
+###<a name="Publishing_of_application_specific_data"></a> Publishing application specific data
+
+This section is for YARN application developers that want to integrate with Timeline Service v.2.
+
+Developers can continue to use the `TimelineClient` API to publish per-framework data to the
+Timeline Service v.2. You only need to instantiate the right type of the client to write to v.2.
+On the other hand, the entity/object API for v.2 is different than v.1 as the object model is
+significantly changed. The v.2 timeline entity class is `org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity`
+whereas the v.1 class is `org.apache.hadoop.yarn.api.records.timeline.TimelineEntity`. The methods
+on `TimelineClient` suitable for writing to the Timeline Service v.2 are clearly delineated, and
+they use the v.2 types as arguments.
+
+Timeline Service v.2 `putEntities` methods come in 2 varieties: `putEntities` and
+`putEntitiesAsync`. The former is a blocking operation which should be used for writing more
+critical data (e.g. lifecycle events). The latter is a non-blocking operation. Note that neither
+has a return value.
+
+Creating a `TimelineClient` for v.2 involves passing in the application id to the factory method.
+
+For example:
+
+
+    // Create and start the Timeline client v.2
+    TimelineClient client = TimelineClient.createTimelineClient(appId);
+    client.init(conf);
+    client.start();
+
+    try {
+      TimelineEntity myEntity = new TimelineEntity();
+      myEntity.setEntityType("MY_APPLICATION");
+      myEntity.setEntityId("MyApp1")
+      // Compose other entity info
+
+      // Blocking write
+      client.putEntities(entity);
+
+      TimelineEntity myEntity2 = new TimelineEntity();
+      // Compose other info
+
+      // Non-blocking write
+      timelineClient.putEntitiesAsync(entity);
+
+    } catch (IOException e) {
+      // Handle the exception
+    } catch (RuntimeException e) {
+      // In Hadoop 2.6, if attempts submit information to the Timeline Server fail more than the retry limit,
+      // a RuntimeException will be raised. This may change in future releases, being
+      // replaced with a IOException that is (or wraps) that which triggered retry failures.
+    } catch (YarnException e) {
+      // Handle the exception
+    } finally {
+      // Stop the Timeline client
+      client.stop();
+    }
+
+As evidenced above, you need to specify the YARN application id to be able to write to the Timeline
+Service v.2. Note that currently you need to be on the cluster to be able to write to the Timeline
+Service. For example, an application master or code in the container can write to the Timeline
+Service, while an off-cluster MapReduce job submitter cannot.
+
+You can create and publish your own entities, events, and metrics as with previous versions.
+
+Application frameworks should set the "flow context" whenever possible in order to take advantage
+of the flow support Timeline Service v.2 provides. The flow context consists of the following:
+
+* Flow name: a string that identifies the high-level flow (e.g. "distributed grep" or any
+identifiable name that can uniquely represent the app)
+* Flow run id: a monotonically-increasing sequence of numbers that distinguish different runs of
+the same flow
+* (optional) Flow version: a string identifier that denotes a version of the flow
+
+If the flow context is not specified, defaults are supplied for these attributes:
+
+* Flow name: the YARN application name (or the application id if the name is not set)
+* Flow run id: the application start time in Unix time (milliseconds)
+* Flow version: "1"
+
+You can provide the flow context via YARN application tags:
+
+    ApplicationSubmissionContext appContext = app.getApplicationSubmissionContext();
+
+    // set the flow context as YARN application tags
+    Set<String> tags = new HashSet<>();
+    tags.add(TimelineUtils.generateFlowNameTag("distributed grep"));
+    tags.add(Timelineutils.generateFlowVersionTag("3df8b0d6100530080d2e0decf9e528e57c42a90a"));
+    tags.add(TimelineUtils.generateFlowRunIdTag(System.currentTimeMillis()));
+
+    appContext.setApplicationTags(tags);
+
+# <a name="Timeline_Service_REST_API_v2"></a>Timeline Service v.2 REST API
+
+Querying the Timeline Service v.2 is currently only supported via REST API; there is no API
+client implemented in the YARN libraries.
+
+The v.2 REST API is implemented at under the path, `/ws/v2/timeline/` on the Timeline Service web
+service.
+
+Here is an informal description of the API.
+
+### Root path
+
+    GET /ws/v2/timeline/
+
+Returns a JSON object describing the service instance and version information.
+
+    {
+      "About":"Timeline Reader API",
+      "timeline-service-version":"3.0.0-SNAPSHOT",
+      "timeline-service-build-version":"3.0.0-SNAPSHOT from fb0acd08e6f0b030d82eeb7cbfa5404376313e60 by sjlee source checksum be6cba0e42417d53be16459e1685e7",
+      "timeline-service-version-built-on":"2016-04-11T23:15Z",
+      "hadoop-version":"3.0.0-SNAPSHOT",
+      "hadoop-build-version":"3.0.0-SNAPSHOT from fb0acd08e6f0b030d82eeb7cbfa5404376313e60 by sjlee source checksum ee968fd0aedcc7384230ee3ca216e790",
+      "hadoop-version-built-on":"2016-04-11T23:14Z"
+    }
+
+### Request Examples
+
+The following shows some of the supported queries on the REST API. For example, to get the most
+recent flow activities,
+
+HTTP request:
+
+    GET /ws/v2/timeline/clusters/{cluster name}/flows/
+
+Response:
+
+    [
+      {
+        "metrics": [],
+        "events": [],
+        "id": "test-cluster/1460419200000/sjlee@ds-date",
+        "type": "YARN_FLOW_ACTIVITY",
+        "createdtime": 0,
+        "flowruns": [
+          {
+            "metrics": [],
+            "events": [],
+            "id": "sjlee@ds-date/1460420305659",
+            "type": "YARN_FLOW_RUN",
+            "createdtime": 0,
+            "info": {
+              "SYSTEM_INFO_FLOW_VERSION": "1",
+              "SYSTEM_INFO_FLOW_RUN_ID": 1460420305659,
+              "SYSTEM_INFO_FLOW_NAME": "ds-date",
+              "SYSTEM_INFO_USER": "sjlee"
+            },
+            "isrelatedto": {},
+            "relatesto": {}
+          },
+          {
+            "metrics": [],
+            "events": [],
+            "id": "sjlee@ds-date/1460420587974",
+            "type": "YARN_FLOW_RUN",
+            "createdtime": 0,
+            "info": {
+              "SYSTEM_INFO_FLOW_VERSION": "1",
+              "SYSTEM_INFO_FLOW_RUN_ID": 1460420587974,
+              "SYSTEM_INFO_FLOW_NAME": "ds-date",
+              "SYSTEM_INFO_USER": "sjlee"
+            },
+            "isrelatedto": {},
+            "relatesto": {}
+          }
+        ],
+        "info": {
+          "SYSTEM_INFO_CLUSTER": "test-cluster",
+          "UID": "test-cluster!sjlee!ds-date",
+          "SYSTEM_INFO_FLOW_NAME": "ds-date",
+          "SYSTEM_INFO_DATE": 1460419200000,
+          "SYSTEM_INFO_USER": "sjlee"
+        },
+        "isrelatedto": {},
+        "relatesto": {}
+      }
+    ]
+
+It returns the flows that had runs (specific instances of the flows) most recently.
+
+You can drill further down to get the runs (specific instances) of a given flow.
+
+HTTP request:
+
+    GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs/
+
+Response:
+
+    [
+      {
+        "metrics": [],
+        "events": [],
+        "id": "sjlee@ds-date/1460420587974",
+        "type": "YARN_FLOW_RUN",
+        "createdtime": 1460420587974,
+        "info": {
+          "UID": "test-cluster!sjlee!ds-date!1460420587974",
+          "SYSTEM_INFO_FLOW_RUN_ID": 1460420587974,
+          "SYSTEM_INFO_FLOW_NAME": "ds-date",
+          "SYSTEM_INFO_FLOW_RUN_END_TIME": 1460420595198,
+          "SYSTEM_INFO_USER": "sjlee"
+        },
+        "isrelatedto": {},
+        "relatesto": {}
+      },
+      {
+        "metrics": [],
+        "events": [],
+        "id": "sjlee@ds-date/1460420305659",
+        "type": "YARN_FLOW_RUN",
+        "createdtime": 1460420305659,
+        "info": {
+          "UID": "test-cluster!sjlee!ds-date!1460420305659",
+          "SYSTEM_INFO_FLOW_RUN_ID": 1460420305659,
+          "SYSTEM_INFO_FLOW_NAME": "ds-date",
+          "SYSTEM_INFO_FLOW_RUN_END_TIME": 1460420311966,
+          "SYSTEM_INFO_USER": "sjlee"
+        },
+        "isrelatedto": {},
+        "relatesto": {}
+      }
+    ]
+
+This returns the most recent runs that belong to the given flow.
+
+You can provide a `limit` query parameter to limit the number of entries that returned in a query.
+If you want to limit the number of flow runs in the above query, you can do the following:
+
+HTTP request:
+
+    GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs?limit=1
+
+Response:
+
+    [
+      {
+        "metrics": [],
+        "events": [],
+        "id": "sjlee@ds-date/1460420587974",
+        "type": "YARN_FLOW_RUN",
+        "createdtime": 1460420587974,
+        "info": {
+          "UID": "test-cluster!sjlee!ds-date!1460420587974",
+          "SYSTEM_INFO_FLOW_RUN_ID": 1460420587974,
+          "SYSTEM_INFO_FLOW_NAME": "ds-date",
+          "SYSTEM_INFO_FLOW_RUN_END_TIME": 1460420595198,
+          "SYSTEM_INFO_USER": "sjlee"
+        },
+        "isrelatedto": {},
+        "relatesto": {}
+      }
+    ]
+
+Most queries in the v.2 REST API support the following query parameters:
+
+* `limit`
+* `createdtimestart`
+* `createdtimeend`
+* `relatesto`
+* `isrelatedto`
+* `infofilters`
+* `conffilters`
+* `metricfilters`
+* `eventfilters`
+* `fields`
+
+Given a flow run, you can query all the YARN applications that are part of that flow run:
+
+HTTP request:
+
+    GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs/{run id}/apps/
+
+Response:
+
+    [
+      {
+        "metrics": [],
+        "events": [],
+        "id": "application_1460419579913_0002",
+        "type": "YARN_APPLICATION",
+        "createdtime": 0,
+        "info": {
+          "UID": "test-cluster!sjlee!ds-date!1460420587974!application_1460419579913_0002"
+        },
+        "configs": {},
+        "isrelatedto": {},
+        "relatesto": {}
+      }
+    ]
+
+You can also provide per-framework entity type to query for them. For example,
+
+HTTP request:
+
+    GET /ws/v2/timeline/clusters/{cluster name}/apps/{app id}/entities/DS_APP_ATTEMPT
+
+Response:
+
+    [
+      {
+        "metrics": [],
+        "events": [],
+        "id": "appattempt_1460419579913_0002_000001",
+        "type": "DS_APP_ATTEMPT",
+        "createdtime": 0,
+        "info": {
+          "UID": "test-cluster!application_1460419579913_0002!DS_APP_ATTEMPT!appattempt_1460419579913_0002_000001"
+        },
+        "configs": {},
+        "isrelatedto": {},
+        "relatesto": {}
+      }
+    ]

http://git-wip-us.apache.org/repos/asf/hadoop/blob/89e4a94c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_v2.jpg
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_v2.jpg b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_v2.jpg
new file mode 100644
index 0000000..a356e26
Binary files /dev/null and b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/resources/images/timeline_v2.jpg differ


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


[11/50] [abbrv] hadoop git commit: YARN-4447. Provide a mechanism to represent complex filters and parse them at the REST layer (Varun Saxena via sjlee)

Posted by vr...@apache.org.
YARN-4447. Provide a mechanism to represent complex filters and parse them at the REST layer (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/8c00fef2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8c00fef2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8c00fef2

Branch: refs/heads/YARN-2928
Commit: 8c00fef2e85b7940240ed0af0680e4f5017bd2ec
Parents: 5f5c98e
Author: Sangjin Lee <sj...@apache.org>
Authored: Mon May 2 14:06:19 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:07 2016 -0700

----------------------------------------------------------------------
 .../reader/TimelineParseConstants.java          |  34 +
 .../reader/TimelineParseException.java          |  36 +
 .../timelineservice/reader/TimelineParser.java  |  37 +
 .../reader/TimelineParserForCompareExpr.java    | 300 ++++++
 .../reader/TimelineParserForDataToRetrieve.java |  95 ++
 .../reader/TimelineParserForEqualityExpr.java   | 343 +++++++
 .../reader/TimelineParserForExistFilters.java   |  51 +
 .../reader/TimelineParserForKVFilters.java      |  78 ++
 .../reader/TimelineParserForNumericFilters.java |  72 ++
 .../TimelineParserForRelationFilters.java       |  71 ++
 .../reader/TimelineReaderWebServices.java       | 220 ++++-
 .../reader/TimelineReaderWebServicesUtils.java  | 196 ++--
 .../reader/filter/TimelineCompareFilter.java    |  73 +-
 .../reader/filter/TimelineExistsFilter.java     |  49 +-
 .../reader/filter/TimelineFilterList.java       |  36 +
 .../reader/filter/TimelineKeyValueFilter.java   |  13 +
 .../reader/filter/TimelineKeyValuesFilter.java  |  61 +-
 .../reader/filter/TimelinePrefixFilter.java     |  37 +
 .../reader/TestTimelineReaderWebServices.java   |  14 +-
 ...stTimelineReaderWebServicesHBaseStorage.java | 900 +++++++++++++++++-
 .../TestTimelineReaderWebServicesUtils.java     | 923 +++++++++++++++++++
 21 files changed, 3442 insertions(+), 197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseConstants.java
----------------------------------------------------------------------
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/TimelineParseConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseConstants.java
new file mode 100644
index 0000000..662a102
--- /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/TimelineParseConstants.java
@@ -0,0 +1,34 @@
+/**
+ * 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;
+
+/**
+ * Set of constants used while parsing filter expressions.
+ */
+final class TimelineParseConstants {
+  private TimelineParseConstants() {
+  }
+  static final String COMMA_DELIMITER = ",";
+  static final String COLON_DELIMITER = ":";
+  static final char NOT_CHAR = '!';
+  static final char SPACE_CHAR = ' ';
+  static final char OPENING_BRACKET_CHAR = '(';
+  static final char CLOSING_BRACKET_CHAR = ')';
+  static final char COMMA_CHAR = ',';
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseException.java
----------------------------------------------------------------------
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/TimelineParseException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParseException.java
new file mode 100644
index 0000000..8d4a5dc
--- /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/TimelineParseException.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;
+
+/**
+ * Exception thrown to indicate that a timeline filter expression cannot be
+ * parsed.
+ */
+class TimelineParseException extends Exception {
+
+  private static final long serialVersionUID = 1L;
+
+  public TimelineParseException() {
+    super();
+  }
+
+  public TimelineParseException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParser.java
----------------------------------------------------------------------
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/TimelineParser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParser.java
new file mode 100644
index 0000000..6b461a0
--- /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/TimelineParser.java
@@ -0,0 +1,37 @@
+/**
+ * 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 java.io.Closeable;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+
+@Private
+@Unstable
+interface TimelineParser extends Closeable {
+  /**
+   * Method used for parsing.
+   *
+   * @return a {@link TimelineFilterList} object.
+   * @throws TimelineParseException if any problem occurs while parsing.
+   */
+  TimelineFilterList parse() throws TimelineParseException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForCompareExpr.java
----------------------------------------------------------------------
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/TimelineParserForCompareExpr.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForCompareExpr.java
new file mode 100644
index 0000000..1b020d9
--- /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/TimelineParserForCompareExpr.java
@@ -0,0 +1,300 @@
+/**
+ * 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 java.util.Deque;
+import java.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator;
+
+/**
+ * Abstract class for parsing compare expressions.
+ * Compare expressions are of the form :
+ * (&lt;key&gt; &lt;compareop&gt; &lt;value&gt;) &lt;op&gt; (&lt;key
+ * &gt; &lt;compareop&gt; &lt;value&gt;)
+ * compareop is used to compare value of a the specified key in the backend
+ * storage. compareop can be :
+ * 1. eq - Equals
+ * 2. ne - Not equals (matches if key does not exist)
+ * 3. ene - Exists and not equals (key must exist for match to occur)
+ * 4. lt - Less than
+ * 5. gt - Greater than
+ * 6. le - Less than or equals
+ * 7. ge - Greater than or equals
+ * compareop's supported would depend on implementation. For instance, all
+ * the above compareops' will be supported for metric filters but only eq,ne and
+ * ene would be supported for KV filters like config/info filters.
+ *
+ * op is a logical operator and can be either AND or OR.
+ *
+ * The way values will be interpreted would also depend on implementation
+ *
+ * A typical compare expression would look as under:
+ * ((key1 eq val1 OR key2 ne val2) AND (key5 gt val45))
+ */
+@Private
+@Unstable
+abstract class TimelineParserForCompareExpr implements TimelineParser {
+  private enum ParseState {
+    PARSING_KEY,
+    PARSING_VALUE,
+    PARSING_OP,
+    PARSING_COMPAREOP
+  }
+  // Main expression.
+  private final String expr;
+  // Expression in lower case.
+  private final String exprInLowerCase;
+  private final String exprName;
+  private int offset = 0;
+  private int kvStartOffset = 0;
+  private final int exprLength;
+  private ParseState currentParseState = ParseState.PARSING_KEY;
+  // Linked list implemented as a stack.
+  private Deque<TimelineFilterList> filterListStack = new LinkedList<>();
+  private TimelineFilter currentFilter = null;
+  private TimelineFilterList filterList = null;
+  public TimelineParserForCompareExpr(String expression, String name) {
+    if (expression != null) {
+      expr = expression.trim();
+      exprLength = expr.length();
+      exprInLowerCase = expr.toLowerCase();
+    } else {
+      expr = null;
+      exprInLowerCase = null;
+      exprLength = 0;
+    }
+    this.exprName = name;
+  }
+
+  protected TimelineFilter getCurrentFilter() {
+    return currentFilter;
+  }
+
+  protected TimelineFilter getFilterList() {
+    return filterList;
+  }
+
+  protected abstract TimelineFilter createFilter();
+
+  protected abstract Object parseValue(String strValue)
+      throws TimelineParseException;
+
+  protected abstract void setCompareOpToCurrentFilter(
+      TimelineCompareOp compareOp, boolean keyMustExistFlag)
+      throws TimelineParseException;
+
+  protected abstract void setValueToCurrentFilter(Object value);
+
+  private void handleSpaceChar() throws TimelineParseException {
+    if (currentParseState == ParseState.PARSING_KEY ||
+        currentParseState == ParseState.PARSING_VALUE) {
+      if (kvStartOffset == offset) {
+        kvStartOffset++;
+        offset++;
+        return;
+      }
+      String str = expr.substring(kvStartOffset, offset);
+      if (currentParseState == ParseState.PARSING_KEY) {
+        if (currentFilter == null) {
+          currentFilter = createFilter();
+        }
+        ((TimelineCompareFilter)currentFilter).setKey(str);
+        currentParseState = ParseState.PARSING_COMPAREOP;
+      } else if (currentParseState == ParseState.PARSING_VALUE) {
+        if (currentFilter != null) {
+          setValueToCurrentFilter(parseValue(str));
+        }
+        currentParseState = ParseState.PARSING_OP;
+      }
+    }
+    offset++;
+  }
+
+  private void handleOpeningBracketChar() throws TimelineParseException {
+    if (currentParseState != ParseState.PARSING_KEY) {
+      throw new TimelineParseException("Encountered unexpected opening " +
+          "bracket while parsing " + exprName + ".");
+    }
+    offset++;
+    kvStartOffset = offset;
+    filterListStack.push(filterList);
+    filterList = null;
+  }
+
+  private void handleClosingBracketChar() throws TimelineParseException {
+    if (currentParseState != ParseState.PARSING_VALUE &&
+        currentParseState != ParseState.PARSING_OP) {
+      throw new TimelineParseException("Encountered unexpected closing " +
+          "bracket while parsing " + exprName + ".");
+    }
+    if (!filterListStack.isEmpty()) {
+      if (currentParseState == ParseState.PARSING_VALUE) {
+        setValueToCurrentFilter(
+            parseValue(expr.substring(kvStartOffset, offset)));
+        currentParseState = ParseState.PARSING_OP;
+      }
+      if (currentFilter != null) {
+        filterList.addFilter(currentFilter);
+      }
+      // As bracket is closing, pop the filter list from top of the stack and
+      // combine it with current filter list.
+      TimelineFilterList fList = filterListStack.pop();
+      if (fList != null) {
+        fList.addFilter(filterList);
+        filterList = fList;
+      }
+      currentFilter = null;
+      offset++;
+      kvStartOffset = offset;
+    } else {
+      throw new TimelineParseException("Encountered unexpected closing " +
+          "bracket while parsing " + exprName + ".");
+    }
+  }
+
+  private void parseCompareOp() throws TimelineParseException {
+    if (offset + 2 >= exprLength) {
+      throw new TimelineParseException("Compare op cannot be parsed for " +
+          exprName + ".");
+    }
+    TimelineCompareOp compareOp = null;
+    boolean keyExistFlag = true;
+    if (expr.charAt(offset + 2) == TimelineParseConstants.SPACE_CHAR) {
+      if (exprInLowerCase.startsWith("eq", offset)) {
+        compareOp = TimelineCompareOp.EQUAL;
+      } else if (exprInLowerCase.startsWith("ne", offset)) {
+        compareOp = TimelineCompareOp.NOT_EQUAL;
+        keyExistFlag = false;
+      } else if (exprInLowerCase.startsWith("lt", offset)) {
+        compareOp = TimelineCompareOp.LESS_THAN;
+      } else if (exprInLowerCase.startsWith("le", offset)) {
+        compareOp = TimelineCompareOp.LESS_OR_EQUAL;
+      } else if (exprInLowerCase.startsWith("gt", offset)) {
+        compareOp = TimelineCompareOp.GREATER_THAN;
+      } else if (exprInLowerCase.startsWith("ge", offset)) {
+        compareOp = TimelineCompareOp.GREATER_OR_EQUAL;
+      }
+      offset = offset + 3;
+    } else if (exprInLowerCase.startsWith("ene ", offset)) {
+      // Not equal but key should be present.
+      compareOp = TimelineCompareOp.NOT_EQUAL;
+      offset = offset + 4;
+    }
+    if (compareOp == null) {
+      throw new TimelineParseException("Compare op cannot be parsed for " +
+          exprName + ".");
+    }
+    setCompareOpToCurrentFilter(compareOp, keyExistFlag);
+    kvStartOffset = offset;
+    currentParseState = ParseState.PARSING_VALUE;
+  }
+
+  private void parseOp(boolean closingBracket) throws TimelineParseException {
+    Operator operator = null;
+    if (exprInLowerCase.startsWith("or ", offset)) {
+      operator = Operator.OR;
+      offset = offset + 3;
+    } else if (exprInLowerCase.startsWith("and ", offset)) {
+      operator = Operator.AND;
+      offset = offset + 4;
+    }
+    if (operator == null) {
+      throw new TimelineParseException("Operator cannot be parsed for " +
+          exprName + ".");
+    }
+    if (filterList == null) {
+      filterList = new TimelineFilterList(operator);
+    }
+    if (currentFilter != null) {
+      filterList.addFilter(currentFilter);
+    }
+    if (closingBracket || filterList.getOperator() != operator) {
+      filterList = new TimelineFilterList(operator, filterList);
+    }
+    currentFilter = null;
+    kvStartOffset = offset;
+    currentParseState = ParseState.PARSING_KEY;
+  }
+
+  @Override
+  public TimelineFilterList parse() throws TimelineParseException {
+    if (expr == null || exprLength == 0) {
+      return null;
+    }
+    boolean closingBracket = false;
+    while (offset < exprLength) {
+      char offsetChar = expr.charAt(offset);
+      switch(offsetChar) {
+      case TimelineParseConstants.SPACE_CHAR:
+        handleSpaceChar();
+        break;
+      case TimelineParseConstants.OPENING_BRACKET_CHAR:
+        handleOpeningBracketChar();
+        break;
+      case TimelineParseConstants.CLOSING_BRACKET_CHAR:
+        handleClosingBracketChar();
+        closingBracket = true;
+        break;
+      default: // other characters.
+        // Parse based on state.
+        if (currentParseState == ParseState.PARSING_COMPAREOP) {
+          parseCompareOp();
+        } else if (currentParseState == ParseState.PARSING_OP) {
+          parseOp(closingBracket);
+          closingBracket = false;
+        } else {
+          // Might be a key or value. Move ahead.
+          offset++;
+        }
+        break;
+      }
+    }
+    if (!filterListStack.isEmpty()) {
+      filterListStack.clear();
+      throw new TimelineParseException("Encountered improper brackets while " +
+          "parsing " + exprName + ".");
+    }
+    if (currentParseState == ParseState.PARSING_VALUE) {
+      setValueToCurrentFilter(
+          parseValue(expr.substring(kvStartOffset, offset)));
+    }
+    if (filterList == null || filterList.getFilterList().isEmpty()) {
+      filterList = new TimelineFilterList(currentFilter);
+    } else if (currentFilter != null) {
+      filterList.addFilter(currentFilter);
+    }
+    return filterList;
+  }
+
+  @Override
+  public void close() {
+    if (filterListStack != null) {
+      filterListStack.clear();
+    }
+    filterList = null;
+    currentFilter = null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForDataToRetrieve.java
----------------------------------------------------------------------
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/TimelineParserForDataToRetrieve.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForDataToRetrieve.java
new file mode 100644
index 0000000..1e6039d
--- /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/TimelineParserForDataToRetrieve.java
@@ -0,0 +1,95 @@
+/**
+ * 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 org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+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;
+
+/**
+ * Used for parsing metrics or configs to retrieve.
+ */
+@Private
+@Unstable
+public class TimelineParserForDataToRetrieve implements TimelineParser {
+  private String expr;
+  private final int exprLength;
+  public TimelineParserForDataToRetrieve(String expression) {
+    this.expr = expression;
+    if (expression != null) {
+      this.expr = expr.trim();
+      exprLength = expr.length();
+    } else {
+      exprLength = 0;
+    }
+  }
+
+  @Override
+  public TimelineFilterList parse() throws TimelineParseException {
+    if (expr == null || exprLength == 0) {
+      return null;
+    }
+    TimelineCompareOp compareOp = null;
+    int openingBracketIndex =
+        expr.indexOf(TimelineParseConstants.OPENING_BRACKET_CHAR);
+    if (expr.charAt(0) == TimelineParseConstants.NOT_CHAR) {
+      if (openingBracketIndex == -1) {
+        throw new TimelineParseException("Invalid config/metric to retrieve " +
+            "expression");
+      }
+      if (openingBracketIndex != 1 &&
+          expr.substring(1, openingBracketIndex + 1).trim().length() != 1) {
+        throw new TimelineParseException("Invalid config/metric to retrieve " +
+            "expression");
+      }
+      compareOp = TimelineCompareOp.NOT_EQUAL;
+    } else if (openingBracketIndex <= 0) {
+      compareOp = TimelineCompareOp.EQUAL;
+    }
+    char lastChar = expr.charAt(exprLength - 1);
+    if (compareOp == TimelineCompareOp.NOT_EQUAL &&
+        lastChar != TimelineParseConstants.CLOSING_BRACKET_CHAR) {
+      throw new TimelineParseException("Invalid config/metric to retrieve " +
+          "expression");
+    }
+    if (openingBracketIndex != -1 &&
+        expr.charAt(exprLength - 1) ==
+            TimelineParseConstants.CLOSING_BRACKET_CHAR) {
+      expr = expr.substring(openingBracketIndex + 1, exprLength - 1).trim();
+    }
+    if (expr.isEmpty()) {
+      return null;
+    }
+    Operator op =
+        (compareOp == TimelineCompareOp.NOT_EQUAL) ? Operator.AND : Operator.OR;
+    TimelineFilterList list = new TimelineFilterList(op);
+    String[] splits = expr.split(TimelineParseConstants.COMMA_DELIMITER);
+    for (String split : splits) {
+      list.addFilter(new TimelinePrefixFilter(compareOp, split.trim()));
+    }
+    return list;
+  }
+
+  @Override
+  public void close() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForEqualityExpr.java
----------------------------------------------------------------------
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/TimelineParserForEqualityExpr.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForEqualityExpr.java
new file mode 100644
index 0000000..7451713
--- /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/TimelineParserForEqualityExpr.java
@@ -0,0 +1,343 @@
+/**
+ * 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 java.util.Deque;
+import java.util.LinkedList;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator;
+
+/**
+ * Abstract class for parsing equality expressions. This means the values in
+ * expression would either be equal or not equal.
+ * Equality expressions are of the form :
+ * (&lt;value&gt;,&lt;value&gt;,&lt;value&gt;) &lt;op&gt; !(&lt;value&gt;,
+ * &lt;value&gt;)
+ *
+ * Here, "!" means all the values should not exist/should not be equal.
+ * If not specified, they should exist/be equal.
+ *
+ * op is a logical operator and can be either AND or OR.
+ *
+ * The way values will be interpreted would also depend on implementation.
+ *
+ * For instance for event filters this expression may look like,
+ * (event1,event2) AND !(event3,event4)
+ * This means for an entity to match, event1 and event2 should exist. But event3
+ * and event4 should not exist.
+ */
+@Private
+@Unstable
+abstract class TimelineParserForEqualityExpr implements TimelineParser {
+  private enum ParseState {
+    PARSING_VALUE,
+    PARSING_OP,
+    PARSING_COMPAREOP
+  }
+  private final String expr;
+  // Expression in lower case.
+  private final String exprInLowerCase;
+  // Expression name.
+  private final String exprName;
+  // Expression offset.
+  private int offset = 0;
+  // Offset used to parse values in the expression.
+  private int startOffset = 0;
+  private final int exprLength;
+  private ParseState currentParseState = ParseState.PARSING_COMPAREOP;
+  private TimelineCompareOp currentCompareOp = null;
+  // Used to store filter lists which can then be combined as brackets are
+  // closed.
+  private Deque<TimelineFilterList> filterListStack = new LinkedList<>();
+  private TimelineFilter currentFilter = null;
+  private TimelineFilterList filterList = null;
+  // Delimiter used to separate values.
+  private final char delimiter;
+  public TimelineParserForEqualityExpr(String expression, String name,
+      char delim) {
+    if (expression != null) {
+      expr = expression.trim();
+      exprLength = expr.length();
+      exprInLowerCase = expr.toLowerCase();
+    } else {
+      exprLength = 0;
+      expr = null;
+      exprInLowerCase = null;
+    }
+    exprName = name;
+    delimiter = delim;
+  }
+
+  protected TimelineFilter getCurrentFilter() {
+    return currentFilter;
+  }
+
+  protected TimelineFilter getFilterList() {
+    return filterList;
+  }
+
+  /**
+   * Creates filter as per implementation.
+   *
+   * @return a {@link TimelineFilter} implementation.
+   */
+  protected abstract TimelineFilter createFilter();
+
+  /**
+   * Sets compare op to the current filter as per filter implementation.
+   *
+   * @param compareOp compare op to be set.
+   * @throws Exception if any problem occurs.
+   */
+  protected abstract void setCompareOpToCurrentFilter(
+      TimelineCompareOp compareOp) throws TimelineParseException;
+
+  /**
+   * Sets value to the current filter as per filter implementation.
+   *
+   * @param value value to be set.
+   * @throws Exception if any problem occurs.
+   */
+  protected abstract void setValueToCurrentFilter(String value)
+      throws TimelineParseException;
+
+  private void createAndSetFilter(boolean checkIfNull)
+      throws TimelineParseException {
+    if (!checkIfNull || currentFilter == null) {
+      currentFilter = createFilter();
+      setCompareOpToCurrentFilter(currentCompareOp);
+    }
+    setValueToCurrentFilter(expr.substring(startOffset, offset).trim());
+  }
+
+  private void handleSpaceChar() throws TimelineParseException {
+    if (currentParseState == ParseState.PARSING_VALUE) {
+      if (startOffset == offset) {
+        startOffset++;
+      } else {
+        createAndSetFilter(true);
+        currentParseState = ParseState.PARSING_OP;
+      }
+    }
+    offset++;
+  }
+
+  private void handleDelimiter() throws TimelineParseException {
+    if (currentParseState == ParseState.PARSING_OP ||
+        currentParseState == ParseState.PARSING_VALUE) {
+      if (currentParseState == ParseState.PARSING_VALUE) {
+        createAndSetFilter(false);
+      }
+      if (filterList == null) {
+        filterList = new TimelineFilterList();
+      }
+      // Add parsed filter into filterlist and make it null to move on to next
+      // filter.
+      filterList.addFilter(currentFilter);
+      currentFilter = null;
+      offset++;
+      startOffset = offset;
+      currentParseState = ParseState.PARSING_VALUE;
+    } else {
+      throw new TimelineParseException("Invalid " + exprName + "expression.");
+    }
+  }
+
+  private void handleOpeningBracketChar(boolean encounteredNot)
+      throws TimelineParseException {
+    if (currentParseState == ParseState.PARSING_COMPAREOP ||
+        currentParseState == ParseState.PARSING_VALUE) {
+      offset++;
+      startOffset = offset;
+      filterListStack.push(filterList);
+      filterList = null;
+      if (currentFilter == null) {
+        currentFilter = createFilter();
+      }
+      currentCompareOp = encounteredNot ?
+          TimelineCompareOp.NOT_EQUAL : TimelineCompareOp.EQUAL;
+      setCompareOpToCurrentFilter(currentCompareOp);
+      currentParseState = ParseState.PARSING_VALUE;
+    } else {
+      throw new TimelineParseException("Encountered unexpected opening " +
+          "bracket while parsing " + exprName + ".");
+    }
+  }
+
+  private void handleNotChar() throws TimelineParseException {
+    if (currentParseState == ParseState.PARSING_COMPAREOP ||
+        currentParseState == ParseState.PARSING_VALUE) {
+      offset++;
+      while (offset < exprLength &&
+          expr.charAt(offset) == TimelineParseConstants.SPACE_CHAR) {
+        offset++;
+      }
+      if (offset == exprLength) {
+        throw new TimelineParseException("Invalid " + exprName + "expression");
+      }
+      if (expr.charAt(offset) == TimelineParseConstants.OPENING_BRACKET_CHAR) {
+        handleOpeningBracketChar(true);
+      } else {
+        throw new TimelineParseException("Invalid " + exprName + "expression");
+      }
+    } else {
+      throw new TimelineParseException("Encountered unexpected not(!) char " +
+         "while parsing " + exprName + ".");
+    }
+  }
+
+  private void handleClosingBracketChar() throws TimelineParseException {
+    if (currentParseState != ParseState.PARSING_VALUE &&
+        currentParseState != ParseState.PARSING_OP) {
+      throw new TimelineParseException("Encountered unexpected closing " +
+          "bracket while parsing " + exprName + ".");
+    }
+    if (!filterListStack.isEmpty()) {
+      if (currentParseState == ParseState.PARSING_VALUE) {
+        if (startOffset != offset) {
+          createAndSetFilter(true);
+          currentParseState = ParseState.PARSING_OP;
+        }
+      }
+      if (filterList == null) {
+        filterList = new TimelineFilterList();
+      }
+      if (currentFilter != null) {
+        filterList.addFilter(currentFilter);
+      }
+      // As bracket is closing, pop the filter list from top of the stack and
+      // combine it with current filter list.
+      TimelineFilterList fList = filterListStack.pop();
+      if (fList != null) {
+        fList.addFilter(filterList);
+        filterList = fList;
+      }
+      currentFilter = null;
+      offset++;
+      startOffset = offset;
+    } else {
+      throw new TimelineParseException("Encountered unexpected closing " +
+          "bracket while parsing " + exprName + ".");
+    }
+  }
+
+  private void parseOp(boolean closingBracket) throws TimelineParseException {
+    Operator operator = null;
+    if (exprInLowerCase.startsWith("or ", offset)) {
+      operator = Operator.OR;
+      offset = offset + 3;
+    } else if (exprInLowerCase.startsWith("and ", offset)) {
+      operator = Operator.AND;
+      offset = offset + 4;
+    }
+    if (operator == null) {
+      throw new TimelineParseException("Operator cannot be parsed for " +
+          exprName + ".");
+    }
+    if (filterList == null) {
+      filterList = new TimelineFilterList(operator);
+    }
+    if (currentFilter != null) {
+      filterList.addFilter(currentFilter);
+    }
+    if (closingBracket || filterList.getOperator() != operator) {
+      filterList = new TimelineFilterList(operator, filterList);
+    }
+    currentFilter = null;
+    startOffset = offset;
+    currentParseState = ParseState.PARSING_COMPAREOP;
+  }
+
+  private void parseCompareOp() throws TimelineParseException {
+    if (currentFilter == null) {
+      currentFilter = createFilter();
+    }
+    currentCompareOp = TimelineCompareOp.EQUAL;
+    setCompareOpToCurrentFilter(currentCompareOp);
+    currentParseState = ParseState.PARSING_VALUE;
+  }
+
+  @Override
+  public TimelineFilterList parse() throws TimelineParseException {
+    if (expr == null || exprLength == 0) {
+      return null;
+    }
+    boolean closingBracket = false;
+    while (offset < exprLength) {
+      char offsetChar = expr.charAt(offset);
+      switch(offsetChar) {
+      case TimelineParseConstants.NOT_CHAR:
+        handleNotChar();
+        break;
+      case TimelineParseConstants.SPACE_CHAR:
+        handleSpaceChar();
+        break;
+      case TimelineParseConstants.OPENING_BRACKET_CHAR:
+        handleOpeningBracketChar(false);
+        break;
+      case TimelineParseConstants.CLOSING_BRACKET_CHAR:
+        handleClosingBracketChar();
+        closingBracket = true;
+        break;
+      default: // other characters.
+        if (offsetChar == delimiter) {
+          handleDelimiter();
+        } else if (currentParseState == ParseState.PARSING_COMPAREOP) {
+          parseCompareOp();
+        } else if (currentParseState == ParseState.PARSING_OP) {
+          parseOp(closingBracket);
+          closingBracket = false;
+        } else {
+          offset++;
+        }
+        break;
+      }
+    }
+    if (!filterListStack.isEmpty()) {
+      filterListStack.clear();
+      throw new TimelineParseException("Encountered improper brackets while " +
+          "parsing " + exprName + ".");
+    }
+    if (currentParseState == ParseState.PARSING_VALUE) {
+      if (startOffset != offset) {
+        createAndSetFilter(true);
+      }
+    }
+    if (filterList == null || filterList.getFilterList().isEmpty()) {
+      filterList = new TimelineFilterList(currentFilter);
+    } else if (currentFilter != null) {
+      filterList.addFilter(currentFilter);
+    }
+    return filterList;
+  }
+
+  @Override
+  public void close() {
+    if (filterListStack != null) {
+      filterListStack.clear();
+    }
+    currentFilter = null;
+    filterList = null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForExistFilters.java
----------------------------------------------------------------------
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/TimelineParserForExistFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForExistFilters.java
new file mode 100644
index 0000000..8048c6e
--- /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/TimelineParserForExistFilters.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.reader;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
+
+/**
+ * Used for parsing existence filters such as event filters. These filters
+ * check for existence of a value. For example, in case of event filters, they
+ * check if an event exists or not and accordingly return an entity.
+ */
+@Private
+@Unstable
+class TimelineParserForExistFilters extends TimelineParserForEqualityExpr {
+
+  public TimelineParserForExistFilters(String expression, char delimiter) {
+    super(expression, "Event Filter", delimiter);
+  }
+
+  protected TimelineFilter createFilter() {
+    return new TimelineExistsFilter();
+  }
+
+  protected void setValueToCurrentFilter(String value) {
+    ((TimelineExistsFilter)getCurrentFilter()).setValue(value);
+  }
+
+  protected void setCompareOpToCurrentFilter(TimelineCompareOp compareOp) {
+    ((TimelineExistsFilter)getCurrentFilter()).setCompareOp(compareOp);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForKVFilters.java
----------------------------------------------------------------------
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/TimelineParserForKVFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForKVFilters.java
new file mode 100644
index 0000000..ec68bec
--- /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/TimelineParserForKVFilters.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.reader;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+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.TimelineFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
+
+/**
+ * Used for parsing key-value filters such as config and info filters.
+ */
+@Private
+@Unstable
+class TimelineParserForKVFilters extends TimelineParserForCompareExpr {
+  // Indicates if value has to be interpreted as a string.
+  private final boolean valueAsString;
+  public TimelineParserForKVFilters(String expression, boolean valAsStr) {
+    super(expression, "Config/Info Filter");
+    this.valueAsString = valAsStr;
+  }
+
+  protected TimelineFilter createFilter() {
+    return new TimelineKeyValueFilter();
+  }
+
+  protected Object parseValue(String strValue) {
+    if (!valueAsString) {
+      try {
+        return GenericObjectMapper.OBJECT_READER.readValue(strValue);
+      } catch (IOException e) {
+        return strValue;
+      }
+    } else {
+      return strValue;
+    }
+  }
+
+  @Override
+  protected void setCompareOpToCurrentFilter(TimelineCompareOp compareOp,
+      boolean keyMustExistFlag) throws TimelineParseException {
+    if (compareOp != TimelineCompareOp.EQUAL &&
+        compareOp != TimelineCompareOp.NOT_EQUAL) {
+      throw new TimelineParseException("TimelineCompareOp for kv-filter " +
+          "should be EQUAL or NOT_EQUAL");
+    }
+    ((TimelineKeyValueFilter)getCurrentFilter()).setCompareOp(
+        compareOp, keyMustExistFlag);
+  }
+
+  @Override
+  protected void setValueToCurrentFilter(Object value) {
+    TimelineFilter currentFilter = getCurrentFilter();
+    if (currentFilter != null) {
+      ((TimelineKeyValueFilter)currentFilter).setValue(value);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForNumericFilters.java
----------------------------------------------------------------------
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/TimelineParserForNumericFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForNumericFilters.java
new file mode 100644
index 0000000..7c14a9f
--- /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/TimelineParserForNumericFilters.java
@@ -0,0 +1,72 @@
+/**
+ * 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 java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+
+/**
+ * Used for parsing numerical filters such as metric filters.
+ */
+@Private
+@Unstable
+class TimelineParserForNumericFilters extends TimelineParserForCompareExpr {
+
+  public TimelineParserForNumericFilters(String expression) {
+    super(expression, "Metric Filter");
+  }
+
+  protected TimelineFilter createFilter() {
+    return new TimelineCompareFilter();
+  }
+
+  @Override
+  protected void setCompareOpToCurrentFilter(TimelineCompareOp compareOp,
+      boolean keyMustExistFlag) {
+    ((TimelineCompareFilter)getCurrentFilter()).setCompareOp(
+        compareOp, keyMustExistFlag);
+  }
+
+  protected Object parseValue(String strValue) throws TimelineParseException {
+    Object value = null;
+    try {
+      value = GenericObjectMapper.OBJECT_READER.readValue(strValue);
+    } catch (IOException e) {
+      throw new TimelineParseException("Value cannot be parsed.");
+    }
+    if (value == null || !(TimelineStorageUtils.isIntegralValue(value))) {
+      throw new TimelineParseException("Value is not a number.");
+    }
+    return value;
+  }
+
+  protected void setValueToCurrentFilter(Object value) {
+    TimelineFilter currentFilter = getCurrentFilter();
+    if (currentFilter != null) {
+      ((TimelineCompareFilter)currentFilter).setValue(value);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForRelationFilters.java
----------------------------------------------------------------------
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/TimelineParserForRelationFilters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineParserForRelationFilters.java
new file mode 100644
index 0000000..cde11e4
--- /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/TimelineParserForRelationFilters.java
@@ -0,0 +1,71 @@
+/**
+ * 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 java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
+
+/**
+ * Used for parsing relation filters.
+ */
+@Private
+@Unstable
+class TimelineParserForRelationFilters extends
+    TimelineParserForEqualityExpr {
+  private final String valueDelimiter;
+  public TimelineParserForRelationFilters(String expression, char valuesDelim,
+      String valueDelim) {
+    super(expression, "Relation Filter", valuesDelim);
+    valueDelimiter = valueDelim;
+  }
+
+  @Override
+  protected TimelineFilter createFilter() {
+    return new TimelineKeyValuesFilter();
+  }
+
+  @Override
+  protected void setCompareOpToCurrentFilter(TimelineCompareOp compareOp) {
+    ((TimelineKeyValuesFilter)getCurrentFilter()).setCompareOp(compareOp);
+  }
+
+  @Override
+  protected void setValueToCurrentFilter(String value)
+       throws TimelineParseException {
+    if (value != null) {
+      String[] pairStrs = value.split(valueDelimiter);
+      if (pairStrs.length < 2) {
+        throw new TimelineParseException("Invalid relation filter expression");
+      }
+      String key = pairStrs[0].trim();
+      Set<Object> values = new HashSet<Object>();
+      for (int i = 1; i < pairStrs.length; i++) {
+        values.add(pairStrs[i].trim());
+      }
+      ((TimelineKeyValuesFilter)getCurrentFilter()).
+          setKeyAndValues(key, values);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/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 0b9549b..6ec59ea 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
@@ -178,6 +178,9 @@ public class TimelineReaderWebServices {
           "Requested Invalid Field." : e.getMessage());
     } else if (e instanceof NotFoundException) {
       throw (NotFoundException)e;
+    } else if (e instanceof TimelineParseException) {
+      throw new BadRequestException(e.getMessage() == null ?
+          "Filter Parsing failed." : e.getMessage());
     } else if (e instanceof BadRequestException) {
       throw (BadRequestException)e;
     } else {
@@ -239,6 +242,14 @@ public class TimelineReaderWebServices {
    *     metricfilters=metricid1, metricid2... (Optional query param).
    * @param eventfilters If specified, matched entities should contain the given
    *     events. This is represented as eventfilters=eventid1, eventid2...
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the entity object to retrieve, see
    *     {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type, id and created time is returned
@@ -270,6 +281,8 @@ public class TimelineReaderWebServices {
       @QueryParam("conffilters") String conffilters,
       @QueryParam("metricfilters") String metricfilters,
       @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -295,7 +308,7 @@ public class TimelineReaderWebServices {
           limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
           infofilters, conffilters, metricfilters, eventfilters),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, null, fields));
+          confsToRetrieve, metricsToRetrieve, fields));
     } catch (Exception e) {
       handleException(e, url, startTime,
           "createdTime start/end or limit or flowrunid");
@@ -354,6 +367,14 @@ public class TimelineReaderWebServices {
    *     metricfilters=metricid1, metricid2... (Optional query param).
    * @param eventfilters If specified, matched entities should contain the given
    *     events. This is represented as eventfilters=eventid1, eventid2...
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the entity object to retrieve, see
    *     {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type, id, created time is returned
@@ -390,11 +411,13 @@ public class TimelineReaderWebServices {
       @QueryParam("conffilters") String conffilters,
       @QueryParam("metricfilters") String metricfilters,
       @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     return getEntities(req, res, null, appId, entityType, userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        fields);
+        confsToRetrieve, metricsToRetrieve, fields);
   }
 
   /**
@@ -443,6 +466,14 @@ public class TimelineReaderWebServices {
    *     metricfilters=metricid1, metricid2... (Optional query param).
    * @param eventfilters If specified, matched entities should contain the given
    *     events. This is represented as eventfilters=eventid1, eventid2...
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the entity object to retrieve, see
    *     {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type, id, created time is returned
@@ -480,6 +511,8 @@ public class TimelineReaderWebServices {
       @QueryParam("conffilters") String conffilters,
       @QueryParam("metricfilters") String metricfilters,
       @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -496,12 +529,11 @@ public class TimelineReaderWebServices {
       entities = timelineReaderManager.getEntities(
           TimelineReaderWebServicesUtils.createTimelineReaderContext(
           clusterId, userId, flowName, flowRunId, appId, entityType, null),
-
           TimelineReaderWebServicesUtils.createTimelineEntityFilters(
           limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
           infofilters, conffilters, metricfilters, eventfilters),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, null, fields));
+          confsToRetrieve, metricsToRetrieve, fields));
     } catch (Exception e) {
       handleException(e, url, startTime,
           "createdTime start/end or limit or flowrunid");
@@ -524,6 +556,14 @@ public class TimelineReaderWebServices {
    * @param uId a delimited string containing clusterid, userid, flow name,
    *     flowrun id, app id, entity type and entity id which are extracted from
    *     UID and then used to query backend(Mandatory path param).
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the entity object to retrieve, see
    *     {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type, id, created time is returned
@@ -546,6 +586,8 @@ public class TimelineReaderWebServices {
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("uid") String uId,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -566,7 +608,7 @@ public class TimelineReaderWebServices {
       }
       entity = timelineReaderManager.getEntity(context,
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, null, fields));
+          confsToRetrieve, metricsToRetrieve, fields));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -601,6 +643,14 @@ public class TimelineReaderWebServices {
    *     param).
    * @param flowRunId Run id which should match for the entity(Optional query
    *     param).
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the entity object to retrieve, see
    *     {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type, id, created time is returned
@@ -628,9 +678,11 @@ public class TimelineReaderWebServices {
       @QueryParam("userid") String userId,
       @QueryParam("flowname") String flowName,
       @QueryParam("flowrunid") String flowRunId,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     return getEntity(req, res, null, appId, entityType, entityId, userId,
-        flowName, flowRunId, fields);
+        flowName, flowRunId, confsToRetrieve, metricsToRetrieve, fields);
   }
 
   /**
@@ -653,6 +705,14 @@ public class TimelineReaderWebServices {
    *     param).
    * @param flowRunId Run id which should match for the entity(Optional query
    *     param).
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the entity object to retrieve, see
    *     {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type, id and created time is returned
@@ -681,6 +741,8 @@ public class TimelineReaderWebServices {
       @QueryParam("userid") String userId,
       @QueryParam("flowname") String flowName,
       @QueryParam("flowrunid") String flowRunId,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -698,7 +760,7 @@ public class TimelineReaderWebServices {
           TimelineReaderWebServicesUtils.createTimelineReaderContext(
           clusterId, userId, flowName, flowRunId, appId, entityType, entityId),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, null, fields));
+          confsToRetrieve, metricsToRetrieve, fields));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -723,6 +785,8 @@ public class TimelineReaderWebServices {
    * @param uId a delimited string containing clusterid, userid, flow name and
    *     flowrun id which are extracted from UID and then used to query backend
    *     (Mandatory path param).
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>FlowRunEntity</cite> instance is returned. By default, all
@@ -741,7 +805,8 @@ public class TimelineReaderWebServices {
   public TimelineEntity getFlowRun(
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
-      @PathParam("uid") String uId) {
+      @PathParam("uid") String uId,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -761,7 +826,8 @@ public class TimelineReaderWebServices {
       }
       context.setEntityType(TimelineEntityType.YARN_FLOW_RUN.toString());
       entity = timelineReaderManager.getEntity(context,
-          new TimelineDataToRetrieve());
+          TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
+          null, metricsToRetrieve, null));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -787,6 +853,8 @@ public class TimelineReaderWebServices {
    * @param flowName Flow name to which the flow run to be queried belongs to(
    *     Mandatory path param).
    * @param flowRunId Id of the flow run to be queried(Mandatory path param).
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>FlowRunEntity</cite> instance is returned. By default, all
@@ -807,8 +875,10 @@ public class TimelineReaderWebServices {
       @Context HttpServletResponse res,
       @PathParam("userid") String userId,
       @PathParam("flowname") String flowName,
-      @PathParam("flowrunid") String flowRunId) {
-    return getFlowRun(req, res, null, userId, flowName, flowRunId);
+      @PathParam("flowrunid") String flowRunId,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve) {
+    return getFlowRun(req, res, null, userId, flowName, flowRunId,
+        metricsToRetrieve);
   }
 
   /**
@@ -823,6 +893,8 @@ public class TimelineReaderWebServices {
    * @param flowName Flow name to which the flow run to be queried belongs to(
    *     Mandatory path param).
    * @param flowRunId Id of the flow run to be queried(Mandatory path param).
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response.
    *
    * @return If successful, a HTTP 200(OK) response having a JSON representing a
    *     <cite>FlowRunEntity</cite> instance is returned. By default, all
@@ -845,7 +917,8 @@ public class TimelineReaderWebServices {
       @PathParam("clusterid") String clusterId,
       @PathParam("userid") String userId,
       @PathParam("flowname") String flowName,
-      @PathParam("flowrunid") String flowRunId) {
+      @PathParam("flowrunid") String flowRunId,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
             QUERY_STRING_SEP + req.getQueryString());
@@ -862,7 +935,8 @@ public class TimelineReaderWebServices {
           TimelineReaderWebServicesUtils.createTimelineReaderContext(
           clusterId, userId, flowName, flowRunId, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
-          new TimelineDataToRetrieve());
+          TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
+          null, metricsToRetrieve, null));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -894,6 +968,10 @@ public class TimelineReaderWebServices {
    *     created before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched flow runs should not be created
    *     after this timestamp(Optional query param).
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields to retrieve, see {@link Field}.
    *     All fields will be retrieved if fields=ALL. Fields other than METRICS
    *     have no meaning for this REST endpoint. If not specified, all fields
@@ -918,6 +996,7 @@ public class TimelineReaderWebServices {
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -942,7 +1021,7 @@ public class TimelineReaderWebServices {
           limit, createdTimeStart, createdTimeEnd, null, null, null,
           null, null, null),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, null, fields));
+          null, metricsToRetrieve, fields));
     } catch (Exception e) {
       handleException(e, url, startTime, "createdTime start/end or limit");
     }
@@ -970,6 +1049,10 @@ public class TimelineReaderWebServices {
    *     created before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched flow runs should not be created
    *     after this timestamp(Optional query param).
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields to retrieve, see {@link Field}.
    *     All fields will be retrieved if fields=ALL. Fields other than METRICS
    *     have no meaning for this REST endpoint. If not specified, all fields
@@ -995,9 +1078,10 @@ public class TimelineReaderWebServices {
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     return getFlowRuns(req, res, null, userId, flowName, limit,
-        createdTimeStart, createdTimeEnd, fields);
+        createdTimeStart, createdTimeEnd, metricsToRetrieve, fields);
   }
 
   /**
@@ -1016,6 +1100,10 @@ public class TimelineReaderWebServices {
    *     created before this timestamp(Optional query param).
    * @param createdTimeEnd If specified, matched flow runs should not be created
    *     after this timestamp(Optional query param).
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields to retrieve, see {@link Field}.
    *     All fields will be retrieved if fields=ALL. Fields other than METRICS
    *     have no meaning for this REST endpoint. If not specified, all fields
@@ -1042,6 +1130,7 @@ public class TimelineReaderWebServices {
       @QueryParam("limit") String limit,
       @QueryParam("createdtimestart") String createdTimeStart,
       @QueryParam("createdtimeend") String createdTimeEnd,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -1063,7 +1152,7 @@ public class TimelineReaderWebServices {
           limit, createdTimeStart, createdTimeEnd, null, null, null,
           null, null, null),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, null, fields));
+          null, metricsToRetrieve, fields));
     } catch (Exception e) {
       handleException(e, url, startTime, "createdTime start/end or limit");
     }
@@ -1204,6 +1293,14 @@ public class TimelineReaderWebServices {
    * @param uId a delimited string containing clusterid, userid, flow name, flow
    *     run id and app id which are extracted from UID and then used to query
    *     backend(Mandatory path param).
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the app entity object to retrieve,
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
@@ -1226,6 +1323,8 @@ public class TimelineReaderWebServices {
       @Context HttpServletRequest req,
       @Context HttpServletResponse res,
       @PathParam("uid") String uId,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -1247,7 +1346,7 @@ public class TimelineReaderWebServices {
       context.setEntityType(TimelineEntityType.YARN_APPLICATION.toString());
       entity = timelineReaderManager.getEntity(context,
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, null, fields));
+          confsToRetrieve, metricsToRetrieve, fields));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -1277,6 +1376,14 @@ public class TimelineReaderWebServices {
    * @param flowRunId Run id which should match for the app(Optional query
    *     param).
    * @param userId User id which should match for the app(Optional query param).
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the app entity object to retrieve,
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
@@ -1302,8 +1409,11 @@ public class TimelineReaderWebServices {
       @QueryParam("flowname") String flowName,
       @QueryParam("flowrunid") String flowRunId,
       @QueryParam("userid") String userId,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
-    return getApp(req, res, null, appId, flowName, flowRunId, userId, fields);
+    return getApp(req, res, null, appId, flowName, flowRunId, userId,
+        confsToRetrieve, metricsToRetrieve, fields);
   }
 
   /**
@@ -1322,6 +1432,14 @@ public class TimelineReaderWebServices {
    * @param flowRunId Run id which should match for the app(Optional query
    *     param).
    * @param userId User id which should match for the app(Optional query param).
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the app entity object to retrieve,
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
@@ -1348,6 +1466,8 @@ public class TimelineReaderWebServices {
       @QueryParam("flowname") String flowName,
       @QueryParam("flowrunid") String flowRunId,
       @QueryParam("userid") String userId,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -1366,7 +1486,7 @@ public class TimelineReaderWebServices {
           clusterId, userId, flowName, flowRunId, appId,
           TimelineEntityType.YARN_APPLICATION.toString(), null),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, null, fields));
+          confsToRetrieve, metricsToRetrieve, fields));
     } catch (Exception e) {
       handleException(e, url, startTime, "flowrunid");
     }
@@ -1417,6 +1537,14 @@ public class TimelineReaderWebServices {
    *     (Optional query param).
    * @param eventfilters If specified, matched apps should contain the given
    *     events. This is represented as eventfilters=eventid1, eventid2...
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the app entity object to retrieve,
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
@@ -1447,6 +1575,8 @@ public class TimelineReaderWebServices {
       @QueryParam("conffilters") String conffilters,
       @QueryParam("metricfilters") String metricfilters,
       @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     String url = req.getRequestURI() +
         (req.getQueryString() == null ? "" :
@@ -1471,7 +1601,7 @@ public class TimelineReaderWebServices {
           limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
           infofilters, conffilters, metricfilters, eventfilters),
           TimelineReaderWebServicesUtils.createTimelineDataToRetrieve(
-          null, null, fields));
+          confsToRetrieve, metricsToRetrieve, fields));
     } catch (Exception e) {
       handleException(e, url, startTime,
           "createdTime start/end or limit or flowrunid");
@@ -1523,6 +1653,14 @@ public class TimelineReaderWebServices {
    *     (Optional query param).
    * @param eventfilters If specified, matched apps should contain the given
    *     events. This is represented as eventfilters=eventid1, eventid2...
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the app entity object to retrieve,
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
@@ -1555,12 +1693,14 @@ public class TimelineReaderWebServices {
       @QueryParam("conffilters") String conffilters,
       @QueryParam("metricfilters") String metricfilters,
       @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     return getEntities(req, res, null, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        fields);
+        confsToRetrieve, metricsToRetrieve, fields);
   }
 
   /**
@@ -1602,6 +1742,14 @@ public class TimelineReaderWebServices {
    *     (Optional query param).
    * @param eventfilters If specified, matched apps should contain the given
    *     events. This is represented as eventfilters=eventid1, eventid2...
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the app entity object to retrieve,
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
@@ -1636,12 +1784,14 @@ public class TimelineReaderWebServices {
       @QueryParam("conffilters") String conffilters,
       @QueryParam("metricfilters") String metricfilters,
       @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     return getEntities(req, res, clusterId, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         flowRunId, limit, createdTimeStart, createdTimeEnd, relatesTo,
         isRelatedTo, infofilters, conffilters, metricfilters, eventfilters,
-        fields);
+        confsToRetrieve, metricsToRetrieve, fields);
   }
 
   /**
@@ -1680,6 +1830,14 @@ public class TimelineReaderWebServices {
    *     (Optional query param).
    * @param eventfilters If specified, matched apps should contain the given
    *     events. This is represented as eventfilters=eventid1, eventid2...
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the app entity object to retrieve,
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
@@ -1711,11 +1869,14 @@ public class TimelineReaderWebServices {
       @QueryParam("conffilters") String conffilters,
       @QueryParam("metricfilters") String metricfilters,
       @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     return getEntities(req, res, null, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         null, limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
-        infofilters, conffilters, metricfilters, eventfilters, fields);
+        infofilters, conffilters, metricfilters, eventfilters,
+        confsToRetrieve, metricsToRetrieve, fields);
   }
 
   /**
@@ -1756,6 +1917,14 @@ public class TimelineReaderWebServices {
    *     (Optional query param).
    * @param eventfilters If specified, matched apps should contain the given
    *     events. This is represented as eventfilters=eventid1, eventid2...
+   * @param confsToRetrieve If specified, defines which configurations to
+   *     retrieve and send back in response. These configs will be retrieved
+   *     irrespective of whether configs are specified in fields to retrieve or
+   *     not.
+   * @param metricsToRetrieve If specified, defines which metrics to retrieve
+   *     and send back in response. These metrics will be retrieved
+   *     irrespective of whether metrics are specified in fields to retrieve or
+   *     not.
    * @param fields Specifies which fields of the app entity object to retrieve,
    *     see {@link Field}. All fields will be retrieved if fields=ALL. If not
    *     specified, 3 fields i.e. entity type(equivalent to YARN_APPLICATION),
@@ -1788,10 +1957,13 @@ public class TimelineReaderWebServices {
       @QueryParam("conffilters") String conffilters,
       @QueryParam("metricfilters") String metricfilters,
       @QueryParam("eventfilters") String eventfilters,
+      @QueryParam("confstoretrieve") String confsToRetrieve,
+      @QueryParam("metricstoretrieve") String metricsToRetrieve,
       @QueryParam("fields") String fields) {
     return getEntities(req, res, clusterId, null,
         TimelineEntityType.YARN_APPLICATION.toString(), userId, flowName,
         null, limit, createdTimeStart, createdTimeEnd, relatesTo, isRelatedTo,
-        infofilters, conffilters, metricfilters, eventfilters, fields);
+        infofilters, conffilters, metricfilters, eventfilters,
+        confsToRetrieve, metricsToRetrieve, fields);
   }
 }
\ No newline at end of file


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


[49/50] [abbrv] hadoop git commit: YARN-5070. upgrade HBase version for first merge (Vrushali C via sjlee)

Posted by vr...@apache.org.
YARN-5070. upgrade HBase version for first merge (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/d15654c3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d15654c3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d15654c3

Branch: refs/heads/YARN-2928
Commit: d15654c34ab9f4ad88f6a1e57883fd71c5320296
Parents: a8779be
Author: Sangjin Lee <sj...@apache.org>
Authored: Wed Jun 15 11:43:36 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:16 2016 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |   4 +-
 .../storage/flow/TestHBaseStorageFlowRun.java   | 168 ++++++++++++++++++-
 .../flow/TestHBaseStorageFlowRunCompaction.java | 159 +++++++++++++++++-
 .../storage/flow/FlowRunCoprocessor.java        |  17 +-
 .../storage/flow/FlowScanner.java               | 132 ++++++++-------
 5 files changed, 392 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d15654c3/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 0a4f058..4d064e8 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -49,8 +49,8 @@
     <xerces.jdiff.version>2.11.0</xerces.jdiff.version>
 
     <kafka.version>0.8.2.1</kafka.version>
-    <hbase.version>1.0.1</hbase.version>
-    <phoenix.version>4.5.0-SNAPSHOT</phoenix.version>
+    <hbase.version>1.1.3</hbase.version>
+    <phoenix.version>4.7.0-HBase-1.1</phoenix.version>
     <hbase-compatible-hadoop.version>2.5.1</hbase-compatible-hadoop.version>
 
     <hadoop.assemblies.version>${project.version}</hadoop.assemblies.version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d15654c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
index 328b25a..6c4c810 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.api.records.timelineservice.FlowRunEntity;
@@ -107,8 +107,8 @@ public class TestHBaseStorageFlowRun {
       // check in flow run table
       util.waitUntilAllRegionsAssigned(table);
       HRegionServer server = util.getRSForFirstRegionInTable(table);
-      List<HRegion> regions = server.getOnlineRegions(table);
-      for (HRegion region : regions) {
+      List<Region> regions = server.getOnlineRegions(table);
+      for (Region region : regions) {
         assertTrue(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
             hbaseConf));
       }
@@ -122,8 +122,8 @@ public class TestHBaseStorageFlowRun {
       // check in flow activity table
       util.waitUntilAllRegionsAssigned(table);
       HRegionServer server = util.getRSForFirstRegionInTable(table);
-      List<HRegion> regions = server.getOnlineRegions(table);
-      for (HRegion region : regions) {
+      List<Region> regions = server.getOnlineRegions(table);
+      for (Region region : regions) {
         assertFalse(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
             hbaseConf));
       }
@@ -137,8 +137,8 @@ public class TestHBaseStorageFlowRun {
       // check in entity run table
       util.waitUntilAllRegionsAssigned(table);
       HRegionServer server = util.getRSForFirstRegionInTable(table);
-      List<HRegion> regions = server.getOnlineRegions(table);
-      for (HRegion region : regions) {
+      List<Region> regions = server.getOnlineRegions(table);
+      for (Region region : regions) {
         assertFalse(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
             hbaseConf));
       }
@@ -311,6 +311,9 @@ public class TestHBaseStorageFlowRun {
     // check flow run
     checkFlowRunTable(cluster, user, flow, runid, c1);
 
+    // check various batch limits in scanning the table for this flow
+    checkFlowRunTableBatchLimit(cluster, user, flow, runid, c1);
+
     // use the timeline reader to verify data
     HBaseTimelineReaderImpl hbr = null;
     try {
@@ -350,6 +353,157 @@ public class TestHBaseStorageFlowRun {
     }
   }
 
+  /*
+   * checks the batch limits on a scan
+   */
+   void checkFlowRunTableBatchLimit(String cluster, String user,
+      String flow, long runid, Configuration c1) throws IOException {
+
+    Scan s = new Scan();
+    s.addFamily(FlowRunColumnFamily.INFO.getBytes());
+    byte[] startRow =  new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
+    s.setStartRow(startRow);
+    // set a batch limit
+    int batchLimit = 2;
+    s.setBatch(batchLimit);
+    String clusterStop = cluster + "1";
+    byte[] stopRow = new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey();
+    s.setStopRow(stopRow);
+    Connection conn = ConnectionFactory.createConnection(c1);
+    Table table1 = conn
+        .getTable(TableName.valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
+    ResultScanner scanner = table1.getScanner(s);
+
+    int loopCount = 0;
+    for (Result result : scanner) {
+      assertNotNull(result);
+      assertTrue(!result.isEmpty());
+      assertTrue(result.rawCells().length <= batchLimit);
+      Map<byte[], byte[]> values = result
+          .getFamilyMap(FlowRunColumnFamily.INFO.getBytes());
+      assertNotNull(values);
+      assertTrue(values.size() <= batchLimit);
+      loopCount++;
+    }
+    assertTrue(loopCount > 0);
+
+    // test with a diff batch limit
+    s = new Scan();
+    s.addFamily(FlowRunColumnFamily.INFO.getBytes());
+    s.setStartRow(startRow);
+    // set a batch limit
+    batchLimit = 1;
+    s.setBatch(batchLimit);
+    s.setMaxResultsPerColumnFamily(2);
+    s.setStopRow(stopRow);
+    scanner = table1.getScanner(s);
+
+    loopCount = 0;
+    for (Result result : scanner) {
+      assertNotNull(result);
+      assertTrue(!result.isEmpty());
+      assertEquals(batchLimit, result.rawCells().length);
+      Map<byte[], byte[]> values = result
+          .getFamilyMap(FlowRunColumnFamily.INFO.getBytes());
+      assertNotNull(values);
+      assertEquals(batchLimit, values.size());
+      loopCount++;
+    }
+    assertTrue(loopCount > 0);
+
+    // test with a diff batch limit
+    // set it high enough
+    // we expect back 3 since there are
+    // column = m!HDFS_BYTES_READ value=57
+    // column = m!MAP_SLOT_MILLIS value=141
+    // column min_start_time value=1425016501000
+    s = new Scan();
+    s.addFamily(FlowRunColumnFamily.INFO.getBytes());
+    s.setStartRow(startRow);
+    // set a batch limit
+    batchLimit = 100;
+    s.setBatch(batchLimit);
+    s.setStopRow(stopRow);
+    scanner = table1.getScanner(s);
+
+    loopCount = 0;
+    for (Result result : scanner) {
+      assertNotNull(result);
+      assertTrue(!result.isEmpty());
+      assertTrue(result.rawCells().length <= batchLimit);
+      Map<byte[], byte[]> values = result
+          .getFamilyMap(FlowRunColumnFamily.INFO.getBytes());
+      assertNotNull(values);
+      // assert that with every next invocation
+      // we get back <= batchLimit values
+      assertTrue(values.size() <= batchLimit);
+      assertTrue(values.size() == 3); // see comment above
+      loopCount++;
+    }
+    // should loop through only once
+    assertTrue(loopCount == 1);
+
+    // set it to a negative number
+    // we expect all 3 back since there are
+    // column = m!HDFS_BYTES_READ value=57
+    // column = m!MAP_SLOT_MILLIS value=141
+    // column min_start_time value=1425016501000
+    s = new Scan();
+    s.addFamily(FlowRunColumnFamily.INFO.getBytes());
+    s.setStartRow(startRow);
+    // set a batch limit
+    batchLimit = -671;
+    s.setBatch(batchLimit);
+    s.setStopRow(stopRow);
+    scanner = table1.getScanner(s);
+
+    loopCount = 0;
+    for (Result result : scanner) {
+      assertNotNull(result);
+      assertTrue(!result.isEmpty());
+      assertEquals(3, result.rawCells().length);
+      Map<byte[], byte[]> values = result
+          .getFamilyMap(FlowRunColumnFamily.INFO.getBytes());
+      assertNotNull(values);
+      // assert that with every next invocation
+      // we get back <= batchLimit values
+      assertEquals(3, values.size());
+      loopCount++;
+    }
+    // should loop through only once
+    assertEquals(1, loopCount);
+
+    // set it to 0
+    // we expect all 3 back since there are
+    // column = m!HDFS_BYTES_READ value=57
+    // column = m!MAP_SLOT_MILLIS value=141
+    // column min_start_time value=1425016501000
+    s = new Scan();
+    s.addFamily(FlowRunColumnFamily.INFO.getBytes());
+    s.setStartRow(startRow);
+    // set a batch limit
+    batchLimit = 0;
+    s.setBatch(batchLimit);
+    s.setStopRow(stopRow);
+    scanner = table1.getScanner(s);
+
+    loopCount = 0;
+    for (Result result : scanner) {
+      assertNotNull(result);
+      assertTrue(!result.isEmpty());
+      assertEquals(3, result.rawCells().length);
+      Map<byte[], byte[]> values = result
+          .getFamilyMap(FlowRunColumnFamily.INFO.getBytes());
+      assertNotNull(values);
+      // assert that with every next invocation
+      // we get back <= batchLimit values
+      assertEquals(3, values.size());
+      loopCount++;
+    }
+    // should loop through only once
+    assertEquals(1, loopCount);
+  }
+
   private void checkFlowRunTable(String cluster, String user, String flow,
       long runid, Configuration c1) throws IOException {
     Scan s = new Scan();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d15654c3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
index e1bef53..71523b8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
@@ -45,7 +45,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.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -124,6 +124,153 @@ public class TestHBaseStorageFlowRunCompaction {
   }
 
   @Test
+  public void testWriteScanBatchLimit() throws Exception {
+    String rowKey = "nonNumericRowKey";
+    String column = "nonNumericColumnName";
+    String value = "nonNumericValue";
+    String column2 = "nonNumericColumnName2";
+    String value2 = "nonNumericValue2";
+    String column3 = "nonNumericColumnName3";
+    String value3 = "nonNumericValue3";
+    String column4 = "nonNumericColumnName4";
+    String value4 = "nonNumericValue4";
+
+    byte[] rowKeyBytes = Bytes.toBytes(rowKey);
+    byte[] columnNameBytes = Bytes.toBytes(column);
+    byte[] valueBytes = Bytes.toBytes(value);
+    byte[] columnName2Bytes = Bytes.toBytes(column2);
+    byte[] value2Bytes = Bytes.toBytes(value2);
+    byte[] columnName3Bytes = Bytes.toBytes(column3);
+    byte[] value3Bytes = Bytes.toBytes(value3);
+    byte[] columnName4Bytes = Bytes.toBytes(column4);
+    byte[] value4Bytes = Bytes.toBytes(value4);
+
+    Put p = new Put(rowKeyBytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnNameBytes,
+        valueBytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName2Bytes,
+        value2Bytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName3Bytes,
+        value3Bytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName4Bytes,
+        value4Bytes);
+
+    Configuration hbaseConf = util.getConfiguration();
+    TableName table = TableName.valueOf(hbaseConf.get(
+        FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+    Connection conn = null;
+    conn = ConnectionFactory.createConnection(hbaseConf);
+    Table flowRunTable = conn.getTable(table);
+    flowRunTable.put(p);
+
+    String rowKey2 = "nonNumericRowKey2";
+    byte[] rowKey2Bytes = Bytes.toBytes(rowKey2);
+    p = new Put(rowKey2Bytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnNameBytes,
+        valueBytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName2Bytes,
+        value2Bytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName3Bytes,
+        value3Bytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName4Bytes,
+        value4Bytes);
+    flowRunTable.put(p);
+
+    String rowKey3 = "nonNumericRowKey3";
+    byte[] rowKey3Bytes = Bytes.toBytes(rowKey3);
+    p = new Put(rowKey3Bytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnNameBytes,
+        valueBytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName2Bytes,
+        value2Bytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName3Bytes,
+        value3Bytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnName4Bytes,
+        value4Bytes);
+    flowRunTable.put(p);
+
+    Scan s = new Scan();
+    s.addFamily(FlowRunColumnFamily.INFO.getBytes());
+    s.setStartRow(rowKeyBytes);
+    // set number of cells to fetch per scanner next invocation
+    int batchLimit = 2;
+    s.setBatch(batchLimit);
+    ResultScanner scanner = flowRunTable.getScanner(s);
+    for (Result result : scanner) {
+      assertNotNull(result);
+      assertTrue(!result.isEmpty());
+      assertTrue(result.rawCells().length <= batchLimit);
+      Map<byte[], byte[]> values = result
+          .getFamilyMap(FlowRunColumnFamily.INFO.getBytes());
+      assertTrue(values.size() <= batchLimit);
+    }
+
+    s = new Scan();
+    s.addFamily(FlowRunColumnFamily.INFO.getBytes());
+    s.setStartRow(rowKeyBytes);
+    // set number of cells to fetch per scanner next invocation
+    batchLimit = 3;
+    s.setBatch(batchLimit);
+    scanner = flowRunTable.getScanner(s);
+    for (Result result : scanner) {
+      assertNotNull(result);
+      assertTrue(!result.isEmpty());
+      assertTrue(result.rawCells().length <= batchLimit);
+      Map<byte[], byte[]> values = result
+          .getFamilyMap(FlowRunColumnFamily.INFO.getBytes());
+      assertTrue(values.size() <= batchLimit);
+    }
+
+    s = new Scan();
+    s.addFamily(FlowRunColumnFamily.INFO.getBytes());
+    s.setStartRow(rowKeyBytes);
+    // set number of cells to fetch per scanner next invocation
+    batchLimit = 1000;
+    s.setBatch(batchLimit);
+    scanner = flowRunTable.getScanner(s);
+    int rowCount = 0;
+    for (Result result : scanner) {
+      assertNotNull(result);
+      assertTrue(!result.isEmpty());
+      assertTrue(result.rawCells().length <= batchLimit);
+      Map<byte[], byte[]> values = result
+          .getFamilyMap(FlowRunColumnFamily.INFO.getBytes());
+      assertTrue(values.size() <= batchLimit);
+      // we expect all back in one next call
+      assertEquals(4, values.size());
+      rowCount++;
+    }
+    // should get back 1 row with each invocation
+    // if scan batch is set sufficiently high
+    assertEquals(3, rowCount);
+
+    // test with a negative number
+    // should have same effect as setting it to a high number
+    s = new Scan();
+    s.addFamily(FlowRunColumnFamily.INFO.getBytes());
+    s.setStartRow(rowKeyBytes);
+    // set number of cells to fetch per scanner next invocation
+    batchLimit = -2992;
+    s.setBatch(batchLimit);
+    scanner = flowRunTable.getScanner(s);
+    rowCount = 0;
+    for (Result result : scanner) {
+      assertNotNull(result);
+      assertTrue(!result.isEmpty());
+      assertEquals(4, result.rawCells().length);
+      Map<byte[], byte[]> values = result
+          .getFamilyMap(FlowRunColumnFamily.INFO.getBytes());
+      // we expect all back in one next call
+      assertEquals(4, values.size());
+      System.out.println(" values size " + values.size() +  " " + batchLimit );
+      rowCount++;
+    }
+    // should get back 1 row with each invocation
+    // if scan batch is set sufficiently high
+    assertEquals(3, rowCount);
+  }
+
+  @Test
   public void testWriteFlowRunCompaction() throws Exception {
     String cluster = "kompaction_cluster1";
     String user = "kompaction_FlowRun__user1";
@@ -176,13 +323,13 @@ public class TestHBaseStorageFlowRunCompaction {
     // check in flow run table
     HRegionServer server = util.getRSForFirstRegionInTable(TableName
         .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
-    List<HRegion> regions = server.getOnlineRegions(TableName
+    List<Region> regions = server.getOnlineRegions(TableName
         .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
     assertTrue("Didn't find any regions for primary table!", regions.size() > 0);
     // flush and compact all the regions of the primary table
-    for (HRegion region : regions) {
-       region.flushcache();
-      region.compactStores(true);
+    for (Region region : regions) {
+       region.flush(true);
+       region.compact(true);
     }
 
     // check flow run for one flow many apps
@@ -237,7 +384,7 @@ public class TestHBaseStorageFlowRunCompaction {
     request.setIsMajor(true, true);
     // okay to pass in nulls for the constructor arguments
     // because all we want to do is invoke the process summation
-    FlowScanner fs = new FlowScanner(null, -1, null,
+    FlowScanner fs = new FlowScanner(null, null,
         (request.isMajor() == true ? FlowScannerOperation.MAJOR_COMPACTION
             : FlowScannerOperation.MINOR_COMPACTION));
     assertNotNull(fs);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d15654c3/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 8ea51a1..a9dcfaa 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
@@ -39,7 +39,7 @@ 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.Region;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.ScanType;
@@ -59,7 +59,7 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
   private static final Log LOG = LogFactory.getLog(FlowRunCoprocessor.class);
   private boolean isFlowRunRegion = false;
 
-  private HRegion region;
+  private Region region;
   /**
    * generate a timestamp that is unique per row in a region this is per region.
    */
@@ -178,7 +178,7 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
     scan.setMaxVersions();
     RegionScanner scanner = null;
     try {
-      scanner = new FlowScanner(e.getEnvironment(), scan.getBatch(),
+      scanner = new FlowScanner(e.getEnvironment(), scan,
           region.getScanner(scan), FlowScannerOperation.READ);
       scanner.next(results);
       e.bypass();
@@ -233,7 +233,7 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
     if (!isFlowRunRegion) {
       return scanner;
     }
-    return new FlowScanner(e.getEnvironment(), scan.getBatch(),
+    return new FlowScanner(e.getEnvironment(), scan,
         scanner, FlowScannerOperation.READ);
   }
 
@@ -257,7 +257,7 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
             + " storeFilesCount=" + store.getStorefilesCount());
       }
     }
-    return new FlowScanner(c.getEnvironment(), -1, scanner,
+    return new FlowScanner(c.getEnvironment(), scanner,
         FlowScannerOperation.FLUSH);
   }
 
@@ -296,10 +296,9 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
       requestOp = (request.isMajor() ? FlowScannerOperation.MAJOR_COMPACTION
           : FlowScannerOperation.MINOR_COMPACTION);
       LOG.info("Compactionrequest= " + request.toString() + " "
-          + requestOp.toString() + " RegionName="
-          + e.getEnvironment().getRegion().getRegionNameAsString());
+          + requestOp.toString() + " RegionName=" + e.getEnvironment()
+              .getRegion().getRegionInfo().getRegionNameAsString());
     }
-
-    return new FlowScanner(e.getEnvironment(), -1, scanner, requestOp);
+    return new FlowScanner(e.getEnvironment(), scanner, requestOp);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d15654c3/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 648c77b..6e67722 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
@@ -35,10 +35,12 @@ 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.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -68,9 +70,9 @@ class FlowScanner implements RegionScanner, Closeable {
    */
   private static final String FLOW_APP_ID = "application_00000000000_0000";
 
-  private final HRegion region;
+  private final Region region;
   private final InternalScanner flowRunScanner;
-  private final int limit;
+  private final int batchSize;
   private final long appFinalValueRetentionThreshold;
   private RegionScanner regionScanner;
   private boolean hasMore;
@@ -79,9 +81,15 @@ class FlowScanner implements RegionScanner, Closeable {
   private int currentIndex;
   private FlowScannerOperation action = FlowScannerOperation.READ;
 
-  FlowScanner(RegionCoprocessorEnvironment env, int limit,
+  FlowScanner(RegionCoprocessorEnvironment env, InternalScanner internalScanner,
+      FlowScannerOperation action) {
+    this(env, null, internalScanner, action);
+  }
+
+  FlowScanner(RegionCoprocessorEnvironment env, Scan incomingScan,
       InternalScanner internalScanner, FlowScannerOperation action) {
-    this.limit = limit;
+    this.batchSize = incomingScan == null ? -1 : incomingScan.getBatch();
+    // TODO initialize other scan attributes like Scan#maxResultSize
     this.flowRunScanner = internalScanner;
     if (internalScanner instanceof RegionScanner) {
       this.regionScanner = (RegionScanner) internalScanner;
@@ -98,8 +106,12 @@ class FlowScanner implements RegionScanner, Closeable {
           YarnConfiguration.APP_FINAL_VALUE_RETENTION_THRESHOLD,
           YarnConfiguration.DEFAULT_APP_FINAL_VALUE_RETENTION_THRESHOLD);
     }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(" batch size=" + batchSize);
+    }
   }
 
+
   /*
    * (non-Javadoc)
    *
@@ -112,22 +124,24 @@ class FlowScanner implements RegionScanner, Closeable {
 
   @Override
   public boolean nextRaw(List<Cell> cells) throws IOException {
-    return nextRaw(cells, limit);
+    return nextRaw(cells, ScannerContext.newBuilder().build());
   }
 
   @Override
-  public boolean nextRaw(List<Cell> cells, int cellLimit) throws IOException {
-    return nextInternal(cells, cellLimit);
+  public boolean nextRaw(List<Cell> cells, ScannerContext scannerContext)
+      throws IOException {
+    return nextInternal(cells, scannerContext);
   }
 
   @Override
   public boolean next(List<Cell> cells) throws IOException {
-    return next(cells, limit);
+    return next(cells, ScannerContext.newBuilder().build());
   }
 
   @Override
-  public boolean next(List<Cell> cells, int cellLimit) throws IOException {
-    return nextInternal(cells, cellLimit);
+  public boolean next(List<Cell> cells, ScannerContext scannerContext)
+      throws IOException {
+    return nextInternal(cells, scannerContext);
   }
 
   /**
@@ -159,29 +173,17 @@ class FlowScanner implements RegionScanner, Closeable {
   }
 
   /**
-   * 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
    * column or returns the cell as is.
    *
    * @param cells
-   * @param cellLimit
+   * @param scannerContext
    * @return true if next row is available for the scanner, false otherwise
    * @throws IOException
    */
-  @SuppressWarnings("deprecation")
-  private boolean nextInternal(List<Cell> cells, int cellLimit)
+  private boolean nextInternal(List<Cell> cells, ScannerContext scannerContext)
       throws IOException {
     Cell cell = null;
     startNext();
@@ -194,48 +196,47 @@ 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 = Separator.EMPTY_BYTES;
+    byte[] previousColumnQualifier = Separator.EMPTY_BYTES;
     AggregationOperation currentAggOp = null;
     SortedSet<Cell> currentColumnCells = new TreeSet<>(KeyValue.COMPARATOR);
     Set<String> alreadySeenAggDim = new HashSet<>();
     int addedCnt = 0;
     long currentTimestamp = System.currentTimeMillis();
     ValueConverter converter = null;
+    int limit = batchSize;
 
-    while (cellLimit <= 0 || addedCnt < cellLimit) {
-      cell = peekAtNextCell(cellLimit);
+    while (limit <= 0 || addedCnt < limit) {
+      cell = peekAtNextCell(scannerContext);
       if (cell == null) {
         break;
       }
-      byte[] newColumnQualifier = CellUtil.cloneQualifier(cell);
-      if (comp.compare(currentColumnQualifier, newColumnQualifier) != 0) {
-        if (converter != null && isNumericConverter(converter)) {
-          addedCnt += emitCells(cells, currentColumnCells, currentAggOp,
-              converter, currentTimestamp);
-        }
+      byte[] currentColumnQualifier = CellUtil.cloneQualifier(cell);
+      if (previousColumnQualifier == null) {
+        // first time in loop
+        previousColumnQualifier = currentColumnQualifier;
+      }
+
+      converter = getValueConverter(currentColumnQualifier);
+      if (comp.compare(previousColumnQualifier, currentColumnQualifier) != 0) {
+        addedCnt += emitCells(cells, currentColumnCells, currentAggOp,
+            converter, currentTimestamp);
         resetState(currentColumnCells, alreadySeenAggDim);
-        currentColumnQualifier = newColumnQualifier;
+        previousColumnQualifier = currentColumnQualifier;
         currentAggOp = getCurrentAggOp(cell);
-        converter = getValueConverter(newColumnQualifier);
-      }
-      // No operation needs to be performed on non numeric converters.
-      if (!isNumericConverter(converter)) {
-        currentColumnCells.add(cell);
-        nextCell(cellLimit);
-        continue;
+        converter = getValueConverter(currentColumnQualifier);
       }
       collectCells(currentColumnCells, currentAggOp, cell, alreadySeenAggDim,
-          (NumericValueConverter)converter);
-      nextCell(cellLimit);
+          converter, scannerContext);
+      nextCell(scannerContext);
     }
-    if (!currentColumnCells.isEmpty()) {
-      addedCnt += emitCells(cells, currentColumnCells, currentAggOp,
-          converter, currentTimestamp);
+    if ((!currentColumnCells.isEmpty()) && ((limit <= 0 || addedCnt < limit))) {
+      addedCnt += emitCells(cells, currentColumnCells, currentAggOp, converter,
+          currentTimestamp);
       if (LOG.isDebugEnabled()) {
         if (addedCnt > 0) {
           LOG.debug("emitted cells. " + addedCnt + " for " + this.action
               + " rowKey="
-              + FlowRunRowKey.parseRowKey(cells.get(0).getRow()).toString());
+              + FlowRunRowKey.parseRowKey(CellUtil.cloneRow(cells.get(0))));
         } else {
           LOG.debug("emitted no cells for " + this.action);
         }
@@ -252,7 +253,7 @@ class FlowScanner implements RegionScanner, Closeable {
   }
 
   /**
-   * resets the parameters to an intialized state for next loop iteration.
+   * resets the parameters to an initialized state for next loop iteration.
    *
    * @param cell
    * @param currentAggOp
@@ -268,12 +269,12 @@ class FlowScanner implements RegionScanner, Closeable {
 
   private void collectCells(SortedSet<Cell> currentColumnCells,
       AggregationOperation currentAggOp, Cell cell,
-      Set<String> alreadySeenAggDim, NumericValueConverter converter)
-      throws IOException {
+      Set<String> alreadySeenAggDim, ValueConverter converter,
+      ScannerContext scannerContext) throws IOException {
+
     if (currentAggOp == null) {
       // not a min/max/metric cell, so just return it as is
       currentColumnCells.add(cell);
-      nextCell(limit);
       return;
     }
 
@@ -284,7 +285,7 @@ class FlowScanner implements RegionScanner, Closeable {
       } else {
         Cell currentMinCell = currentColumnCells.first();
         Cell newMinCell = compareCellValues(currentMinCell, cell, currentAggOp,
-            converter);
+            (NumericValueConverter) converter);
         if (!currentMinCell.equals(newMinCell)) {
           currentColumnCells.remove(currentMinCell);
           currentColumnCells.add(newMinCell);
@@ -297,7 +298,7 @@ class FlowScanner implements RegionScanner, Closeable {
       } else {
         Cell currentMaxCell = currentColumnCells.first();
         Cell newMaxCell = compareCellValues(currentMaxCell, cell, currentAggOp,
-            converter);
+            (NumericValueConverter) converter);
         if (!currentMaxCell.equals(newMaxCell)) {
           currentColumnCells.remove(currentMaxCell);
           currentColumnCells.add(newMaxCell);
@@ -610,15 +611,14 @@ class FlowScanner implements RegionScanner, Closeable {
    * pointer to the next cell. This method can be called multiple times in a row
    * to advance through all the available cells.
    *
-   * @param cellLimit
-   *          the limit of number of cells to return if the next batch must be
-   *          fetched by the wrapped scanner
+   * @param scannerContext
+   *          context information for the batch of cells under consideration
    * @return the next available cell or null if no more cells are available for
    *         the current row
    * @throws IOException
    */
-  public Cell nextCell(int cellLimit) throws IOException {
-    Cell cell = peekAtNextCell(cellLimit);
+  public Cell nextCell(ScannerContext scannerContext) throws IOException {
+    Cell cell = peekAtNextCell(scannerContext);
     if (cell != null) {
       currentIndex++;
     }
@@ -630,20 +630,19 @@ class FlowScanner implements RegionScanner, Closeable {
    * pointer. Calling this method multiple times in a row will continue to
    * return the same cell.
    *
-   * @param cellLimit
-   *          the limit of number of cells to return if the next batch must be
-   *          fetched by the wrapped scanner
+   * @param scannerContext
+   *          context information for the batch of cells under consideration
    * @return the next available cell or null if no more cells are available for
    *         the current row
    * @throws IOException if any problem is encountered while grabbing the next
    *     cell.
    */
-  public Cell peekAtNextCell(int cellLimit) throws IOException {
+  public Cell peekAtNextCell(ScannerContext scannerContext) throws IOException {
     if (currentIndex >= availableCells.size()) {
       // done with current batch
       availableCells.clear();
       currentIndex = 0;
-      hasMore = flowRunScanner.next(availableCells, cellLimit);
+      hasMore = flowRunScanner.next(availableCells, scannerContext);
     }
     Cell cell = null;
     if (currentIndex < availableCells.size()) {
@@ -720,4 +719,9 @@ class FlowScanner implements RegionScanner, Closeable {
     }
     return regionScanner.reseek(bytes);
   }
+
+  @Override
+  public int getBatch() {
+    return batchSize;
+  }
 }


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


[50/50] [abbrv] hadoop git commit: Cleanup changes during rebase with trunk (Vrushali C)

Posted by vr...@apache.org.
Cleanup changes during rebase with trunk (Vrushali C)


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

Branch: refs/heads/YARN-2928
Commit: 92faa01524e99ec1d981a0b04b0e06746f15f0b2
Parents: d15654c
Author: Vrushali <vr...@twitter.com>
Authored: Tue Jun 21 16:38:20 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Tue Jun 21 16:38:20 2016 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |  5 -----
 hadoop-yarn-project/hadoop-yarn/bin/yarn        | 11 +++++-----
 .../nodemanager/NodeStatusUpdaterImpl.java      |  1 -
 .../containermanager/ContainerManagerImpl.java  |  6 ++----
 .../application/ApplicationImpl.java            | 22 +++++++++-----------
 .../metrics/TimelineServiceV1Publisher.java     | 12 +++++++++++
 .../metrics/TimelineServiceV2Publisher.java     | 16 +++++++++++++-
 .../pom.xml                                     |  4 ++--
 8 files changed, 46 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/92faa015/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 4d064e8..e4c983c 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -1117,11 +1117,6 @@
         <scope>test</scope>
         <optional>true</optional>
         <exclusions>
-          <!-- exclude jdk.tools (1.7) as we're not managing it -->
-          <exclusion>
-            <groupId>jdk.tools</groupId>
-            <artifactId>jdk.tools</artifactId>
-          </exclusion>
           <exclusion>
             <groupId>org.jruby</groupId>
             <artifactId>jruby-complete</artifactId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92faa015/hadoop-yarn-project/hadoop-yarn/bin/yarn
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/bin/yarn b/hadoop-yarn-project/hadoop-yarn/bin/yarn
index 42f2c24..409f47c 100755
--- a/hadoop-yarn-project/hadoop-yarn/bin/yarn
+++ b/hadoop-yarn-project/hadoop-yarn/bin/yarn
@@ -48,11 +48,10 @@ function hadoop_usage
   hadoop_add_subcommand "rmadmin" "admin tools"
   hadoop_add_subcommand "scmadmin" "SharedCacheManager admin tools"
   hadoop_add_subcommand "sharedcachemanager" "run the SharedCacheManager daemon"
-  hadoop_add_subcommand "timelineserver" "run the timeline server"
   hadoop_add_subcommand "timelinereader" "run the timeline reader server"
+  hadoop_add_subcommand "timelineserver" "run the timeline server"
   hadoop_add_subcommand "top" "view cluster information"
   hadoop_add_subcommand "version" "print the version"
-  hadoop_generate_usage "${MYNAME}" true
   hadoop_generate_usage "${HADOOP_SHELL_EXECNAME}" true
 }
 
@@ -173,6 +172,10 @@ function yarncmd_case
       hadoop_debug "Append YARN_SHAREDCACHEMANAGER_OPTS onto HADOOP_OPTS"
       HADOOP_OPTS="${HADOOP_OPTS} ${YARN_SHAREDCACHEMANAGER_OPTS}"
     ;;
+    timelinereader)
+      HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
+      HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer'
+    ;;
     timelineserver)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
       HADOOP_CLASSNAME='org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer'
@@ -184,10 +187,6 @@ function yarncmd_case
         HADOOP_HEAPSIZE_MAX="${YARN_TIMELINESERVER_HEAPSIZE}"
       fi
     ;;
-    timelinereader)
-      supportdaemonization="true"
-      CLASS='org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer'
-    ;;
     version)
       HADOOP_CLASSNAME=org.apache.hadoop.util.VersionInfo
       hadoop_debug "Append YARN_CLIENT_OPTS onto HADOOP_OPTS"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92faa015/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 6e0e760..b37b38e 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
@@ -81,7 +81,6 @@ 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.nodemanager.NodeManager.NMContext;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92faa015/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 663a2e1..89e81ca 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
@@ -154,7 +154,6 @@ import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.ByteString;
-import org.apache.hadoop.yarn.util.resource.Resources;
 
 public class ContainerManagerImpl extends CompositeService implements
     ContainerManager {
@@ -351,9 +350,8 @@ 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, appId, creds, context, 
-        p.getAppLogAggregationInitedTime());
+    ApplicationImpl app = new ApplicationImpl(dispatcher, p.getUser(), appId,
+        creds, context, p.getAppLogAggregationInitedTime());
     context.getApplications().put(appId, app);
     app.handle(new ApplicationInitEvent(appId, acls, logAggregationContext));
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92faa015/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 5a5d30c..531693e 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
@@ -40,7 +40,6 @@ import org.apache.hadoop.yarn.api.records.LogAggregationContext;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.LogAggregationContextPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.proto.YarnProtos;
@@ -118,10 +117,19 @@ public class ApplicationImpl implements Application {
       Context context, long recoveredLogInitedTime) {
     this.dispatcher = dispatcher;
     this.user = user;
-    this.flowContext = flowContext;
     this.appId = appId;
     this.credentials = credentials;
     this.aclsManager = context.getApplicationACLsManager();
+    Configuration conf = context.getConf();
+    if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
+      if (flowContext == null) {
+        throw new IllegalArgumentException("flow context cannot be null");
+      }
+      this.flowContext = flowContext;
+      if (YarnConfiguration.systemMetricsPublisherEnabled(conf)) {
+        context.getNMTimelinePublisher().createTimelineClient(appId);
+      }
+    }
     this.context = context;
     this.appStateStore = context.getNMStateStore();
     ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
@@ -136,16 +144,6 @@ public class ApplicationImpl implements Application {
       Credentials credentials, Context context) {
     this(dispatcher, user, flowContext, appId, credentials,
       context, -1);
-    Configuration conf = context.getConf();
-    if (YarnConfiguration.timelineServiceV2Enabled(conf)) {
-      if (flowContext == null) {
-        throw new IllegalArgumentException("flow context cannot be null");
-      }
-      this.flowContext = flowContext;
-      if (YarnConfiguration.systemMetricsPublisherEnabled(conf)) {
-        context.getNMTimelinePublisher().createTimelineClient(appId);
-      }
-    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92faa015/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 ed8cf06..61e7d65 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
@@ -181,6 +181,8 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
         SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
   }
 
+  @SuppressWarnings("unchecked")
+  @Override
   public void appACLsUpdated(RMApp app, String appViewACLs, long updatedTime) {
     TimelineEntity entity = createApplicationEntity(app.getApplicationId());
     TimelineEvent tEvent = new TimelineEvent();
@@ -248,6 +250,10 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
         app.getFinalApplicationStatus().toString());
     eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO, RMServerUtils
         .createApplicationAttemptState(appAttemtpState).toString());
+    if (appAttempt.getMasterContainer() != null) {
+      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
+          appAttempt.getMasterContainer().getId().toString());
+    }
     tEvent.setEventInfo(eventInfo);
 
     entity.addEvent(tEvent);
@@ -301,6 +307,12 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
         container.getContainerExitStatus());
     eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO,
         container.getContainerState().toString());
+    Map<String, Object> entityInfo = new HashMap<String, Object>();
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
+        container.getAllocatedNode().getHost());
+    entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
+       container.getAllocatedNode().getPort());
+    entity.setOtherInfo(entityInfo);
     tEvent.setEventInfo(eventInfo);
 
     entity.addEvent(tEvent);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92faa015/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 955570f..a248199 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TimelineServiceV2Publisher.java
@@ -196,10 +196,14 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
   @Override
   public void appACLsUpdated(RMApp app, String appViewACLs, long updatedTime) {
     ApplicationEntity entity = createApplicationEntity(app.getApplicationId());
+    TimelineEvent tEvent = new TimelineEvent();
     Map<String, Object> entityInfo = new HashMap<String, Object>();
     entityInfo.put(ApplicationMetricsConstants.APP_VIEW_ACLS_ENTITY_INFO,
-        appViewACLs);
+        (appViewACLs == null) ? "" : appViewACLs);
     entity.setInfo(entityInfo);
+    tEvent.setId(ApplicationMetricsConstants.ACLS_UPDATED_EVENT_TYPE);
+    tEvent.setTimestamp(updatedTime);
+    entity.addEvent(tEvent);
 
     getDispatcher().getEventHandler().handle(new TimelineV2PublishEvent(
         SystemMetricsEventType.PUBLISH_ENTITY, entity, app.getApplicationId()));
@@ -285,6 +289,10 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
         app.getFinalApplicationStatus().toString());
     eventInfo.put(AppAttemptMetricsConstants.STATE_EVENT_INFO, RMServerUtils
         .createApplicationAttemptState(appAttemtpState).toString());
+    if (appAttempt.getMasterContainer() != null) {
+      eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
+          appAttempt.getMasterContainer().getId().toString());
+    }
     tEvent.setInfo(eventInfo);
 
     entity.addEvent(tEvent);
@@ -353,6 +361,12 @@ public class TimelineServiceV2Publisher extends AbstractSystemMetricsPublisher {
           container.getContainerExitStatus());
       eventInfo.put(ContainerMetricsConstants.STATE_EVENT_INFO,
           container.getContainerState().toString());
+      Map<String, Object> entityInfo = new HashMap<String, Object>();
+      entityInfo.put(ContainerMetricsConstants.ALLOCATED_HOST_ENTITY_INFO,
+          container.getAllocatedNode().getHost());
+      entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
+          container.getAllocatedNode().getPort());
+      entity.setInfo(entityInfo);
       tEvent.setInfo(eventInfo);
 
       entity.addEvent(tEvent);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/92faa015/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
index a8e5195..5641572 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
@@ -22,12 +22,12 @@
   <parent>
     <artifactId>hadoop-yarn-server</artifactId>
     <groupId>org.apache.hadoop</groupId>
-    <version>3.0.0-SNAPSHOT</version>
+    <version>3.0.0-alpha1-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.hadoop</groupId>
   <artifactId>hadoop-yarn-server-timelineservice-hbase-tests</artifactId>
-  <version>3.0.0-SNAPSHOT</version>
+  <version>3.0.0-alpha1-SNAPSHOT</version>
   <name>Apache Hadoop YARN Timeline Service HBase tests</name>
 
   <properties>


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


[25/50] [abbrv] hadoop git commit: YARN-5096 addendum. Turned another logging statement to debug. Contributed by Sangjin Lee.

Posted by vr...@apache.org.
YARN-5096 addendum. Turned another logging statement to debug. 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/1c0d74d9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1c0d74d9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1c0d74d9

Branch: refs/heads/YARN-2928
Commit: 1c0d74d9b95685e6b07733a0361755f93ad1396c
Parents: 3a11e5c
Author: Sangjin Lee <sj...@apache.org>
Authored: Thu May 19 15:40:15 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:09 2016 -0700

----------------------------------------------------------------------
 .../yarn/server/timelineservice/storage/common/ColumnHelper.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c0d74d9/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 dff677b..759bf27 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
@@ -258,7 +258,9 @@ public class ColumnHelper<T> {
 
           String columnName = null;
           if (columnPrefixBytes == null) {
-            LOG.info("null prefix was specified; returning all columns");
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("null prefix was specified; returning all columns");
+            }
             // Decode the spaces we encoded in the column name.
             columnName = Separator.decode(columnKey, Separator.SPACE);
           } else {


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


[16/50] [abbrv] hadoop git commit: YARN-5045. hbase unit tests fail due to dependency issues. (Sangjin Lee via varunsaxena)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
deleted file mode 100644
index e7e7ba4..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
+++ /dev/null
@@ -1,671 +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.flow;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertNotEquals;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.List;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.ArrayList;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.KeyValue;
-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.Put;
-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.regionserver.HRegion;
-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.server.timelineservice.storage.common.LongConverter;
-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.TimestampGenerator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
-
-/**
- * Tests the FlowRun and FlowActivity Tables
- */
-public class TestHBaseStorageFlowRunCompaction {
-
-  private static HBaseTestingUtility util;
-
-  private final String metric1 = "MAP_SLOT_MILLIS";
-  private final String metric2 = "HDFS_BYTES_READ";
-
-  private final byte[] aRowKey = Bytes.toBytes("a");
-  private final byte[] aFamily = Bytes.toBytes("family");
-  private final byte[] aQualifier = Bytes.toBytes("qualifier");
-
-  @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 non numeric data into flow run table
-   * reads it back
-   *
-   * @throws Exception
-   */
-  @Test
-  public void testWriteNonNumericData() throws Exception {
-    String rowKey = "nonNumericRowKey";
-    String column = "nonNumericColumnName";
-    String value = "nonNumericValue";
-    byte[] rowKeyBytes = Bytes.toBytes(rowKey);
-    byte[] columnNameBytes = Bytes.toBytes(column);
-    byte[] valueBytes = Bytes.toBytes(value);
-    Put p = new Put(rowKeyBytes);
-    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnNameBytes,
-        valueBytes);
-    Configuration hbaseConf = util.getConfiguration();
-    TableName table = TableName.valueOf(hbaseConf.get(
-        FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
-    Connection conn = null;
-    conn = ConnectionFactory.createConnection(hbaseConf);
-    Table flowRunTable = conn.getTable(table);
-    flowRunTable.put(p);
-
-    Get g = new Get(rowKeyBytes);
-    Result r = flowRunTable.get(g);
-    assertNotNull(r);
-    assertTrue(r.size() >= 1);
-    Cell actualValue = r.getColumnLatestCell(
-        FlowRunColumnFamily.INFO.getBytes(), columnNameBytes);
-    assertNotNull(CellUtil.cloneValue(actualValue));
-    assertEquals(Bytes.toString(CellUtil.cloneValue(actualValue)), value);
-  }
-
-  @Test
-  public void testWriteFlowRunCompaction() throws Exception {
-    String cluster = "kompaction_cluster1";
-    String user = "kompaction_FlowRun__user1";
-    String flow = "kompaction_flowRun_flow_name";
-    String flowVersion = "AF1021C19F1351";
-    long runid = 1449526652000L;
-
-    int start = 10;
-    int count = 2000;
-    int appIdSuffix = 1;
-    HBaseTimelineWriterImpl hbi = null;
-    long insertTs = System.currentTimeMillis() - count;
-    Configuration c1 = util.getConfiguration();
-    TimelineEntities te1 = null;
-    TimelineEntity entityApp1 = null;
-    try {
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      // now insert count * ( 100 + 100) metrics
-      // each call to getEntityMetricsApp1 brings back 100 values
-      // of metric1 and 100 of metric2
-      for (int i = start; i < start + count; i++) {
-        String appName = "application_10240000000000_" + appIdSuffix;
-        insertTs++;
-        te1 = new TimelineEntities();
-        entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1(insertTs, c1);
-        te1.addEntity(entityApp1);
-        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
-
-        appName = "application_2048000000000_7" + appIdSuffix;
-        insertTs++;
-        te1 = new TimelineEntities();
-        entityApp1 = TestFlowDataGenerator.getEntityMetricsApp2(insertTs);
-        te1.addEntity(entityApp1);
-        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
-      }
-    } finally {
-      String appName = "application_10240000000000_" + appIdSuffix;
-      te1 = new TimelineEntities();
-      entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1Complete(
-          insertTs + 1, c1);
-      te1.addEntity(entityApp1);
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
-      hbi.flush();
-      hbi.close();
-    }
-
-    // check in flow run table
-    HRegionServer server = util.getRSForFirstRegionInTable(TableName
-        .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
-    List<HRegion> regions = server.getOnlineRegions(TableName
-        .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
-    assertTrue("Didn't find any regions for primary table!", regions.size() > 0);
-    // flush and compact all the regions of the primary table
-    for (HRegion region : regions) {
-       region.flushcache();
-      region.compactStores(true);
-    }
-
-    // check flow run for one flow many apps
-    checkFlowRunTable(cluster, user, flow, runid, c1, 4);
-  }
-
-
-  private void checkFlowRunTable(String cluster, String user, String flow,
-      long runid, Configuration c1, int valueCount) 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());
-      assertEquals(valueCount, values.size());
-
-      rowCount++;
-      // check metric1
-      byte[] q = ColumnHelper.getColumnQualifier(
-          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric1);
-      assertTrue(values.containsKey(q));
-      assertEquals(141, Bytes.toLong(values.get(q)));
-
-      // check metric2
-      q = ColumnHelper.getColumnQualifier(
-          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric2);
-      assertTrue(values.containsKey(q));
-      assertEquals(57, Bytes.toLong(values.get(q)));
-    }
-    assertEquals(1, rowCount);
-  }
-
-
-  private FlowScanner getFlowScannerForTestingCompaction() {
-    // create a FlowScanner object with the sole purpose of invoking a process
-    // summation;
-    CompactionRequest request = new CompactionRequest();
-    request.setIsMajor(true, true);
-    // okay to pass in nulls for the constructor arguments
-    // because all we want to do is invoke the process summation
-    FlowScanner fs = new FlowScanner(null, -1, null,
-        (request.isMajor() == true ? FlowScannerOperation.MAJOR_COMPACTION
-            : FlowScannerOperation.MINOR_COMPACTION));
-    assertNotNull(fs);
-    return fs;
-  }
-
-  @Test
-  public void checkProcessSummationMoreCellsSumFinal2()
-      throws IOException {
-    long cellValue1 = 1236L;
-    long cellValue2 = 28L;
-    long cellValue3 = 1236L;
-    long cellValue4 = 1236L;
-    FlowScanner fs = getFlowScannerForTestingCompaction();
-
-    // note down the current timestamp
-    long currentTimestamp = System.currentTimeMillis();
-    long cell1Ts = 1200120L;
-    long cell2Ts = TimestampGenerator.getSupplementedTimestamp(
-        System.currentTimeMillis(),"application_123746661110_11202");
-    long cell3Ts = 1277719L;
-    long cell4Ts = currentTimestamp - 10;
-
-    SortedSet<Cell> currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
-
-    List<Tag> tags = new ArrayList<>();
-    Tag t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
-        "application_1234588888_91188");
-    tags.add(t);
-    byte[] tagByteArray = Tag.fromList(tags);
-    // create a cell with a VERY old timestamp and attribute SUM_FINAL
-    Cell c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-        cell1Ts, Bytes.toBytes(cellValue1), tagByteArray);
-    currentColumnCells.add(c1);
-
-    tags = new ArrayList<>();
-    t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
-        "application_12700000001_29102");
-    tags.add(t);
-    tagByteArray = Tag.fromList(tags);
-    // create a cell with a recent timestamp and attribute SUM_FINAL
-    Cell c2 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-        cell2Ts, Bytes.toBytes(cellValue2), tagByteArray);
-    currentColumnCells.add(c2);
-
-    tags = new ArrayList<>();
-    t = new Tag(AggregationOperation.SUM.getTagType(),
-        "application_191780000000001_8195");
-    tags.add(t);
-    tagByteArray = Tag.fromList(tags);
-    // create a cell with a VERY old timestamp but has attribute SUM
-    Cell c3 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-        cell3Ts, Bytes.toBytes(cellValue3), tagByteArray);
-    currentColumnCells.add(c3);
-
-    tags = new ArrayList<>();
-    t = new Tag(AggregationOperation.SUM.getTagType(),
-        "application_191780000000001_98104");
-    tags.add(t);
-    tagByteArray = Tag.fromList(tags);
-    // create a cell with a VERY old timestamp but has attribute SUM
-    Cell c4 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-        cell4Ts, Bytes.toBytes(cellValue4), tagByteArray);
-    currentColumnCells.add(c4);
-
-    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
-    assertNotNull(cells);
-
-    // we should be getting back 4 cells
-    // one is the flow sum cell
-    // two are the cells with SUM attribute
-    // one cell with SUM_FINAL
-    assertEquals(4, cells.size());
-
-    for (int i = 0; i < cells.size(); i++) {
-      Cell returnedCell = cells.get(0);
-      assertNotNull(returnedCell);
-
-      long returnTs = returnedCell.getTimestamp();
-      long returnValue = Bytes.toLong(CellUtil
-          .cloneValue(returnedCell));
-      if (returnValue == cellValue2) {
-        assertTrue(returnTs == cell2Ts);
-      } else if (returnValue == cellValue3) {
-        assertTrue(returnTs == cell3Ts);
-      } else if (returnValue == cellValue4) {
-        assertTrue(returnTs == cell4Ts);
-      } else if (returnValue == cellValue1) {
-        assertTrue(returnTs != cell1Ts);
-        assertTrue(returnTs > cell1Ts);
-        assertTrue(returnTs >= currentTimestamp);
-      } else {
-        // raise a failure since we expect only these two values back
-        Assert.fail();
-      }
-    }
-  }
-
-  // tests with many cells
-  // of type SUM and SUM_FINAL
-  // all cells of SUM_FINAL will expire
-  @Test
-  public void checkProcessSummationMoreCellsSumFinalMany() throws IOException {
-    FlowScanner fs = getFlowScannerForTestingCompaction();
-    int count = 200000;
-
-    long cellValueFinal = 1000L;
-    long cellValueNotFinal = 28L;
-
-    // note down the current timestamp
-    long currentTimestamp = System.currentTimeMillis();
-    long cellTsFinalStart = 10001120L;
-    long cellTsFinal = cellTsFinalStart;
-    long cellTsNotFinalStart = currentTimestamp - 5;
-    long cellTsNotFinal = cellTsNotFinalStart;
-
-    SortedSet<Cell> currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
-    List<Tag> tags = null;
-    Tag t = null;
-    Cell c1 = null;
-
-    // insert SUM_FINAL cells
-    for (int i = 0; i < count; i++) {
-      tags = new ArrayList<>();
-      t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
-          "application_123450000" + i + "01_19" + i);
-      tags.add(t);
-      byte[] tagByteArray = Tag.fromList(tags);
-      // create a cell with a VERY old timestamp and attribute SUM_FINAL
-      c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-          cellTsFinal, Bytes.toBytes(cellValueFinal), tagByteArray);
-      currentColumnCells.add(c1);
-      cellTsFinal++;
-    }
-
-    // add SUM cells
-    for (int i = 0; i < count; i++) {
-      tags = new ArrayList<>();
-      t = new Tag(AggregationOperation.SUM.getTagType(),
-          "application_1987650000" + i + "83_911" + i);
-      tags.add(t);
-      byte[] tagByteArray = Tag.fromList(tags);
-      // create a cell with attribute SUM
-      c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-          cellTsNotFinal, Bytes.toBytes(cellValueNotFinal), tagByteArray);
-      currentColumnCells.add(c1);
-      cellTsNotFinal++;
-    }
-
-    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
-    assertNotNull(cells);
-
-    // we should be getting back count + 1 cells
-    // one is the flow sum cell
-    // others are the cells with SUM attribute
-    assertEquals(count + 1, cells.size());
-
-    for (int i = 0; i < cells.size(); i++) {
-      Cell returnedCell = cells.get(0);
-      assertNotNull(returnedCell);
-
-      long returnTs = returnedCell.getTimestamp();
-      long returnValue = Bytes.toLong(CellUtil
-          .cloneValue(returnedCell));
-      if (returnValue == (count * cellValueFinal)) {
-        assertTrue(returnTs > (cellTsFinalStart + count));
-        assertTrue(returnTs >= currentTimestamp);
-      } else if ((returnValue >= cellValueNotFinal)
-          && (returnValue <= cellValueNotFinal * count)) {
-        assertTrue(returnTs >= cellTsNotFinalStart);
-        assertTrue(returnTs <= cellTsNotFinalStart * count);
-      } else {
-        // raise a failure since we expect only these values back
-        Assert.fail();
-      }
-    }
-  }
-
-  // tests with many cells
-  // of type SUM and SUM_FINAL
-  // NOT cells of SUM_FINAL will expire
-  @Test
-  public void checkProcessSummationMoreCellsSumFinalVariedTags() throws IOException {
-    FlowScanner fs = getFlowScannerForTestingCompaction();
-    int countFinal = 20100;
-    int countNotFinal = 1000;
-    int countFinalNotExpire = 7009;
-
-    long cellValueFinal = 1000L;
-    long cellValueNotFinal = 28L;
-
-    // note down the current timestamp
-    long currentTimestamp = System.currentTimeMillis();
-    long cellTsFinalStart = 10001120L;
-    long cellTsFinal = cellTsFinalStart;
-
-    long cellTsFinalStartNotExpire = TimestampGenerator.getSupplementedTimestamp(
-        System.currentTimeMillis(), "application_10266666661166_118821");
-    long cellTsFinalNotExpire = cellTsFinalStartNotExpire;
-
-    long cellTsNotFinalStart = currentTimestamp - 5;
-    long cellTsNotFinal = cellTsNotFinalStart;
-
-    SortedSet<Cell> currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
-    List<Tag> tags = null;
-    Tag t = null;
-    Cell c1 = null;
-
-    // insert SUM_FINAL cells which will expire
-    for (int i = 0; i < countFinal; i++) {
-      tags = new ArrayList<>();
-      t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
-          "application_123450000" + i + "01_19" + i);
-      tags.add(t);
-      byte[] tagByteArray = Tag.fromList(tags);
-      // create a cell with a VERY old timestamp and attribute SUM_FINAL
-      c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-          cellTsFinal, Bytes.toBytes(cellValueFinal), tagByteArray);
-      currentColumnCells.add(c1);
-      cellTsFinal++;
-    }
-
-    // insert SUM_FINAL cells which will NOT expire
-    for (int i = 0; i < countFinalNotExpire; i++) {
-      tags = new ArrayList<>();
-      t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
-          "application_123450000" + i + "01_19" + i);
-      tags.add(t);
-      byte[] tagByteArray = Tag.fromList(tags);
-      // create a cell with a VERY old timestamp and attribute SUM_FINAL
-      c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-          cellTsFinalNotExpire, Bytes.toBytes(cellValueFinal), tagByteArray);
-      currentColumnCells.add(c1);
-      cellTsFinalNotExpire++;
-    }
-
-    // add SUM cells
-    for (int i = 0; i < countNotFinal; i++) {
-      tags = new ArrayList<>();
-      t = new Tag(AggregationOperation.SUM.getTagType(),
-          "application_1987650000" + i + "83_911" + i);
-      tags.add(t);
-      byte[] tagByteArray = Tag.fromList(tags);
-      // create a cell with attribute SUM
-      c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-          cellTsNotFinal, Bytes.toBytes(cellValueNotFinal), tagByteArray);
-      currentColumnCells.add(c1);
-      cellTsNotFinal++;
-    }
-
-    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
-    assertNotNull(cells);
-
-    // we should be getting back
-    // countNotFinal + countFinalNotExpire + 1 cells
-    // one is the flow sum cell
-    // count = the cells with SUM attribute
-    // count = the cells with SUM_FINAL attribute but not expired
-    assertEquals(countFinalNotExpire + countNotFinal + 1, cells.size());
-
-    for (int i = 0; i < cells.size(); i++) {
-      Cell returnedCell = cells.get(0);
-      assertNotNull(returnedCell);
-
-      long returnTs = returnedCell.getTimestamp();
-      long returnValue = Bytes.toLong(CellUtil
-          .cloneValue(returnedCell));
-      if (returnValue == (countFinal * cellValueFinal)) {
-        assertTrue(returnTs > (cellTsFinalStart + countFinal));
-        assertTrue(returnTs >= currentTimestamp);
-      } else if (returnValue == cellValueNotFinal) {
-        assertTrue(returnTs >= cellTsNotFinalStart);
-        assertTrue(returnTs <= cellTsNotFinalStart + countNotFinal);
-      } else if (returnValue == cellValueFinal){
-        assertTrue(returnTs >= cellTsFinalStartNotExpire);
-        assertTrue(returnTs <= cellTsFinalStartNotExpire + countFinalNotExpire);
-      } else {
-        // raise a failure since we expect only these values back
-        Assert.fail();
-      }
-    }
-  }
-
-  @Test
-  public void testProcessSummationMoreCellsSumFinal() throws IOException {
-    FlowScanner fs = getFlowScannerForTestingCompaction();
-    // note down the current timestamp
-    long currentTimestamp = System.currentTimeMillis();
-    long cellValue1 = 1236L;
-    long cellValue2 = 28L;
-
-    List<Tag> tags = new ArrayList<>();
-    Tag t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
-        "application_1234588888_999888");
-    tags.add(t);
-    byte[] tagByteArray = Tag.fromList(tags);
-    SortedSet<Cell> currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
-
-    // create a cell with a VERY old timestamp and attribute SUM_FINAL
-    Cell c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-        120L, Bytes.toBytes(cellValue1), tagByteArray);
-    currentColumnCells.add(c1);
-
-    tags = new ArrayList<>();
-    t = new Tag(AggregationOperation.SUM.getTagType(),
-        "application_100000000001_119101");
-    tags.add(t);
-    tagByteArray = Tag.fromList(tags);
-
-    // create a cell with a VERY old timestamp but has attribute SUM
-    Cell c2 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-        130L, Bytes.toBytes(cellValue2), tagByteArray);
-    currentColumnCells.add(c2);
-    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
-    assertNotNull(cells);
-
-    // we should be getting back two cells
-    // one is the flow sum cell
-    // another is the cell with SUM attribute
-    assertEquals(2, cells.size());
-
-    Cell returnedCell = cells.get(0);
-    assertNotNull(returnedCell);
-    long inputTs1 = c1.getTimestamp();
-    long inputTs2 = c2.getTimestamp();
-
-    long returnTs = returnedCell.getTimestamp();
-    long returnValue = Bytes.toLong(CellUtil
-        .cloneValue(returnedCell));
-    // the returned Ts will be far greater than input ts as well as the noted
-    // current timestamp
-    if (returnValue == cellValue2) {
-      assertTrue(returnTs == inputTs2);
-    } else if (returnValue == cellValue1) {
-      assertTrue(returnTs >= currentTimestamp);
-      assertTrue(returnTs != inputTs1);
-    } else {
-      // raise a failure since we expect only these two values back
-      Assert.fail();
-    }
-  }
-
-  @Test
-  public void testProcessSummationOneCellSumFinal() throws IOException {
-    FlowScanner fs = getFlowScannerForTestingCompaction();
-
-    // note down the current timestamp
-    long currentTimestamp = System.currentTimeMillis();
-    List<Tag> tags = new ArrayList<>();
-    Tag t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
-        "application_123458888888_999888");
-    tags.add(t);
-    byte[] tagByteArray = Tag.fromList(tags);
-    SortedSet<Cell> currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
-
-    // create a cell with a VERY old timestamp
-    Cell c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-        120L, Bytes.toBytes(1110L), tagByteArray);
-    currentColumnCells.add(c1);
-
-    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
-    assertNotNull(cells);
-    // we should not get the same cell back
-    // but we get back the flow cell
-    assertEquals(1, cells.size());
-
-    Cell returnedCell = cells.get(0);
-    // it's NOT the same cell
-    assertNotEquals(c1, returnedCell);
-    long inputTs = c1.getTimestamp();
-    long returnTs = returnedCell.getTimestamp();
-    // the returned Ts will be far greater than input ts as well as the noted
-    // current timestamp
-    assertTrue(returnTs > inputTs);
-    assertTrue(returnTs >= currentTimestamp);
-  }
-
-  @Test
-  public void testProcessSummationOneCell() throws IOException {
-    FlowScanner fs = getFlowScannerForTestingCompaction();
-
-    // note down the current timestamp
-    long currentTimestamp = System.currentTimeMillis();
-
-    // try for 1 cell with tag SUM
-    List<Tag> tags = new ArrayList<>();
-    Tag t = new Tag(AggregationOperation.SUM.getTagType(),
-        "application_123458888888_999888");
-    tags.add(t);
-    byte[] tagByteArray = Tag.fromList(tags);
-
-    SortedSet<Cell> currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
-
-    Cell c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
-        currentTimestamp, Bytes.toBytes(1110L), tagByteArray);
-    currentColumnCells.add(c1);
-    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
-    assertNotNull(cells);
-    // we expect the same cell back
-    assertEquals(1, cells.size());
-    Cell c2 = cells.get(0);
-    assertEquals(c1, c2);
-    assertEquals(currentTimestamp, c2.getTimestamp());
-  }
-
-  @Test
-  public void testProcessSummationEmpty() throws IOException {
-    FlowScanner fs = getFlowScannerForTestingCompaction();
-    long currentTimestamp = System.currentTimeMillis();
-
-    SortedSet<Cell> currentColumnCells = null;
-    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
-    assertNotNull(cells);
-    assertEquals(0, cells.size());
-
-    currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
-    cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
-    assertNotNull(cells);
-    assertEquals(0, cells.size());
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    util.shutdownMiniCluster();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
index 6a77c0c..6e1f461 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
@@ -44,5 +44,6 @@
     <module>hadoop-yarn-server-applicationhistoryservice</module>
     <module>hadoop-yarn-server-timeline-pluginstorage</module>
     <module>hadoop-yarn-server-timelineservice</module>
+    <module>hadoop-yarn-server-timelineservice-hbase-tests</module>
   </modules>
 </project>


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


[29/50] [abbrv] hadoop git commit: YARN-5018. Online aggregation logic should not run immediately after collectors got started (Li Lu via sjlee)

Posted by vr...@apache.org.
YARN-5018. Online aggregation logic should not run immediately after collectors got started (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/bab078d7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bab078d7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bab078d7

Branch: refs/heads/YARN-2928
Commit: bab078d70091057a2fc7b8873e183d43365503f5
Parents: ef12e0e
Author: Sangjin Lee <sj...@apache.org>
Authored: Tue May 24 11:02:56 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:10 2016 -0700

----------------------------------------------------------------------
 .../RMTimelineCollectorManager.java               |  2 +-
 .../collector/AppLevelTimelineCollector.java      | 17 +++++++++++++++--
 .../collector/NodeTimelineCollectorManager.java   |  2 +-
 .../collector/TimelineCollector.java              | 12 +++++++++++-
 .../collector/TimelineCollectorManager.java       | 18 +++++++++++++++++-
 5 files changed, 45 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bab078d7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/RMTimelineCollectorManager.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/timelineservice/RMTimelineCollectorManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/RMTimelineCollectorManager.java
index a4f1084..64c3749 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/RMTimelineCollectorManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/timelineservice/RMTimelineCollectorManager.java
@@ -49,7 +49,7 @@ public class RMTimelineCollectorManager extends TimelineCollectorManager {
   }
 
   @Override
-  public void postPut(ApplicationId appId, TimelineCollector collector) {
+  protected void doPostPut(ApplicationId appId, TimelineCollector collector) {
     RMApp app = rmContext.getRMApps().get(appId);
     if (app == null) {
       throw new YarnRuntimeException(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bab078d7/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 eb05262..d276269 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
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import com.google.common.base.Preconditions;
 
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -93,7 +94,8 @@ public class AppLevelTimelineCollector extends TimelineCollector {
         new ThreadFactoryBuilder()
             .setNameFormat("TimelineCollector Aggregation thread #%d")
             .build());
-    appAggregationExecutor.scheduleAtFixedRate(new AppLevelAggregator(), 0,
+    appAggregationExecutor.scheduleAtFixedRate(new AppLevelAggregator(),
+        AppLevelTimelineCollector.AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS,
         AppLevelTimelineCollector.AGGREGATION_EXECUTOR_EXEC_INTERVAL_SECS,
         TimeUnit.SECONDS);
     super.serviceStart();
@@ -126,10 +128,21 @@ public class AppLevelTimelineCollector extends TimelineCollector {
       if (LOG.isDebugEnabled()) {
         LOG.debug("App-level real-time aggregating");
       }
+      if (!isReadyToAggregate()) {
+        LOG.warn("App-level collector is not ready, skip aggregation. ");
+        return;
+      }
       try {
         TimelineCollectorContext currContext = getTimelineEntityContext();
+        Map<String, AggregationStatusTable> aggregationGroups
+            = getAggregationGroups();
+        if (aggregationGroups == null
+            || aggregationGroups.isEmpty()) {
+          LOG.debug("App-level collector is empty, skip aggregation. ");
+          return;
+        }
         TimelineEntity resultEntity = TimelineCollector.aggregateWithoutGroupId(
-            getAggregationGroups(), currContext.getAppId(),
+            aggregationGroups, currContext.getAppId(),
             TimelineEntityType.YARN_APPLICATION.toString());
         TimelineEntities entities = new TimelineEntities();
         entities.addEntity(resultEntity);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bab078d7/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 75557a8..0323d7b 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
@@ -87,7 +87,7 @@ public class NodeTimelineCollectorManager extends TimelineCollectorManager {
   }
 
   @Override
-  public void postPut(ApplicationId appId, TimelineCollector collector) {
+  protected void doPostPut(ApplicationId appId, TimelineCollector collector) {
     try {
       // Get context info from NM
       updateTimelineCollectorContext(appId, collector);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bab078d7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
index 8cd645c..2fc3033 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java
@@ -60,6 +60,8 @@ public abstract class TimelineCollector extends CompositeService {
   private static Set<String> entityTypesSkipAggregation
       = new HashSet<>();
 
+  private volatile boolean readyToAggregate = false;
+
   public TimelineCollector(String name) {
     super(name);
   }
@@ -91,6 +93,14 @@ public abstract class TimelineCollector extends CompositeService {
     return aggregationGroups;
   }
 
+  protected void setReadyToAggregate() {
+    readyToAggregate = true;
+  }
+
+  protected boolean isReadyToAggregate() {
+    return readyToAggregate;
+  }
+
   /**
    * Method to decide the set of timeline entity types the collector should
    * skip on aggregations. Subclasses may want to override this method to
@@ -258,7 +268,7 @@ public abstract class TimelineCollector extends CompositeService {
 
   // Note: In memory aggregation is performed in an eventually consistent
   // fashion.
-  private static class AggregationStatusTable {
+  protected static class AggregationStatusTable {
     // On aggregation, for each metric, aggregate all per-entity accumulated
     // metrics. We only use the id and type for TimelineMetrics in the key set
     // of this table.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bab078d7/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 8f74ffb..a8f88e5 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
@@ -136,8 +136,24 @@ public class TimelineCollectorManager extends AbstractService {
     return collectorInTable;
   }
 
-  protected void postPut(ApplicationId appId, TimelineCollector collector) {
+  /**
+   * Callback handler for the timeline collector manager when a collector has
+   * been added into the collector map.
+   * @param appId Application id of the collector.
+   * @param collector The actual timeline collector that has been added.
+   */
+  public void postPut(ApplicationId appId, TimelineCollector collector) {
+    doPostPut(appId, collector);
+    collector.setReadyToAggregate();
+  }
 
+  /**
+   * A template method that will be called by
+   * {@link  #postPut(ApplicationId, TimelineCollector)}.
+   * @param appId Application id of the collector.
+   * @param collector The actual timeline collector that has been added.
+   */
+  protected void doPostPut(ApplicationId appId, TimelineCollector collector) {
   }
 
   /**


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


[09/50] [abbrv] hadoop git commit: YARN-4447. Provide a mechanism to represent complex filters and parse them at the REST layer (Varun Saxena via sjlee)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesUtils.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/TestTimelineReaderWebServicesUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesUtils.java
new file mode 100644
index 0000000..e991d27
--- /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/TestTimelineReaderWebServicesUtils.java
@@ -0,0 +1,923 @@
+/**
+ * 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.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList.Operator;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefixFilter;
+import org.junit.Test;
+
+import com.google.common.collect.Sets;
+
+public class TestTimelineReaderWebServicesUtils {
+  private static void verifyFilterList(String expr, TimelineFilterList list,
+      TimelineFilterList expectedList) throws Exception {
+    assertNotNull(list);
+    assertTrue("Unexpected List received after parsing expression " + expr +
+        ". Expected=" + expectedList + " but Actual=" + list,
+        list.equals(expectedList));
+  }
+
+  @Test
+  public void testMetricFiltersParsing() throws Exception {
+    String expr = "(((key11 ne 234 AND key12 gt 23) AND " +
+        "(key13 lt 34 OR key14 ge 567)) OR (key21 lt 24 OR key22 le 45))";
+    TimelineFilterList expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            Operator.AND,
+            new TimelineFilterList(
+                Operator.AND,
+                new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                    "key11", 234, false),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+                    "key12", 23, true)
+            ),
+            new TimelineFilterList(
+                Operator.OR,
+                new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                    "key13", 34, true),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+                    "key14", 567, true)
+            )
+        ),
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                "key21", 24, true),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                "key22", 45, true)
+        )
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "abc ene 234";
+    expectedList = new TimelineFilterList(
+        new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+            "abc", 234, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "abc ne 234";
+    expectedList = new TimelineFilterList(
+        new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+            "abc", 234, false)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "abc ne 234 AND def gt 23";
+    expectedList = new TimelineFilterList(
+        new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+            "abc", 234, false),
+        new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+            "def", 23, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "(abc ne 234 AND def gt 23)";
+    expectedList = new TimelineFilterList(
+        new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+            "abc", 234, false),
+        new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+            "def", 23, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "abc ne 234 AND def gt 23 OR rst lt 24";
+    expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                "abc", 234, false),
+            new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+                "def", 23, true)
+        ),
+        new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+            "rst", 24, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "abc ne 234 AND def gt 23 OR rst lt 24 OR xyz le 456";
+    expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                "abc", 234, false),
+            new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+                "def", 23, true)
+        ),
+        new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+            "rst", 24, true),
+        new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+            "xyz", 456, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "abc ne 234 AND def gt 23 OR rst lt 24 OR xyz le 456 AND pqr ge 2";
+    expectedList = new TimelineFilterList(
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineFilterList(
+                new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                    "abc", 234, false),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+                    "def", 23, true)
+            ),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                "rst", 24, true),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                "xyz", 456, true)
+        ),
+        new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+            "pqr", 2, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    // Test with unnecessary spaces.
+    expr = "  abc ne   234       AND       def           gt 23 OR     rst lt "+
+        "           24     OR xyz     le     456    AND pqr ge 2        ";
+    expectedList = new TimelineFilterList(
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineFilterList(
+                new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                    "abc", 234, false),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_THAN,
+                    "def", 23, true)
+            ),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                "rst", 24, true),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                "xyz", 456, true)
+        ),
+        new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+            "pqr", 2, true)
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "(((key11 ne 234 AND key12 gt 23 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            Operator.AND,
+            new TimelineFilterList(
+                new TimelineFilterList(
+                    Operator.OR,
+                    new TimelineFilterList(
+                        new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                            "key11", 234, false),
+                        new TimelineCompareFilter(
+                            TimelineCompareOp.GREATER_THAN, "key12", 23, true)
+                    ),
+                    new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                        "key13", 24, true),
+                    new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                        "key14", 456, true)
+                ),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+                    "key15", 2, true)
+            ),
+            new TimelineFilterList(
+                Operator.OR,
+                new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                    "key16", 34, true),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+                    "key17", 567, true)
+            )
+        ),
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                "key21", 24, true),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                "key22", 45, true)
+        )
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "   (  (     (        key11      ne     234    AND key12    gt   " +
+        "23    OR    key13    lt    24 OR key14 le 456   AND   key15   ge   2" +
+        "   )   AND ( key16 lt 34 OR key17 ge 567 )    )     OR " +
+        "(   key21 lt 24 OR key22 le 45 )   )    ";
+    expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            Operator.AND,
+            new TimelineFilterList(
+                new TimelineFilterList(
+                    Operator.OR,
+                    new TimelineFilterList(
+                        new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL,
+                            "key11", 234, false),
+                        new TimelineCompareFilter(
+                            TimelineCompareOp.GREATER_THAN, "key12", 23, true)
+                    ),
+                    new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                        "key13", 24, true),
+                    new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                        "key14", 456, true)
+                ),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+                    "key15", 2, true)
+            ),
+            new TimelineFilterList(
+                Operator.OR,
+                new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                    "key16", 34, true),
+                new TimelineCompareFilter(TimelineCompareOp.GREATER_OR_EQUAL,
+                    "key17", 567, true)
+            )
+        ),
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineCompareFilter(TimelineCompareOp.LESS_THAN,
+                "key21", 24, true),
+            new TimelineCompareFilter(TimelineCompareOp.LESS_OR_EQUAL,
+                "key22", 45, true)
+        )
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseMetricFilters(expr), expectedList);
+
+    expr = "(((key11 ne 234 AND key12 gt 23 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45)";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Improper brackers. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(((key11 ne 234 AND key12 gt v3 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Non Numeric value. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(((key11 ne (234 AND key12 gt 3 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Unexpected opening bracket. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(((k)ey11 ne 234 AND key12 gt 3 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Unexpected closing bracket. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(((key11 rs 234 AND key12 gt 3 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Improper compare op. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(((key11 ne 234 PI key12 gt 3 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Improper op. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(((key11 ne 234 PI key12 gt 3 OR key13 lt 24 OR key14 le 456 " +
+        "AND key15 ge 2) AND (key16 lt 34 OR key17 ge 567)) OR (key21 lt 24 " +
+        "OR key22 le 45))";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Improper op. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(key11 ne 234 AND key12 gt 3)) OR (key13 lt 24 OR key14 le 456)";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Unbalanced brackets. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(key11 rne 234 AND key12 gt 3) OR (key13 lt 24 OR key14 le 456)";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Invalid compareop. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    expr = "(key11 ne 234 AND key12 gt 3) OR (key13 lt 24 OR key14 le";
+    try {
+      TimelineReaderWebServicesUtils.parseMetricFilters(expr);
+      fail("Compareop cant be parsed. Exception should have been thrown.");
+    } catch (TimelineParseException e) {}
+
+    assertNull(TimelineReaderWebServicesUtils.parseMetricFilters(null));
+    assertNull(TimelineReaderWebServicesUtils.parseMetricFilters("   "));
+  }
+
+  @Test
+  public void testConfigFiltersParsing() throws Exception {
+    String expr = "(((key11 ne 234 AND key12 eq val12) AND " +
+        "(key13 ene val13 OR key14 eq 567)) OR (key21 eq val_21 OR key22 eq " +
+        "val.22))";
+    TimelineFilterList expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            Operator.AND,
+            new TimelineFilterList(
+                Operator.AND,
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "key11", "234", false),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "key12", "val12", true)
+            ),
+            new TimelineFilterList(
+                Operator.OR,
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "key13", "val13", true),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "key14", "567", true)
+            )
+        ),
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "key21", "val_21", true),
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "key22", "val.22", true)
+        )
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseKVFilters(expr, true), expectedList);
+
+    expr = "abc ne 234 AND def eq 23 OR rst ene 24 OR xyz eq 456 AND pqr eq 2";
+    expectedList = new TimelineFilterList(
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineFilterList(
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "abc", "234", false),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "def", "23", true)
+            ),
+            new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                "rst", "24", true),
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "xyz", "456", true)
+        ),
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+            "pqr", "2", true)
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseKVFilters(expr, true), expectedList);
+
+    // Test with unnecessary spaces.
+    expr = "  abc ne   234       AND       def           eq 23 OR     rst ene "+
+        "           24     OR xyz     eq     456    AND pqr eq 2        ";
+    expectedList = new TimelineFilterList(
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineFilterList(
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "abc", "234", false),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "def", "23", true)
+            ),
+            new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                "rst", "24", true),
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "xyz", "456", true)
+        ),
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+            "pqr", "2", true)
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseKVFilters(expr, true), expectedList);
+
+    expr = "abc gt 234 AND def eq 23 OR rst ene 24 OR xyz eq 456 AND pqr eq 2";
+    try {
+      TimelineReaderWebServicesUtils.parseKVFilters(expr, true);
+      fail("Invalid compareop specified for config filters. Should be either" +
+          " eq,ne or ene and exception should have been thrown.");
+    } catch (TimelineParseException e){}
+  }
+  @Test
+  public void testInfoFiltersParsing() throws Exception {
+    String expr = "(((key11 ne 234 AND key12 eq val12) AND " +
+        "(key13 ene val13 OR key14 eq 567)) OR (key21 eq val_21 OR key22 eq " +
+        "5.0))";
+    TimelineFilterList expectedList = new TimelineFilterList(
+        Operator.OR,
+        new TimelineFilterList(
+            Operator.AND,
+            new TimelineFilterList(
+                Operator.AND,
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "key11", 234, false),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "key12", "val12", true)
+            ),
+            new TimelineFilterList(
+                Operator.OR,
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "key13", "val13", true),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "key14", 567, true)
+            )
+        ),
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "key21", "val_21", true),
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "key22", 5.0, true)
+        )
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseKVFilters(expr, false), expectedList);
+
+    expr = "abc ne 234 AND def eq 23 OR rst ene 24 OR xyz eq 456 AND pqr eq " +
+        "val.1234";
+    expectedList = new TimelineFilterList(
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineFilterList(
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "abc", 234, false),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "def", 23, true)
+            ),
+            new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                "rst", 24, true),
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "xyz", 456, true)
+        ),
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+            "pqr", "val.1234", true)
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseKVFilters(expr, false), expectedList);
+
+    // Test with unnecessary spaces.
+    expr = "  abc ne   234       AND       def           eq 23 OR     rst ene "+
+        "           24     OR xyz     eq     456    AND pqr eq 2        ";
+    expectedList = new TimelineFilterList(
+        new TimelineFilterList(
+            Operator.OR,
+            new TimelineFilterList(
+                new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                    "abc", 234, false),
+                new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                    "def", 23, true)
+            ),
+            new TimelineKeyValueFilter(TimelineCompareOp.NOT_EQUAL,
+                "rst", 24, true),
+            new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+                "xyz", 456, true)
+        ),
+        new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
+            "pqr", 2, true)
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseKVFilters(expr, false), expectedList);
+  }
+
+  @Test
+  public void testEventFiltersParsing() throws Exception {
+    String expr = "abc,def";
+    TimelineFilterList expectedList = new TimelineFilterList(
+        new TimelineExistsFilter(TimelineCompareOp.EQUAL, "abc"),
+        new TimelineExistsFilter(TimelineCompareOp.EQUAL, "def")
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList);
+
+    expr = "(abc,def)";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList);
+
+    expr = "(abc,def) OR (rst, uvx)";
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelineFilterList(
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "abc"),
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "def")
+        ),
+        new TimelineFilterList(
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "rst"),
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "uvx")
+        )
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList);
+
+    expr = "!(abc,def,uvc) OR (rst, uvx)";
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelineFilterList(
+            new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, "abc"),
+            new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, "def"),
+            new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, "uvc")
+        ),
+        new TimelineFilterList(
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "rst"),
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "uvx")
+        )
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList);
+
+    expr = "(((!(abc,def,uvc) OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" +
+        " OR ((bcd,tyu) AND uvb))";
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelineFilterList(
+            new TimelineFilterList(Operator.OR,
+                new TimelineFilterList(
+                    new TimelineExistsFilter(
+                        TimelineCompareOp.NOT_EQUAL, "abc"),
+                    new TimelineExistsFilter(
+                        TimelineCompareOp.NOT_EQUAL, "def"),
+                    new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL, "uvc")
+                ),
+                new TimelineFilterList(
+                    new TimelineExistsFilter(TimelineCompareOp.EQUAL, "rst"),
+                    new TimelineExistsFilter(TimelineCompareOp.EQUAL, "uvx")
+                )
+            ),
+            new TimelineFilterList(Operator.OR,
+                new TimelineFilterList(
+                    new TimelineExistsFilter(
+                        TimelineCompareOp.NOT_EQUAL, "abcdefg")
+                ),
+                new TimelineFilterList(
+                    new TimelineExistsFilter(
+                        TimelineCompareOp.NOT_EQUAL, "ghj"),
+                    new TimelineExistsFilter(
+                        TimelineCompareOp.NOT_EQUAL, "tyu")
+                )
+            )
+        ),
+        new TimelineFilterList(
+            new TimelineFilterList(
+                new TimelineExistsFilter(TimelineCompareOp.EQUAL, "bcd"),
+                new TimelineExistsFilter(TimelineCompareOp.EQUAL, "tyu")
+            ),
+            new TimelineExistsFilter(TimelineCompareOp.EQUAL, "uvb")
+        )
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList);
+
+    expr = "  (  (  (  !  (  abc , def  ,   uvc)   OR   (   rst  ,   uvx )  )" +
+        "  AND   (  !  (  abcdefg ) OR  !   (  ghj,  tyu)  ) )  OR   (   (" +
+        "   bcd   ,   tyu  )   AND   uvb  )   )";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseEventFilters(expr), expectedList);
+
+    expr = "(((!(abc,def,uvc) OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" +
+        " OR ((bcd,tyu) AND uvb)";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Improper brackets. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "(((!(abc,def,uvc) (OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" +
+        " OR ((bcd,tyu) AND uvb))";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected opening bracket. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "(((!(abc,def,uvc) OR) (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" +
+        " OR ((bcd,tyu) AND uvb))";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected closing bracket. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "(((!(abc,def,uvc) PI (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" +
+        " OR ((bcd,tyu) AND uvb))";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Invalid op. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "(((!(abc,def,uvc) !OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu)))" +
+        " OR ((bcd,tyu) AND uvb))";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected ! char. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "abc,def,uvc) OR (rst, uvx)";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected closing bracket. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "abc,def,uvc OR )rst, uvx)";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected closing bracket. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "abc,def,uvc OR ,rst, uvx)";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected delimiter. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "abc,def,uvc OR !  ";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unexpected not char. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "(abc,def,uvc)) OR (rst, uvx)";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("Unbalanced brackets. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "(((! ,(abc,def,uvc) OR (rst, uvx)) AND (!(abcdefg) OR !(ghj,tyu" +
+        "))) OR ((bcd,tyu) AND uvb))";
+    try {
+      TimelineReaderWebServicesUtils.parseEventFilters(expr);
+      fail("( should follow ! char. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    assertNull(TimelineReaderWebServicesUtils.parseEventFilters(null));
+    assertNull(TimelineReaderWebServicesUtils.parseEventFilters("   "));
+  }
+
+  @Test
+  public void testRelationFiltersParsing() throws Exception {
+    String expr = "type1:entity11,type2:entity21:entity22";
+    TimelineFilterList expectedList = new TimelineFilterList(
+        new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+            "type1", Sets.newHashSet((Object)"entity11")),
+        new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+            "type2", Sets.newHashSet((Object)"entity21", "entity22"))
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseRelationFilters(expr), expectedList);
+
+    expr = "(type1:entity11,type2:entity21:entity22)";
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseRelationFilters(expr), expectedList);
+
+    expr = "(type1:entity11,type2:entity21:entity22) OR (type3:entity31:" +
+        "entity32:entity33,type1:entity11:entity12)";
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelineFilterList(
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                "type1", Sets.newHashSet((Object)"entity11")),
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                "type2", Sets.newHashSet((Object)"entity21", "entity22"))
+        ),
+        new TimelineFilterList(
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                "type3", Sets.newHashSet(
+                    (Object)"entity31", "entity32", "entity33")),
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                "type1", Sets.newHashSet((Object)"entity11", "entity12"))
+        )
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseRelationFilters(expr), expectedList);
+
+    expr = "!(type1:entity11,type2:entity21:entity22,type5:entity51) OR " +
+        "(type3:entity31:entity32:entity33,type1:entity11:entity12)";
+
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelineFilterList(
+            new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                "type1", Sets.newHashSet((Object)"entity11")),
+            new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                "type2", Sets.newHashSet((Object)"entity21", "entity22")),
+            new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                "type5", Sets.newHashSet((Object)"entity51"))
+        ),
+        new TimelineFilterList(
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                "type3", Sets.newHashSet(
+                    (Object)"entity31", "entity32", "entity33")),
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                "type1", Sets.newHashSet((Object)"entity11", "entity12"))
+        )
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseRelationFilters(expr), expectedList);
+
+    expr = "(((!(type1:entity11,type2:entity21:entity22,type5:entity51) OR " +
+        "(type3:entity31:entity32:entity33,type1:entity11:entity12)) AND "+
+        "(!(type11:entity111) OR !(type4:entity43:entity44:entity47:entity49," +
+        "type7:entity71))) OR ((type2:entity2,type8:entity88) AND t9:e:e1))";
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelineFilterList(
+            new TimelineFilterList(Operator.OR,
+                new TimelineFilterList(
+                    new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                        "type1", Sets.newHashSet((Object)"entity11")),
+                    new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                        "type2", Sets.newHashSet(
+                            (Object)"entity21", "entity22")),
+                    new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                        "type5", Sets.newHashSet((Object)"entity51"))
+                ),
+                new TimelineFilterList(
+                    new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                        "type3", Sets.newHashSet(
+                            (Object)"entity31", "entity32", "entity33")),
+                    new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                        "type1", Sets.newHashSet(
+                            (Object)"entity11", "entity12"))
+                )
+            ),
+            new TimelineFilterList(Operator.OR,
+                new TimelineFilterList(
+                    new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                        "type11", Sets.newHashSet((Object)"entity111"))
+                ),
+                new TimelineFilterList(
+                    new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                        "type4", Sets.newHashSet((Object)"entity43","entity44",
+                            "entity47","entity49")),
+                    new TimelineKeyValuesFilter(TimelineCompareOp.NOT_EQUAL,
+                        "type7", Sets.newHashSet((Object)"entity71"))
+                )
+            )
+        ),
+        new TimelineFilterList(
+            new TimelineFilterList(
+                new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                    "type2", Sets.newHashSet((Object)"entity2")),
+                new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL,
+                    "type8", Sets.newHashSet((Object)"entity88"))
+            ),
+            new TimelineKeyValuesFilter(TimelineCompareOp.EQUAL, "t9",
+                Sets.newHashSet((Object)"e", "e1"))
+        )
+    );
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseRelationFilters(expr), expectedList);
+
+    expr = "   (   (  (   !   (   type1:entity11  ,  type2:entity21:entity22" +
+        "  ,  type5:entity51  )   OR  (   type3:entity31:entity32:entity33  " +
+        "     ,   type1:entity11:entity12)) AND (!(  type11:entity111  )  OR " +
+        "    !   (   type4:entity43:entity44:entity47:entity49 , " +
+        "type7:entity71  )  )  ) OR  (  (  type2:entity2 , type8:entity88) " +
+        "AND  t9:e:e1 )    ) ";
+    verifyFilterList(expr, TimelineReaderWebServicesUtils.
+        parseRelationFilters(expr), expectedList);
+
+    expr = "(((!(type1 : entity11,type2:entity21:entity22,type5:entity51) OR " +
+        "(type3:entity31:entity32:entity33,type1:entity11:entity12)) AND "+
+        "(!(type11:entity111) OR !(type4:entity43:entity44:entity47:entity49," +
+        "type7:entity71))) OR ((type2:entity2,type8:entity88) AND t9:e:e1))";
+    try {
+      TimelineReaderWebServicesUtils.parseRelationFilters(expr);
+      fail("Space not allowed in relation expression. Exception should have " +
+          "been thrown");
+    } catch (TimelineParseException e){}
+  }
+
+  @Test
+  public void testDataToRetrieve() throws Exception {
+    String expr = "abc,def";
+    TimelineFilterList expectedList = new TimelineFilterList(Operator.OR,
+        new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "abc"),
+        new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "def")
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "(abc,def)";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "   (   abc  ,   def  )   ";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "    abc  ,   def   ";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "!(abc,def)";
+    expectedList = new TimelineFilterList(
+        new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "abc"),
+        new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "def")
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = " !  (   abc  ,  def  )  ";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "!(abc)";
+    expectedList = new TimelineFilterList(
+        new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "abc")
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "(abc)";
+    expectedList = new TimelineFilterList(Operator.OR,
+        new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "abc")
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "abc";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = " !  (   abc  ,  def  , xyz)  ";
+    expectedList = new TimelineFilterList(
+        new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "abc"),
+        new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "def"),
+        new TimelinePrefixFilter(TimelineCompareOp.NOT_EQUAL, "xyz")
+    );
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "!(abc,def,xyz)";
+    verifyFilterList(expr,
+        TimelineReaderWebServicesUtils.parseDataToRetrieve(expr), expectedList);
+
+    expr = "!(abc,def,xyz";
+    try {
+      TimelineReaderWebServicesUtils.parseDataToRetrieve(expr);
+      fail("No closing bracket. Exception should have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "!abc,def,xyz";
+    try {
+      TimelineReaderWebServicesUtils.parseDataToRetrieve(expr);
+      fail("NOT(!) should be followed by opening bracket. Exception should " +
+          "have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "!abc,def,xyz";
+    try {
+      TimelineReaderWebServicesUtils.parseDataToRetrieve(expr);
+      fail("NOT(!) should be followed by opening bracket. Exception should " +
+          "have been thrown");
+    } catch (TimelineParseException e){}
+
+    expr = "!   r(  abc,def,xyz)";
+    try {
+      TimelineReaderWebServicesUtils.parseDataToRetrieve(expr);
+      fail("NOT(!) should be followed by opening bracket. Exception should " +
+          "have been thrown");
+    } catch (TimelineParseException e){}
+
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve(null));
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("     "));
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("()"));
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("!()"));
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("(     )"));
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("!(   )"));
+    assertNull(TimelineReaderWebServicesUtils.parseDataToRetrieve("!  (   )"));
+  }
+}


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


[41/50] [abbrv] hadoop git commit: YARN-5142. fix findbugs warnings/errors for hadoop-yarn-server-timelineservice-hbase-tests. (Vrushali C via Varun Saxena)

Posted by vr...@apache.org.
YARN-5142. fix findbugs warnings/errors for hadoop-yarn-server-timelineservice-hbase-tests. (Vrushali C 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/4df6d9b8
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4df6d9b8
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4df6d9b8

Branch: refs/heads/YARN-2928
Commit: 4df6d9b824477cdecef80fd0ef6fd6e2b3b02399
Parents: 10809df
Author: Varun Saxena <va...@apache.org>
Authored: Sat Jun 4 19:20:43 2016 +0530
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:13 2016 -0700

----------------------------------------------------------------------
 ...stTimelineReaderWebServicesHBaseStorage.java |  6 +-
 .../storage/TestHBaseTimelineStorage.java       | 48 +++++-----
 .../storage/flow/TestFlowDataGenerator.java     |  6 +-
 .../flow/TestHBaseStorageFlowActivity.java      | 24 +++--
 .../storage/flow/TestHBaseStorageFlowRun.java   | 96 +++++++++++++-------
 .../flow/TestHBaseStorageFlowRunCompaction.java | 12 ++-
 6 files changed, 120 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4df6d9b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
index f9f4607..9af9205 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -345,7 +345,9 @@ public class TestTimelineReaderWebServicesHBaseStorage {
           "application_1111111111_1111", te5);
       hbi.flush();
     } finally {
-      hbi.close();
+      if (hbi != null) {
+        hbi.close();
+      }
     }
   }
 
@@ -390,7 +392,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
         .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
     if (resp == null ||
         resp.getClientResponseStatus() != ClientResponse.Status.OK) {
-      String msg = new String();
+      String msg = "";
       if (resp != null) {
         msg = resp.getClientResponseStatus().toString();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4df6d9b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/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-hbase-tests/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-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index c002ca0..7b647eb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -678,32 +678,34 @@ public class TestHBaseTimelineStorage {
       assertEquals(infoMap, infoColumns);
 
       // Remember isRelatedTo is of type Map<String, Set<String>>
-      for (String isRelatedToKey : isRelatedTo.keySet()) {
+      for (Map.Entry<String, Set<String>> isRelatedToEntry : isRelatedTo
+          .entrySet()) {
         Object isRelatedToValue =
             ApplicationColumnPrefix.IS_RELATED_TO.readResult(result,
-                isRelatedToKey);
+                isRelatedToEntry.getKey());
         String compoundValue = isRelatedToValue.toString();
         // id7?id9?id6
         Set<String> isRelatedToValues =
             new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
-        assertEquals(isRelatedTo.get(isRelatedToKey).size(),
+        assertEquals(isRelatedTo.get(isRelatedToEntry.getKey()).size(),
             isRelatedToValues.size());
-        for (String v : isRelatedTo.get(isRelatedToKey)) {
+        for (String v : isRelatedToEntry.getValue()) {
           assertTrue(isRelatedToValues.contains(v));
         }
       }
 
       // RelatesTo
-      for (String relatesToKey : relatesTo.keySet()) {
+      for (Map.Entry<String, Set<String>> relatesToEntry : relatesTo
+          .entrySet()) {
         String compoundValue =
             ApplicationColumnPrefix.RELATES_TO.readResult(result,
-                relatesToKey).toString();
+                relatesToEntry.getKey()).toString();
         // id3?id4?id5
         Set<String> relatesToValues =
             new HashSet<String>(Separator.VALUES.splitEncoded(compoundValue));
-        assertEquals(relatesTo.get(relatesToKey).size(),
+        assertEquals(relatesTo.get(relatesToEntry.getKey()).size(),
             relatesToValues.size());
-        for (String v : relatesTo.get(relatesToKey)) {
+        for (String v : relatesToEntry.getValue()) {
           assertTrue(relatesToValues.contains(v));
         }
       }
@@ -938,41 +940,43 @@ public class TestHBaseTimelineStorage {
           assertEquals(infoMap, infoColumns);
 
           // Remember isRelatedTo is of type Map<String, Set<String>>
-          for (String isRelatedToKey : isRelatedTo.keySet()) {
+          for (Map.Entry<String, Set<String>> isRelatedToEntry : isRelatedTo
+              .entrySet()) {
             Object isRelatedToValue =
                 EntityColumnPrefix.IS_RELATED_TO.readResult(result,
-                    isRelatedToKey);
+                    isRelatedToEntry.getKey());
             String compoundValue = isRelatedToValue.toString();
             // id7?id9?id6
             Set<String> isRelatedToValues =
                 new HashSet<String>(
                     Separator.VALUES.splitEncoded(compoundValue));
-            assertEquals(isRelatedTo.get(isRelatedToKey).size(),
+            assertEquals(isRelatedTo.get(isRelatedToEntry.getKey()).size(),
                 isRelatedToValues.size());
-            for (String v : isRelatedTo.get(isRelatedToKey)) {
+            for (String v : isRelatedToEntry.getValue()) {
               assertTrue(isRelatedToValues.contains(v));
             }
           }
 
           // RelatesTo
-          for (String relatesToKey : relatesTo.keySet()) {
-            String compoundValue =
-                EntityColumnPrefix.RELATES_TO.readResult(result, relatesToKey)
-                    .toString();
+          for (Map.Entry<String, Set<String>> relatesToEntry : relatesTo
+              .entrySet()) {
+            String compoundValue = EntityColumnPrefix.RELATES_TO
+                .readResult(result, relatesToEntry.getKey()).toString();
             // id3?id4?id5
             Set<String> relatesToValues =
                 new HashSet<String>(
                     Separator.VALUES.splitEncoded(compoundValue));
-            assertEquals(relatesTo.get(relatesToKey).size(),
+            assertEquals(relatesTo.get(relatesToEntry.getKey()).size(),
                 relatesToValues.size());
-            for (String v : relatesTo.get(relatesToKey)) {
+            for (String v : relatesToEntry.getValue()) {
               assertTrue(relatesToValues.contains(v));
             }
           }
 
           // Configuration
           Map<String, Object> configColumns =
-              EntityColumnPrefix.CONFIG.readResults(result, StringKeyConverter.getInstance());
+              EntityColumnPrefix.CONFIG.readResults(result,
+                  StringKeyConverter.getInstance());
           assertEquals(conf, configColumns);
 
           NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
@@ -1273,8 +1277,10 @@ public class TestHBaseTimelineStorage {
         assertTrue(info == null || info.isEmpty());
       }
     } finally {
-      hbi.stop();
-      hbi.close();
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4df6d9b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java
index 3b8036d..0535a13 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestFlowDataGenerator.java
@@ -55,8 +55,8 @@ class TestFlowDataGenerator {
     Map<Long, Number> metricValues = new HashMap<Long, Number>();
     long ts = insertTs;
 
-    for (int k=1; k< 100 ; k++) {
-    metricValues.put(ts - k*200000, 20L);
+    for (int k = 1; k < 100; k++) {
+      metricValues.put(ts - k * 200000L, 20L);
     }
     metricValues.put(ts - 80000, 40L);
     m1.setType(Type.TIME_SERIES);
@@ -68,7 +68,7 @@ class TestFlowDataGenerator {
     metricValues = new HashMap<Long, Number>();
     ts = System.currentTimeMillis();
     for (int k=1; k< 100 ; k++) {
-      metricValues.put(ts - k*100000, 31L);
+      metricValues.put(ts - k*100000L, 31L);
     }
 
     metricValues.put(ts - 80000, 57L);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4df6d9b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
index 072332d..589b78d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
@@ -148,7 +148,9 @@ public class TestHBaseStorageFlowActivity {
       // flush everything to hbase
       hbi.flush();
     } finally {
-      hbi.close();
+      if (hbi != null) {
+        hbi.close();
+      }
     }
 
     Connection conn = ConnectionFactory.createConnection(c1);
@@ -199,7 +201,9 @@ public class TestHBaseStorageFlowActivity {
         assertEquals(1, flowRuns.size());
       }
     } finally {
-      hbr.close();
+      if (hbr != null) {
+        hbr.close();
+      }
     }
   }
 
@@ -230,7 +234,9 @@ public class TestHBaseStorageFlowActivity {
       hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
       hbi.flush();
     } finally {
-      hbi.close();
+      if (hbi != null) {
+        hbi.close();
+      }
     }
     // check flow activity
     checkFlowActivityTable(cluster, user, flow, flowVersion, runid, c1,
@@ -260,7 +266,9 @@ public class TestHBaseStorageFlowActivity {
         }
       }
     } finally {
-      hbr.close();
+      if (hbr != null) {
+        hbr.close();
+      }
     }
   }
 
@@ -351,7 +359,9 @@ public class TestHBaseStorageFlowActivity {
 
       hbi.flush();
     } finally {
-      hbi.close();
+      if (hbi != null) {
+        hbi.close();
+      }
     }
     // check flow activity
     checkFlowActivityTableSeveralRuns(cluster, user, flow, c1, flowVersion1,
@@ -396,7 +406,9 @@ public class TestHBaseStorageFlowActivity {
         }
       }
     } finally {
-      hbr.close();
+      if (hbr != null) {
+        hbr.close();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4df6d9b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
index d0f98a5..a443b50 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
@@ -75,8 +75,8 @@ public class TestHBaseStorageFlowRun {
 
   private static HBaseTestingUtility util;
 
-  private final String metric1 = "MAP_SLOT_MILLIS";
-  private final String metric2 = "HDFS_BYTES_READ";
+  private static final String METRIC1 = "MAP_SLOT_MILLIS";
+  private static final String METRIC2 = "HDFS_BYTES_READ";
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
@@ -213,7 +213,9 @@ public class TestHBaseStorageFlowRun {
       // flush everything to hbase
       hbi.flush();
     } finally {
-      hbi.close();
+      if (hbi != null) {
+        hbi.close();
+      }
     }
 
     Connection conn = ConnectionFactory.createConnection(c1);
@@ -257,7 +259,9 @@ public class TestHBaseStorageFlowRun {
       assertEquals(minStartTs, flowRun.getStartTime());
       assertEquals(endTs, flowRun.getMaxEndTime());
     } finally {
-      hbr.close();
+      if (hbr != null) {
+        hbr.close();
+      }
     }
   }
 
@@ -299,7 +303,9 @@ public class TestHBaseStorageFlowRun {
       hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
       hbi.flush();
     } finally {
-      hbi.close();
+      if (hbi != null) {
+        hbi.close();
+      }
     }
 
     // check flow run
@@ -327,10 +333,10 @@ public class TestHBaseStorageFlowRun {
           value = n;
         }
         switch (id) {
-        case metric1:
+        case METRIC1:
           assertEquals(141L, value);
           break;
-        case metric2:
+        case METRIC2:
           assertEquals(57L, value);
           break;
         default:
@@ -338,7 +344,9 @@ public class TestHBaseStorageFlowRun {
         }
       }
     } finally {
-      hbr.close();
+      if (hbr != null) {
+        hbr.close();
+      }
     }
   }
 
@@ -365,14 +373,14 @@ public class TestHBaseStorageFlowRun {
       rowCount++;
       // check metric1
       byte[] q = ColumnHelper.getColumnQualifier(
-          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric1);
+          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), METRIC1);
       assertTrue(values.containsKey(q));
       assertEquals(141L, Bytes.toLong(values.get(q)));
 
       // check metric2
       assertEquals(3, values.size());
       q = ColumnHelper.getColumnQualifier(
-          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric2);
+          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), METRIC2);
       assertTrue(values.containsKey(q));
       assertEquals(57L, Bytes.toLong(values.get(q)));
     }
@@ -407,7 +415,9 @@ public class TestHBaseStorageFlowRun {
       hbi.write(cluster, user, flow, flowVersion, 1002345678918L, appName, te);
       hbi.flush();
     } finally {
-      hbi.close();
+      if (hbi != null) {
+        hbi.close();
+      }
     }
 
     // use the timeline reader to verify data
@@ -418,7 +428,7 @@ public class TestHBaseStorageFlowRun {
       hbr.start();
       TimelineFilterList metricsToRetrieve = new TimelineFilterList(
           Operator.OR, new TimelinePrefixFilter(TimelineCompareOp.EQUAL,
-              metric1.substring(0, metric1.indexOf("_") + 1)));
+              METRIC1.substring(0, METRIC1.indexOf("_") + 1)));
       TimelineEntity entity = hbr.getEntity(
           new TimelineReaderContext(cluster, user, flow, 1002345678919L, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
@@ -435,7 +445,7 @@ public class TestHBaseStorageFlowRun {
           value = n;
         }
         switch (id) {
-        case metric1:
+        case METRIC1:
           assertEquals(40L, value);
           break;
         default:
@@ -455,7 +465,9 @@ public class TestHBaseStorageFlowRun {
       }
       assertEquals(2, metricCnt);
     } finally {
-      hbr.close();
+      if (hbr != null) {
+        hbr.close();
+      }
     }
   }
 
@@ -488,7 +500,9 @@ public class TestHBaseStorageFlowRun {
       hbi.write(cluster, user, flow, flowVersion, runid, appName, te);
       hbi.flush();
     } finally {
-      hbi.close();
+      if (hbi != null) {
+        hbi.close();
+      }
     }
 
     // check flow run
@@ -528,10 +542,10 @@ public class TestHBaseStorageFlowRun {
             value = n;
           }
           switch (id) {
-          case metric1:
+          case METRIC1:
             assertEquals(141L, value);
             break;
-          case metric2:
+          case METRIC2:
             assertEquals(57L, value);
             break;
           default:
@@ -540,7 +554,9 @@ public class TestHBaseStorageFlowRun {
         }
       }
     } finally {
-      hbr.close();
+      if (hbr != null) {
+        hbr.close();
+      }
     }
   }
 
@@ -595,8 +611,10 @@ public class TestHBaseStorageFlowRun {
         }
       }
     } finally {
-      hbi.flush();
-      hbi.close();
+      if (hbi != null) {
+        hbi.flush();
+        hbi.close();
+      }
       checkMinMaxFlush(c1, minTS, startTs, count, cluster, user, flow, runid,
           true);
     }
@@ -665,7 +683,9 @@ public class TestHBaseStorageFlowRun {
           "application_11111111111111_2222", te);
       hbi.flush();
     } finally {
-      hbi.close();
+      if (hbi != null) {
+        hbi.close();
+      }
     }
 
     // use the timeline reader to verify data
@@ -711,7 +731,9 @@ public class TestHBaseStorageFlowRun {
         }
       }
     } finally {
-      hbr.close();
+      if (hbr != null) {
+        hbr.close();
+      }
     }
   }
 
@@ -742,7 +764,9 @@ public class TestHBaseStorageFlowRun {
           "application_11111111111111_2222", te);
       hbi.flush();
     } finally {
-      hbi.close();
+      if (hbi != null) {
+        hbi.close();
+      }
     }
 
     // use the timeline reader to verify data
@@ -754,12 +778,12 @@ public class TestHBaseStorageFlowRun {
 
       TimelineFilterList list1 = new TimelineFilterList();
       list1.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.GREATER_OR_EQUAL, metric1, 101));
+          TimelineCompareOp.GREATER_OR_EQUAL, METRIC1, 101));
       TimelineFilterList list2 = new TimelineFilterList();
       list2.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.LESS_THAN, metric1, 43));
+          TimelineCompareOp.LESS_THAN, METRIC1, 43));
       list2.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.EQUAL, metric2, 57));
+          TimelineCompareOp.EQUAL, METRIC2, 57));
       TimelineFilterList metricFilterList =
           new TimelineFilterList(Operator.OR, list1, list2);
       Set<TimelineEntity> entities = hbr.getEntities(
@@ -777,8 +801,8 @@ public class TestHBaseStorageFlowRun {
 
       TimelineFilterList metricFilterList1 = new TimelineFilterList(
           new TimelineCompareFilter(
-          TimelineCompareOp.LESS_OR_EQUAL, metric1, 127),
-          new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, metric2, 30));
+          TimelineCompareOp.LESS_OR_EQUAL, METRIC1, 127),
+          new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, METRIC2, 30));
       entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
@@ -793,8 +817,8 @@ public class TestHBaseStorageFlowRun {
       assertEquals(2, metricCnt);
 
       TimelineFilterList metricFilterList2 = new TimelineFilterList(
-          new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, metric1, 32),
-          new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, metric2, 57));
+          new TimelineCompareFilter(TimelineCompareOp.LESS_THAN, METRIC1, 32),
+          new TimelineCompareFilter(TimelineCompareOp.NOT_EQUAL, METRIC2, 57));
       entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
@@ -815,17 +839,17 @@ public class TestHBaseStorageFlowRun {
 
       TimelineFilterList list3 = new TimelineFilterList();
       list3.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.GREATER_OR_EQUAL, metric1, 101));
+          TimelineCompareOp.GREATER_OR_EQUAL, METRIC1, 101));
       TimelineFilterList list4 = new TimelineFilterList();
       list4.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.LESS_THAN, metric1, 43));
+          TimelineCompareOp.LESS_THAN, METRIC1, 43));
       list4.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.EQUAL, metric2, 57));
+          TimelineCompareOp.EQUAL, METRIC2, 57));
       TimelineFilterList metricFilterList4 =
           new TimelineFilterList(Operator.OR, list3, list4);
       TimelineFilterList metricsToRetrieve = new TimelineFilterList(Operator.OR,
           new TimelinePrefixFilter(TimelineCompareOp.EQUAL,
-          metric2.substring(0, metric2.indexOf("_") + 1)));
+          METRIC2.substring(0, METRIC2.indexOf("_") + 1)));
       entities = hbr.getEntities(
           new TimelineReaderContext(cluster, user, flow, null, null,
           TimelineEntityType.YARN_FLOW_RUN.toString(), null),
@@ -840,7 +864,9 @@ public class TestHBaseStorageFlowRun {
       }
       assertEquals(1, metricCnt);
     } finally {
-      hbr.close();
+      if (hbr != null) {
+        hbr.close();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4df6d9b8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
index e7e7ba4..6b0ee5c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
@@ -69,8 +69,8 @@ public class TestHBaseStorageFlowRunCompaction {
 
   private static HBaseTestingUtility util;
 
-  private final String metric1 = "MAP_SLOT_MILLIS";
-  private final String metric2 = "HDFS_BYTES_READ";
+  private static final String metric1 = "MAP_SLOT_MILLIS";
+  private static final String metric2 = "HDFS_BYTES_READ";
 
   private final byte[] aRowKey = Bytes.toBytes("a");
   private final byte[] aFamily = Bytes.toBytes("family");
@@ -166,9 +166,11 @@ public class TestHBaseStorageFlowRunCompaction {
       entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1Complete(
           insertTs + 1, c1);
       te1.addEntity(entityApp1);
-      hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
-      hbi.flush();
-      hbi.close();
+      if (hbi != null) {
+        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+        hbi.flush();
+        hbi.close();
+      }
     }
 
     // check in flow run table


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


[27/50] [abbrv] hadoop git commit: YARN-5096. timelinereader has a lot of logging that's not useful. (Sangjin Lee via Varun Saxena)

Posted by vr...@apache.org.
YARN-5096. timelinereader has a lot of logging that's not useful. (Sangjin Lee 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/3a11e5c1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3a11e5c1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3a11e5c1

Branch: refs/heads/YARN-2928
Commit: 3a11e5c1614ef2331fecc1ad589834a88f2bec6b
Parents: a5daac5
Author: Varun Saxena <va...@apache.org>
Authored: Wed May 18 19:33:14 2016 +0530
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:09 2016 -0700

----------------------------------------------------------------------
 .../yarn/server/timelineservice/storage/common/ColumnHelper.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3a11e5c1/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 4adb413..dff677b 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
@@ -194,7 +194,9 @@ public class ColumnHelper<T> {
             .entrySet()) {
           String columnName = null;
           if (columnPrefixBytes == null) {
-            LOG.info("null prefix was specified; returning all columns");
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("null prefix was specified; returning all columns");
+            }
             // Decode the spaces we encoded in the column name.
             columnName = Separator.decode(entry.getKey(), Separator.SPACE);
           } else {


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


[39/50] [abbrv] hadoop git commit: YARN-5189. Make HBaseTimeline[Reader|Writer]Impl default and move FileSystemTimeline*Impl. (Joep Rottinghuis and Sangjin Lee via gtcarrera9)

Posted by vr...@apache.org.
YARN-5189. Make HBaseTimeline[Reader|Writer]Impl default and move FileSystemTimeline*Impl. (Joep Rottinghuis and Sangjin Lee via gtcarrera9)


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

Branch: refs/heads/YARN-2928
Commit: 10809dfc650b5c36b0de17ac9da71bc63051d40b
Parents: fc78a93
Author: Li Lu <gt...@apache.org>
Authored: Fri Jun 3 16:33:51 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:13 2016 -0700

----------------------------------------------------------------------
 .../hadoop-mapreduce-client-jobclient/pom.xml   |  6 ++++
 .../mapred/TestMRTimelineEventHandling.java     |  3 ++
 .../hadoop/yarn/conf/YarnConfiguration.java     |  8 ++++--
 .../pom.xml                                     |  1 +
 .../distributedshell/TestDistributedShell.java  | 18 ++++++------
 .../src/main/resources/yarn-default.xml         | 29 +++++++++++++++++---
 .../hadoop-yarn-server-resourcemanager/pom.xml  |  6 ++++
 .../TestSystemMetricsPublisherForV2.java        |  3 ++
 .../TestTimelineServiceClientIntegration.java   |  8 +++++-
 .../collector/TimelineCollectorManager.java     |  4 +--
 .../reader/TimelineReaderServer.java            |  4 +--
 .../storage/FileSystemTimelineReaderImpl.java   |  5 +++-
 .../storage/FileSystemTimelineWriterImpl.java   |  4 ++-
 .../TestNMTimelineCollectorManager.java         |  9 ++++--
 ...TestPerNodeTimelineCollectorsAuxService.java |  4 +++
 .../reader/TestTimelineReaderServer.java        |  5 +++-
 .../reader/TestTimelineReaderWebServices.java   | 13 +++++----
 17 files changed, 99 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml
index b3fe622..d19b075 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml
@@ -98,6 +98,12 @@
       <type>test-jar</type>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-timelineservice</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
       <groupId>org.hsqldb</groupId>
       <artifactId>hsqldb</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 fde9e64..27ce48c 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
@@ -53,6 +53,7 @@ import org.apache.hadoop.yarn.server.timeline.TimelineStore;
 import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.junit.Assert;
 import org.junit.Test;
@@ -175,6 +176,8 @@ public class TestMRTimelineEventHandling {
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     // enable new timeline service
     conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+    conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
+        FileSystemTimelineWriterImpl.class, TimelineWriter.class);
     conf.setBoolean(MRJobConfig.MAPREDUCE_JOB_EMIT_TIMELINE_DATA, true);
 
     // enable aux-service based timeline collectors

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 dc38631..ff2aa4f 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
@@ -1981,11 +1981,13 @@ public class YarnConfiguration extends Configuration {
    */
   public static final String APP_FINAL_VALUE_RETENTION_THRESHOLD =
       TIMELINE_SERVICE_PREFIX
-      + "coprocessor.app-final-value-retention-milliseconds";
+      + "hbase.coprocessor.app-final-value-retention-milliseconds";
 
   /**
-   * The setting that controls how long the final value of a metric
-   * of a completed app is retained before merging into the flow sum.
+   * The setting that controls how long the final value of a metric of a
+   * completed app is retained before merging into the flow sum. Up to this time
+   * after an application is completed out-of-order values that arrive can be
+   * recognized and discarded at the cost of increased storage.
    */
   public static final long DEFAULT_APP_FINAL_VALUE_RETENTION_THRESHOLD = 3 * 24
       * 60 * 60 * 1000L;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
index c0e2177..e8cb138 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml
@@ -77,6 +77,7 @@
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-server-timelineservice</artifactId>
+      <type>test-jar</type>
       <scope>test</scope>
     </dependency>
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 c02cd85..74da919 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
@@ -64,29 +64,26 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineDomain;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
-import org.apache.hadoop.yarn.applications.distributedshell.ApplicationMaster;
+import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
+import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.client.api.impl.DirectTimelineWriter;
+import org.apache.hadoop.yarn.client.api.impl.TestTimelineClient;
 import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
 import org.apache.hadoop.yarn.client.api.impl.TimelineWriter;
-import org.apache.hadoop.yarn.client.api.impl.TestTimelineClient;
-import org.apache.hadoop.yarn.client.api.TimelineClient;
-import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
-import org.apache.hadoop.yarn.client.api.YarnClient;
-
 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.timeline.PluginStoreTestUtils;
 import org.apache.hadoop.yarn.server.timeline.NameValuePair;
+import org.apache.hadoop.yarn.server.timeline.PluginStoreTestUtils;
 import org.apache.hadoop.yarn.server.timeline.TimelineVersion;
 import org.apache.hadoop.yarn.server.timeline.TimelineVersionWatcher;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
+import org.apache.hadoop.yarn.server.utils.BuilderUtils;
+import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
@@ -185,6 +182,9 @@ public class TestDistributedShell {
       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.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
+          FileSystemTimelineWriterImpl.class,
+          org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter.class);
     } else {
       Assert.fail("Wrong timeline version number: " + timelineVersion);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 6faf2a2..a54cc75 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
@@ -2150,7 +2150,6 @@
     <value>604800</value>
   </property>
 
-  <!-- Timeline Service v2 Configuration -->
   <property>
     <name>yarn.timeline-service.entity-group-fs-store.leveldb-cache-read-cache-size</name>
     <description>
@@ -2188,6 +2187,23 @@
     <value>300</value>
   </property>
 
+  <!-- Timeline Service v2 Configuration -->
+  <property>
+    <name>yarn.timeline-service.writer.class</name>
+    <description>
+      Storage implementation ATS v2 will use for the TimelineWriter service.
+    </description>
+    <value>org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl</value>
+  </property>
+
+  <property>
+    <name>yarn.timeline-service.reader.class</name>
+    <description>
+      Storage implementation ATS v2 will use for the TimelineReader service.
+    </description>
+    <value>org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl</value>
+  </property>
+
   <property>
     <name>yarn.timeline-service.client.internal-timers-ttl-secs</name>
     <description>
@@ -2221,10 +2237,15 @@
   </property>
 
   <property>
-    <description> The setting that controls how long the final value
+    <description>
+    The setting that controls how long the final value
     of a metric of a completed app is retained before merging into
-    the flow sum.</description>
-    <name>yarn.timeline-service.coprocessor.app-final-value-retention-milliseconds</name>
+    the flow sum. Up to this time after an application is completed
+    out-of-order values that arrive can be recognized and discarded at the
+    cost of increased storage.
+    </description>
+    <name>yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds
+    </name>
     <value>259200000</value>
   </property>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
index d875f15..7b64ba0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
@@ -158,6 +158,12 @@
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-timelineservice</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
 
     <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 13c67f8..0da395a 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
@@ -66,6 +66,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineC
 import org.apache.hadoop.yarn.server.timelineservice.collector.AppLevelTimelineCollector;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -104,6 +105,8 @@ public class TestSystemMetricsPublisherForV2 {
         rmTimelineCollectorManager);
 
     Configuration conf = getTimelineV2Conf();
+    conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
+        FileSystemTimelineWriterImpl.class, TimelineWriter.class);
     rmTimelineCollectorManager.init(conf);
     rmTimelineCollectorManager.start();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 a734340..3a5c797 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
@@ -49,6 +49,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorCon
 import org.apache.hadoop.yarn.server.api.protocolrecords.GetTimelineCollectorContextResponse;
 import org.apache.hadoop.yarn.server.timelineservice.collector.NodeTimelineCollectorManager;
 import org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService;
+import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -66,6 +68,8 @@ public class TestTimelineServiceClientIntegration {
       // enable timeline service v.2
       conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
       conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+      conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
+          FileSystemTimelineWriterImpl.class, TimelineWriter.class);
       auxService =
           PerNodeTimelineCollectorsAuxService.launchServer(new String[0],
               collectorManager, conf);
@@ -159,7 +163,9 @@ public class TestTimelineServiceClientIntegration {
           mock(CollectorNodemanagerProtocol.class);
       try {
         GetTimelineCollectorContextResponse response =
-            GetTimelineCollectorContextResponse.newInstance(null, null, null, 0L);
+            GetTimelineCollectorContextResponse.newInstance(
+                UserGroupInformation.getCurrentUser().getShortUserName(),
+                "test_flow_name", "test_flow_version", 1L);
         when(protocol.getTimelineCollectorContext(any(
             GetTimelineCollectorContextRequest.class))).thenReturn(response);
       } catch (YarnException | IOException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 a8f88e5..9758320 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
@@ -36,7 +36,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineWriterImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -61,7 +61,7 @@ public class TimelineCollectorManager extends AbstractService {
   public void serviceInit(Configuration conf) throws Exception {
     writer = ReflectionUtils.newInstance(conf.getClass(
         YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
-        FileSystemTimelineWriterImpl.class,
+        HBaseTimelineWriterImpl.class,
         TimelineWriter.class), conf);
     writer.init(conf);
     // create a single dedicated thread for flushing the writer on a periodic

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 97725e6..110d1dc 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
@@ -41,7 +41,7 @@ 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.HBaseTimelineReaderImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
@@ -81,7 +81,7 @@ public class TimelineReaderServer extends CompositeService {
   private TimelineReader createTimelineReaderStore(Configuration conf) {
     TimelineReader readerStore = ReflectionUtils.newInstance(conf.getClass(
         YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
-        FileSystemTimelineReaderImpl.class, TimelineReader.class), conf);
+        HBaseTimelineReaderImpl.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/10809dfc/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 00aa686..047f401 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
@@ -56,7 +56,10 @@ import org.codehaus.jackson.map.ObjectMapper;
 import com.google.common.annotations.VisibleForTesting;
 
 /**
- *  File System based implementation for TimelineReader.
+ *  File System based implementation for TimelineReader. This implementation may
+ *  not provide a complete implementation of all the necessary features. This
+ *  implementation is provided solely for basic testing purposes, and should not
+ *  be used in a non-test situation.
  */
 public class FileSystemTimelineReaderImpl extends AbstractService
     implements TimelineReader {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 74a03ac..1bb77a0 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
@@ -38,7 +38,9 @@ import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 
 /**
  * This implements a local file based backend for storing application timeline
- * information.
+ * information. This implementation may not provide a complete implementation of
+ * all the necessary features. This implementation is provided solely for basic
+ * testing purposes, and should not be used in a non-test situation.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 0d69fbc..854e046 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
@@ -24,7 +24,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 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;
@@ -45,6 +45,8 @@ 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.storage.FileSystemTimelineWriterImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -55,7 +57,10 @@ public class TestNMTimelineCollectorManager {
   @Before
   public void setup() throws Exception {
     collectorManager = createCollectorManager();
-    collectorManager.init(new YarnConfiguration());
+    Configuration conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
+        FileSystemTimelineWriterImpl.class, TimelineWriter.class);
+    collectorManager.init(conf);
     collectorManager.start();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 f2775d5..7c2a471 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
@@ -44,6 +44,8 @@ import org.apache.hadoop.yarn.server.api.ContainerTerminationContext;
 import org.apache.hadoop.yarn.server.api.ContainerType;
 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.storage.FileSystemTimelineWriterImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter;
 import org.junit.After;
 import org.junit.Test;
 
@@ -60,6 +62,8 @@ public class TestPerNodeTimelineCollectorsAuxService {
     // enable timeline service v.2
     conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
     conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+    conf.setClass(YarnConfiguration.TIMELINE_SERVICE_WRITER_CLASS,
+        FileSystemTimelineWriterImpl.class, TimelineWriter.class);
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 b42488c..fb95493 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
@@ -23,7 +23,8 @@ import static org.junit.Assert.assertEquals;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.Service.STATE;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderServer;
+import org.apache.hadoop.yarn.server.timelineservice.storage.FileSystemTimelineReaderImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
 import org.junit.Test;
 
 public class TestTimelineReaderServer {
@@ -37,6 +38,8 @@ public class TestTimelineReaderServer {
     config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
     config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
         "localhost:0");
+    config.setClass(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
+        FileSystemTimelineReaderImpl.class, TimelineReader.class);
     try {
       server.init(config);
       assertEquals(STATE.INITED, server.getServiceState());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/10809dfc/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 0bddf1b..4d65922 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
@@ -36,8 +36,10 @@ 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.FileSystemTimelineReaderImpl;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TestFileSystemTimelineReaderImpl;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader;
+import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -74,9 +76,11 @@ public class TestTimelineReaderWebServices {
       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, 
+      config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
           "localhost:0");
       config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
+      config.setClass(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
+          FileSystemTimelineReaderImpl.class, TimelineReader.class);
       server = new TimelineReaderServer();
       server.init(config);
       server.start();
@@ -101,13 +105,12 @@ public class TestTimelineReaderWebServices {
   }
 
   private static void verifyHttpResponse(Client client, URI uri,
-      Status status) {
+      Status expectedStatus) {
     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));
+    assertEquals(resp.getClientResponseStatus(), expectedStatus);
   }
 
   private static Client createClient() {


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


[04/50] [abbrv] hadoop git commit: YARN-3863. Support complex filters in TimelineReader (Varun Saxena via sjlee)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 b5fc214..2d85bab 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
@@ -17,21 +17,26 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.storage.common;
 
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
-import java.io.IOException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
@@ -39,6 +44,15 @@ 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.reader.filter.TimelineCompareFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilter.TimelineFilterType;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 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;
@@ -53,6 +67,8 @@ public final class TimelineStorageUtils {
   private TimelineStorageUtils() {
   }
 
+  private static final Log LOG = LogFactory.getLog(TimelineStorageUtils.class);
+
   /** empty bytes. */
   public static final byte[] EMPTY_BYTES = new byte[0];
 
@@ -312,6 +328,21 @@ public final class TimelineStorageUtils {
   }
 
   /**
+   * Check if we have a certain field amongst fields to retrieve. This method
+   * checks against {@link Field#ALL} as well because that would mean field
+   * passed needs to be matched.
+   *
+   * @param fieldsToRetrieve fields to be retrieved.
+   * @param requiredField fields to be checked in fieldsToRetrieve.
+   * @return true if has the required field, false otherwise.
+   */
+  public static boolean hasField(EnumSet<Field> fieldsToRetrieve,
+      Field requiredField) {
+    return fieldsToRetrieve.contains(Field.ALL) ||
+        fieldsToRetrieve.contains(requiredField);
+  }
+
+  /**
    * Checks if the input TimelineEntity object is an ApplicationEntity.
    *
    * @param te TimelineEntity object.
@@ -385,87 +416,317 @@ public final class TimelineStorageUtils {
   }
 
   /**
+   * Matches key-values filter. Used for relatesTo/isRelatedTo filters.
    *
-   * @param entityRelations the relations of an entity
-   * @param relationFilters the relations for filtering
-   * @return a boolean flag to indicate if both match
+   * @param entity entity which holds relatesTo/isRelatedTo relations which we
+   *     will match against.
+   * @param keyValuesFilter key-values filter.
+   * @param entityFiltersType type of filters we are trying to match.
+   * @return true, if filter matches, false otherwise.
    */
-  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) {
+  private static boolean matchKeyValuesFilter(TimelineEntity entity,
+      TimelineKeyValuesFilter keyValuesFilter,
+      TimelineEntityFiltersType entityFiltersType) {
+    Map<String, Set<String>> relations = null;
+    if (entityFiltersType == TimelineEntityFiltersType.IS_RELATED_TO) {
+      relations = entity.getIsRelatedToEntities();
+    } else if (entityFiltersType == TimelineEntityFiltersType.RELATES_TO) {
+      relations = entity.getRelatesToEntities();
+    }
+    if (relations == null) {
+      return false;
+    }
+    Set<String> ids = relations.get(keyValuesFilter.getKey());
+    if (ids == null) {
+      return false;
+    }
+    boolean matched = false;
+    for (Object id : keyValuesFilter.getValues()) {
+      // Matches if id is found amongst the relationships for an entity and
+      // filter's compare op is EQUAL.
+      // If compare op is NOT_EQUAL, for a match to occur, id should not be
+      // found amongst relationships for an entity.
+      matched = !(ids.contains(id) ^
+          keyValuesFilter.getCompareOp() == TimelineCompareOp.EQUAL);
+      if (!matched) {
         return false;
       }
-      for (String id : relation.getValue()) {
-        if (!ids.contains(id)) {
-          return false;
-        }
-      }
     }
     return true;
   }
 
   /**
+   * Matches relatesto.
    *
-   * @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
+   * @param entity entity which holds relatesto relations.
+   * @param relatesTo the relations for filtering.
+   * @return true, if filter matches, false otherwise.
+   * @throws IOException if an unsupported filter for matching relations is
+   *     being matched.
    */
-  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;
-      }
+  public static boolean matchRelatesTo(TimelineEntity entity,
+      TimelineFilterList relatesTo) throws IOException {
+    return matchFilters(
+        entity, relatesTo, TimelineEntityFiltersType.RELATES_TO);
+  }
+
+  /**
+   * Matches isrelatedto.
+   *
+   * @param entity entity which holds isRelatedTo relations.
+   * @param isRelatedTo the relations for filtering.
+   * @return true, if filter matches, false otherwise.
+   * @throws IOException if an unsupported filter for matching relations is
+   *     being matched.
+   */
+  public static boolean matchIsRelatedTo(TimelineEntity entity,
+      TimelineFilterList isRelatedTo) throws IOException {
+    return matchFilters(
+        entity, isRelatedTo, TimelineEntityFiltersType.IS_RELATED_TO);
+  }
+
+  /**
+   * Matches key-value filter. Used for config and info filters.
+   *
+   * @param entity entity which holds the config/info which we will match
+   *     against.
+   * @param kvFilter a key-value filter.
+   * @param entityFiltersType type of filters we are trying to match.
+   * @return true, if filter matches, false otherwise.
+   */
+  private static boolean matchKeyValueFilter(TimelineEntity entity,
+      TimelineKeyValueFilter kvFilter,
+      TimelineEntityFiltersType entityFiltersType) {
+    Map<String, ? extends Object> map = null;
+    // Supported only for config and info filters.
+    if (entityFiltersType == TimelineEntityFiltersType.CONFIG) {
+      map = entity.getConfigs();
+    } else if (entityFiltersType == TimelineEntityFiltersType.INFO) {
+      map = entity.getInfo();
     }
-    return true;
+    if (map == null) {
+      return false;
+    }
+    Object value = map.get(kvFilter.getKey());
+    if (value == null) {
+      return false;
+    }
+    // Matches if filter's value is equal to the value of the key and filter's
+    // compare op is EQUAL.
+    // If compare op is NOT_EQUAL, for a match to occur, value should not be
+    // equal to the value of the key.
+    return !(value.equals(kvFilter.getValue()) ^
+        kvFilter.getCompareOp() == TimelineCompareOp.EQUAL);
+  }
+
+  /**
+   * Matches config filters.
+   *
+   * @param entity entity which holds a map of config key-value pairs.
+   * @param configFilters list of info filters.
+   * @return a boolean flag to indicate if both match.
+   * @throws IOException if an unsupported filter for matching config filters is
+   *     being matched.
+   */
+  public static boolean matchConfigFilters(TimelineEntity entity,
+      TimelineFilterList configFilters) throws IOException {
+    return
+        matchFilters(entity, configFilters, TimelineEntityFiltersType.CONFIG);
+  }
+
+  /**
+   * Matches info filters.
+   *
+   * @param entity entity which holds a map of info key-value pairs.
+   * @param infoFilters list of info filters.
+   * @return a boolean flag to indicate if both match.
+   * @throws IOException if an unsupported filter for matching info filters is
+   *     being matched.
+   */
+  public static boolean matchInfoFilters(TimelineEntity entity,
+      TimelineFilterList infoFilters) throws IOException {
+    return matchFilters(entity, infoFilters, TimelineEntityFiltersType.INFO);
   }
 
   /**
+   * Matches exists filter. Used for event filters.
    *
-   * @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
+   * @param entity entity which holds the events which we will match against.
+   * @param existsFilter exists filter.
+   * @param entityFiltersType type of filters we are trying to match.
+   * @return true, if filter matches, false otherwise.
    */
-  public static boolean matchEventFilters(Set<TimelineEvent> entityEvents,
-      Set<String> eventFilters) {
+  private static boolean matchExistsFilter(TimelineEntity entity,
+      TimelineExistsFilter existsFilter,
+      TimelineEntityFiltersType entityFiltersType) {
+    // Currently exists filter is only supported for event filters.
+    if (entityFiltersType != TimelineEntityFiltersType.EVENT) {
+      return false;
+    }
     Set<String> eventIds = new HashSet<String>();
-    for (TimelineEvent event : entityEvents) {
+    for (TimelineEvent event : entity.getEvents()) {
       eventIds.add(event.getId());
     }
-    for (String eventFilter : eventFilters) {
-      if (!eventIds.contains(eventFilter)) {
-        return false;
-      }
+    // Matches if filter's value is contained in the list of events filter's
+    // compare op is EQUAL.
+    // If compare op is NOT_EQUAL, for a match to occur, value should not be
+    // contained in the list of events.
+    return !(eventIds.contains(existsFilter.getValue()) ^
+        existsFilter.getCompareOp() == TimelineCompareOp.EQUAL);
+  }
+
+  /**
+   * Matches event filters.
+   *
+   * @param entity entity which holds a set of event objects.
+   * @param eventFilters the set of event Ids for filtering.
+   * @return a boolean flag to indicate if both match.
+   * @throws IOException if an unsupported filter for matching event filters is
+   *     being matched.
+   */
+  public static boolean matchEventFilters(TimelineEntity entity,
+      TimelineFilterList eventFilters) throws IOException {
+    return matchFilters(entity, eventFilters, TimelineEntityFiltersType.EVENT);
+  }
+
+  /**
+   * Compare two values based on comparison operator.
+   *
+   * @param compareOp comparison operator.
+   * @param val1 value 1.
+   * @param val2 value 2.
+   * @return true, if relation matches, false otherwise
+   */
+  private static boolean compareValues(TimelineCompareOp compareOp,
+      long val1, long val2) {
+    switch (compareOp) {
+    case LESS_THAN:
+      return val1 < val2;
+    case LESS_OR_EQUAL:
+      return val1 <= val2;
+    case EQUAL:
+      return val1 == val2;
+    case NOT_EQUAL:
+      return val1 != val2;
+    case GREATER_OR_EQUAL:
+      return val1 >= val2;
+    case GREATER_THAN:
+      return val1 > val2;
+    default:
+      throw new RuntimeException("Unknown TimelineCompareOp " +
+          compareOp.name());
     }
-    return true;
   }
 
   /**
+   * Matches compare filter. Used for metric filters.
    *
-   * @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
+   * @param entity entity which holds the metrics which we will match against.
+   * @param compareFilter compare filter.
+   * @param entityFiltersType type of filters we are trying to match.
+   * @return true, if filter matches, false otherwise.
+   * @throws IOException if metric filters holds non integral values.
    */
-  public static boolean matchMetricFilters(Set<TimelineMetric> metrics,
-      Set<String> metricFilters) {
-    Set<String> metricIds = new HashSet<String>();
-    for (TimelineMetric metric : metrics) {
-      metricIds.add(metric.getId());
+  private static boolean matchCompareFilter(TimelineEntity entity,
+      TimelineCompareFilter compareFilter,
+      TimelineEntityFiltersType entityFiltersType) throws IOException {
+    // Currently exists filter is only supported for metric filters.
+    if (entityFiltersType != TimelineEntityFiltersType.METRIC) {
+      return false;
+    }
+    // We expect only integral values(short/int/long) for metric filters.
+    if (!isIntegralValue(compareFilter.getValue())) {
+      throw new IOException("Metric filters has non integral values");
+    }
+    Map<String, TimelineMetric> metricMap =
+        new HashMap<String, TimelineMetric>();
+    for (TimelineMetric metric : entity.getMetrics()) {
+      metricMap.put(metric.getId(), metric);
     }
+    TimelineMetric metric = metricMap.get(compareFilter.getKey());
+    if (metric == null) {
+      return false;
+    }
+    // We will be using the latest value of metric to compare.
+    return compareValues(compareFilter.getCompareOp(),
+        metric.getValuesJAXB().firstEntry().getValue().longValue(),
+        ((Number)compareFilter.getValue()).longValue());
+  }
 
-    for (String metricFilter : metricFilters) {
-      if (!metricIds.contains(metricFilter)) {
-        return false;
+  /**
+   * Matches metric filters.
+   *
+   * @param entity entity which holds a set of metric objects.
+   * @param metricFilters list of metric filters.
+   * @return a boolean flag to indicate if both match.
+   * @throws IOException if an unsupported filter for matching metric filters is
+   *     being matched.
+   */
+  public static boolean matchMetricFilters(TimelineEntity entity,
+      TimelineFilterList metricFilters) throws IOException {
+    return matchFilters(
+        entity, metricFilters, TimelineEntityFiltersType.METRIC);
+  }
+
+  /**
+   * Common routine to match different filters. Iterates over a filter list and
+   * calls routines based on filter type.
+   *
+   * @param entity Timeline entity.
+   * @param filters filter list.
+   * @param entityFiltersType type of filters which are being matched.
+   * @return a boolean flag to indicate if filter matches.
+   * @throws IOException if an unsupported filter for matching this specific
+   *     filter is being matched.
+   */
+  private static boolean matchFilters(TimelineEntity entity,
+      TimelineFilterList filters, TimelineEntityFiltersType entityFiltersType)
+      throws IOException {
+    if (filters == null || filters.getFilterList().isEmpty()) {
+      return false;
+    }
+    TimelineFilterList.Operator operator = filters.getOperator();
+    for (TimelineFilter filter : filters.getFilterList()) {
+      TimelineFilterType filterType = filter.getFilterType();
+      if (!entityFiltersType.isValidFilter(filterType)) {
+        throw new IOException("Unsupported filter " + filterType);
+      }
+      boolean matched = false;
+      switch (filterType) {
+      case LIST:
+        matched = matchFilters(entity, (TimelineFilterList)filter,
+            entityFiltersType);
+        break;
+      case COMPARE:
+        matched = matchCompareFilter(entity, (TimelineCompareFilter)filter,
+            entityFiltersType);
+        break;
+      case EXISTS:
+        matched = matchExistsFilter(entity, (TimelineExistsFilter)filter,
+            entityFiltersType);
+        break;
+      case KEY_VALUE:
+        matched = matchKeyValueFilter(entity, (TimelineKeyValueFilter)filter,
+            entityFiltersType);
+        break;
+      case KEY_VALUES:
+        matched = matchKeyValuesFilter(entity, (TimelineKeyValuesFilter)filter,
+            entityFiltersType);
+        break;
+      default:
+        throw new IOException("Unsupported filter " + filterType);
+      }
+      if (!matched) {
+        if(operator == TimelineFilterList.Operator.AND) {
+          return false;
+        }
+      } else {
+        if(operator == TimelineFilterList.Operator.OR) {
+          return true;
+        }
       }
     }
-    return true;
+    return operator == TimelineFilterList.Operator.AND;
   }
 
   /**
@@ -530,4 +791,100 @@ public final class TimelineStorageUtils {
     }
     return appId;
   }
+
+  /**
+   * Helper method for reading relationship.
+   *
+   * @param <T> Describes the type of column prefix.
+   * @param entity entity to fill.
+   * @param result result from HBase.
+   * @param prefix column prefix.
+   * @param isRelatedTo if true, means relationship is to be added to
+   *     isRelatedTo, otherwise its added to relatesTo.
+   * @throws IOException if any problem is encountered while reading result.
+   */
+  public 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);
+        }
+      }
+    }
+  }
+
+  /**
+   * Helper method for reading key-value pairs for either info or config.
+   *
+   * @param <T> Describes the type of column prefix.
+   * @param entity entity to fill.
+   * @param result result from HBase.
+   * @param prefix column prefix.
+   * @param isConfig if true, means we are reading configs, otherwise info.
+   * @throws IOException if any problem is encountered while reading result.
+   */
+  public 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.
+   *
+   * @param <T> Describes the type of column prefix.
+   * @param entity entity to fill.
+   * @param result HBase Result.
+   * @param prefix column prefix.
+   * @throws IOException if any problem is encountered while reading result.
+   */
+  public static <T> void readEvents(TimelineEntity entity, Result result,
+      ColumnPrefix<T> prefix) throws IOException {
+    Map<String, TimelineEvent> eventsMap = new HashMap<>();
+    Map<?, Object> eventsResult =
+        prefix.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/366eb54e/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 f47ba93..775879a 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
@@ -24,8 +24,11 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
 /**
@@ -46,7 +49,8 @@ public enum EntityColumn implements Column<EntityTable> {
   /**
    * When the entity was created.
    */
-  CREATED_TIME(EntityColumnFamily.INFO, "created_time"),
+  CREATED_TIME(EntityColumnFamily.INFO, "created_time",
+      LongConverter.getInstance()),
 
   /**
    * The version of the flow that this entity belongs to.
@@ -60,12 +64,17 @@ public enum EntityColumn implements Column<EntityTable> {
 
   EntityColumn(ColumnFamily<EntityTable> columnFamily,
       String columnQualifier) {
+    this(columnFamily, columnQualifier, GenericConverter.getInstance());
+  }
+
+  EntityColumn(ColumnFamily<EntityTable> columnFamily,
+      String columnQualifier, ValueConverter converter) {
     this.columnFamily = columnFamily;
     this.columnQualifier = columnQualifier;
     // Future-proof by ensuring the right column prefix hygiene.
     this.columnQualifierBytes =
         Bytes.toBytes(Separator.SPACE.encode(columnQualifier));
-    this.column = new ColumnHelper<EntityTable>(columnFamily);
+    this.column = new ColumnHelper<EntityTable>(columnFamily, converter);
   }
 
   /**
@@ -108,6 +117,21 @@ public enum EntityColumn implements Column<EntityTable> {
     return null;
   }
 
+  @Override
+  public byte[] getColumnQualifierBytes() {
+    return columnQualifierBytes.clone();
+  }
+
+  @Override
+  public byte[] getColumnFamilyBytes() {
+    return columnFamily.getBytes();
+  }
+
+  @Override
+  public ValueConverter getValueConverter() {
+    return column.getValueConverter();
+  }
+
   /**
    * Retrieve an {@link EntityColumn} given a name, or null if there is no
    * match. The following holds true: {@code columnFor(a,x) == columnFor(b,y)}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 f3c7e7f..de2b29d 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
@@ -56,7 +56,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
   /**
    * Lifecycle events for an entity.
    */
-  EVENT(EntityColumnFamily.INFO, "e"),
+  EVENT(EntityColumnFamily.INFO, "e", true),
 
   /**
    * Config column stores configuration with config key as the column name.
@@ -78,6 +78,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    */
   private final String columnPrefix;
   private final byte[] columnPrefixBytes;
+  private final boolean compoundColQual;
 
   /**
    * Private constructor, meant to be used by the enum definition.
@@ -87,7 +88,18 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    */
   EntityColumnPrefix(ColumnFamily<EntityTable> columnFamily,
       String columnPrefix) {
-    this(columnFamily, columnPrefix, GenericConverter.getInstance());
+    this(columnFamily, columnPrefix, false, GenericConverter.getInstance());
+  }
+
+  EntityColumnPrefix(ColumnFamily<EntityTable> columnFamily,
+      String columnPrefix, boolean compondColQual) {
+    this(columnFamily, columnPrefix, compondColQual,
+        GenericConverter.getInstance());
+  }
+
+  EntityColumnPrefix(ColumnFamily<EntityTable> columnFamily,
+      String columnPrefix, ValueConverter converter) {
+    this(columnFamily, columnPrefix, false, converter);
   }
 
   /**
@@ -99,7 +111,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
    * this column prefix.
    */
   EntityColumnPrefix(ColumnFamily<EntityTable> columnFamily,
-      String columnPrefix, ValueConverter converter) {
+      String columnPrefix, boolean compondColQual, ValueConverter converter) {
     column = new ColumnHelper<EntityTable>(columnFamily, converter);
     this.columnFamily = columnFamily;
     this.columnPrefix = columnPrefix;
@@ -110,6 +122,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
       this.columnPrefixBytes =
           Bytes.toBytes(Separator.SPACE.encode(columnPrefix));
     }
+    this.compoundColQual = compondColQual;
   }
 
   /**
@@ -131,6 +144,24 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
         this.columnPrefixBytes, qualifierPrefix);
   }
 
+  @Override
+  public byte[] getColumnFamilyBytes() {
+    return columnFamily.getBytes();
+  }
+
+  @Override
+  public ValueConverter getValueConverter() {
+    return column.getValueConverter();
+  }
+
+  public byte[] getCompoundColQualBytes(String qualifier,
+      byte[]...components) {
+    if (!compoundColQual) {
+      return ColumnHelper.getColumnQualifier(null, qualifier);
+    }
+    return ColumnHelper.getCompoundColumnQualifierBytes(qualifier, components);
+  }
+
   /*
    * (non-Javadoc)
    *
@@ -287,5 +318,4 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
     // Default to null
     return null;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 a5933da..188c2fe 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
@@ -29,6 +29,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.common.TypedBufferedMutator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 
 /**
  * Identifies partially qualified columns for the {@link FlowActivityTable}.
@@ -50,6 +51,7 @@ public enum FlowActivityColumnPrefix
    */
   private final String columnPrefix;
   private final byte[] columnPrefixBytes;
+  private final boolean compoundColQual;
 
   private final AggregationOperation aggOp;
 
@@ -64,6 +66,12 @@ public enum FlowActivityColumnPrefix
   private FlowActivityColumnPrefix(
       ColumnFamily<FlowActivityTable> columnFamily, String columnPrefix,
       AggregationOperation aggOp) {
+    this(columnFamily, columnPrefix, aggOp, false);
+  }
+
+  private FlowActivityColumnPrefix(
+      ColumnFamily<FlowActivityTable> columnFamily, String columnPrefix,
+      AggregationOperation aggOp, boolean compoundColQual) {
     column = new ColumnHelper<FlowActivityTable>(columnFamily);
     this.columnFamily = columnFamily;
     this.columnPrefix = columnPrefix;
@@ -75,6 +83,7 @@ public enum FlowActivityColumnPrefix
           .encode(columnPrefix));
     }
     this.aggOp = aggOp;
+    this.compoundColQual = compoundColQual;
   }
 
   /**
@@ -100,6 +109,16 @@ public enum FlowActivityColumnPrefix
     return columnPrefixBytes.clone();
   }
 
+  @Override
+  public byte[] getColumnFamilyBytes() {
+    return columnFamily.getBytes();
+  }
+
+  @Override
+  public ValueConverter getValueConverter() {
+    return column.getValueConverter();
+  }
+
   public AggregationOperation getAttribute() {
     return aggOp;
   }
@@ -251,4 +270,20 @@ public enum FlowActivityColumnPrefix
     column.store(rowKey, tableMutator, columnQualifier, null, inputValue,
         combinedAttributes);
   }
+
+  @Override
+  public byte[] getCompoundColQualBytes(String qualifier,
+      byte[]...components) {
+    if (!compoundColQual) {
+      return ColumnHelper.getColumnQualifier(null, qualifier);
+    }
+    return ColumnHelper.getCompoundColumnQualifierBytes(qualifier, components);
+  }
+
+  @Override
+  public Map<?, Object> readResultsHavingCompoundColumnQualifiers(Result result)
+      throws IOException {
+    // There are no compound column qualifiers for flow activity table.
+    return null;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 d50bb16..f1553b8 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
@@ -86,10 +86,12 @@ public enum FlowRunColumn implements Column<FlowRunTable> {
     return columnQualifier;
   }
 
+  @Override
   public byte[] getColumnQualifierBytes() {
     return columnQualifierBytes.clone();
   }
 
+  @Override
   public byte[] getColumnFamilyBytes() {
     return columnFamily.getBytes();
   }
@@ -144,6 +146,7 @@ public enum FlowRunColumn implements Column<FlowRunTable> {
     return null;
   }
 
+  @Override
   public ValueConverter getValueConverter() {
     return column.getValueConverter();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 fa94aae..77f2ab2 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
@@ -52,6 +52,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
    */
   private final String columnPrefix;
   private final byte[] columnPrefixBytes;
+  private final boolean compoundColQual;
 
   private final AggregationOperation aggOp;
 
@@ -65,6 +66,12 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
    */
   private FlowRunColumnPrefix(ColumnFamily<FlowRunTable> columnFamily,
       String columnPrefix, AggregationOperation fra, ValueConverter converter) {
+    this(columnFamily, columnPrefix, fra, converter, false);
+  }
+
+  private FlowRunColumnPrefix(ColumnFamily<FlowRunTable> columnFamily,
+      String columnPrefix, AggregationOperation fra, ValueConverter converter,
+      boolean compoundColQual) {
     column = new ColumnHelper<FlowRunTable>(columnFamily, converter);
     this.columnFamily = columnFamily;
     this.columnPrefix = columnPrefix;
@@ -76,6 +83,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
           .encode(columnPrefix));
     }
     this.aggOp = fra;
+    this.compoundColQual = compoundColQual;
   }
 
   /**
@@ -101,6 +109,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
         this.columnPrefixBytes, qualifierPrefix);
   }
 
+  @Override
   public byte[] getColumnFamilyBytes() {
     return columnFamily.getBytes();
   }
@@ -222,6 +231,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
     return null;
   }
 
+  @Override
   public ValueConverter getValueConverter() {
     return column.getValueConverter();
   }
@@ -257,4 +267,20 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
     // Default to null
     return null;
   }
+
+  @Override
+  public byte[] getCompoundColQualBytes(String qualifier,
+      byte[]...components) {
+    if (!compoundColQual) {
+      return ColumnHelper.getColumnQualifier(null, qualifier);
+    }
+    return ColumnHelper.getCompoundColumnQualifierBytes(qualifier, components);
+  }
+
+  @Override
+  public Map<?, Object> readResultsHavingCompoundColumnQualifiers(Result result)
+      throws IOException {
+    // There are no compound column qualifiers for flow run table.
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 6baea37..0ace529 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
@@ -200,6 +200,7 @@ class FlowScanner implements RegionScanner, Closeable {
     int addedCnt = 0;
     long currentTimestamp = System.currentTimeMillis();
     ValueConverter converter = null;
+
     while (cellLimit <= 0 || addedCnt < cellLimit) {
       cell = peekAtNextCell(cellLimit);
       if (cell == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
index 0de09e0..53210f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/ApplicationEntityReader.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.reader;
 
 import java.io.IOException;
 import java.util.EnumSet;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
@@ -27,7 +28,6 @@ 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;
@@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntityType;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.reader.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;
@@ -76,93 +77,231 @@ class ApplicationEntityReader extends GenericEntityReader {
     return APPLICATION_TABLE;
   }
 
+  /**
+   * This method is called only for multiple entity reads.
+   */
   @Override
-  protected FilterList constructFilterListBasedOnFields() {
-    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
-    TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
-    // Fetch all the columns.
-    if (dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
-        (dataToRetrieve.getConfsToRetrieve() == null ||
-        dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) &&
-        (dataToRetrieve.getMetricsToRetrieve() == null ||
-        dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty())) {
-      return list;
+  protected FilterList constructFilterListBasedOnFilters() throws IOException {
+    // Filters here cannot be null for multiple entity reads as they are set in
+    // augmentParams if null.
+    TimelineEntityFilters filters = getFilters();
+    FilterList listBasedOnFilters = new FilterList();
+    // Create filter list based on created time range and add it to
+    // listBasedOnFilters.
+    long createdTimeBegin = filters.getCreatedTimeBegin();
+    long createdTimeEnd = filters.getCreatedTimeEnd();
+    if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) {
+      listBasedOnFilters.addFilter(
+          TimelineFilterUtils.createSingleColValueFiltersByRange(
+          ApplicationColumn.CREATED_TIME, createdTimeBegin, createdTimeEnd));
     }
-    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);
+    // Create filter list based on metric filters and add it to
+    // listBasedOnFilters.
+    TimelineFilterList metricFilters = filters.getMetricFilters();
+    if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) {
+      listBasedOnFilters.addFilter(
+          TimelineFilterUtils.createHBaseFilterList(
+              ApplicationColumnPrefix.METRIC, metricFilters));
+    }
+    // Create filter list based on config filters and add it to
+    // listBasedOnFilters.
+    TimelineFilterList configFilters = filters.getConfigFilters();
+    if (configFilters != null && !configFilters.getFilterList().isEmpty()) {
+      listBasedOnFilters.addFilter(
+          TimelineFilterUtils.createHBaseFilterList(
+              ApplicationColumnPrefix.CONFIG, configFilters));
+    }
+    // Create filter list based on info filters and add it to listBasedOnFilters
+    TimelineFilterList infoFilters = filters.getInfoFilters();
+    if (infoFilters != null && !infoFilters.getFilterList().isEmpty()) {
+      listBasedOnFilters.addFilter(
+          TimelineFilterUtils.createHBaseFilterList(
+              ApplicationColumnPrefix.INFO, infoFilters));
+    }
+    return listBasedOnFilters;
+  }
+
+  /**
+   * Add {@link QualifierFilter} filters to filter list for each column of
+   * application table.
+   *
+   * @param list filter list to which qualifier filters have to be added.
+   */
+  @Override
+  protected void updateFixedColumns(FilterList list) {
+    for (ApplicationColumn column : ApplicationColumn.values()) {
+      list.addFilter(new QualifierFilter(CompareOp.EQUAL,
+          new BinaryComparator(column.getColumnQualifierBytes())));
+    }
+  }
+
+  /**
+   * Creates a filter list which indicates that only some of the column
+   * qualifiers in the info column family will be returned in result.
+   *
+   * @return filter list.
+   * @throws IOException if any problem occurs while creating filter list.
+   */
+  private FilterList createFilterListForColsOfInfoFamily()
+      throws IOException {
+    FilterList infoFamilyColsFilter = new FilterList(Operator.MUST_PASS_ONE);
+    // Add filters for each column in entity table.
+    updateFixedColumns(infoFamilyColsFilter);
+    EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
+    // If INFO field has to be retrieved, add a filter for fetching columns
+    // with INFO column prefix.
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.EQUAL, ApplicationColumnPrefix.INFO));
+    }
+    TimelineFilterList relatesTo = getFilters().getRelatesTo();
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
+      // If RELATES_TO field has to be retrieved, add a filter for fetching
+      // columns with RELATES_TO column prefix.
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.EQUAL, ApplicationColumnPrefix.RELATES_TO));
+    } else if (relatesTo != null && !relatesTo.getFilterList().isEmpty()) {
+      // Even if fields to retrieve does not contain RELATES_TO, we still
+      // need to have a filter to fetch some of the column qualifiers if
+      // relatesTo filters are specified. relatesTo filters will then be
+      // matched after fetching rows from HBase.
+      Set<String> relatesToCols =
+          TimelineFilterUtils.fetchColumnsFromFilterList(relatesTo);
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createFiltersFromColumnQualifiers(
+              ApplicationColumnPrefix.RELATES_TO, relatesToCols));
+    }
+    TimelineFilterList isRelatedTo = getFilters().getIsRelatedTo();
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
+      // If IS_RELATED_TO field has to be retrieved, add a filter for fetching
+      // columns with IS_RELATED_TO column prefix.
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.EQUAL, ApplicationColumnPrefix.IS_RELATED_TO));
+    } else if (isRelatedTo != null && !isRelatedTo.getFilterList().isEmpty()) {
+      // Even if fields to retrieve does not contain IS_RELATED_TO, we still
+      // need to have a filter to fetch some of the column qualifiers if
+      // isRelatedTo filters are specified. isRelatedTo filters will then be
+      // matched after fetching rows from HBase.
+      Set<String> isRelatedToCols =
+          TimelineFilterUtils.fetchColumnsFromFilterList(isRelatedTo);
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createFiltersFromColumnQualifiers(
+              ApplicationColumnPrefix.IS_RELATED_TO, isRelatedToCols));
+    }
+    TimelineFilterList eventFilters = getFilters().getEventFilters();
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
+      // If EVENTS field has to be retrieved, add a filter for fetching columns
+      // with EVENT column prefix.
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.EQUAL, ApplicationColumnPrefix.EVENT));
+    } else if (eventFilters != null && !eventFilters.getFilterList().isEmpty()){
+      // Even if fields to retrieve does not contain EVENTS, we still need to
+      // have a filter to fetch some of the column qualifiers on the basis of
+      // event filters specified. Event filters will then be matched after
+      // fetching rows from HBase.
+      Set<String> eventCols =
+          TimelineFilterUtils.fetchColumnsFromFilterList(eventFilters);
+      infoFamilyColsFilter.addFilter(
+          TimelineFilterUtils.createFiltersFromColumnQualifiers(
+              ApplicationColumnPrefix.EVENT, eventCols));
+    }
+    return infoFamilyColsFilter;
+  }
+
+  /**
+   * Exclude column prefixes via filters which are not required(based on fields
+   * to retrieve) from info column family. These filters are added to filter
+   * list which contains a filter for getting info column family.
+   *
+   * @param infoColFamilyList filter list for info column family.
+   */
+  private void excludeFieldsFromInfoColFamily(FilterList infoColFamilyList) {
+    EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
     // Events not required.
-    TimelineEntityFilters filters = getFilters();
-    if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.EVENTS) &&
-        !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
-        (isSingleEntityRead() || filters.getEventFilters() == null)) {
+    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
       infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-          ApplicationColumnPrefix.EVENT.getColumnPrefixBytes(""))));
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.NOT_EQUAL, ApplicationColumnPrefix.EVENT));
     }
     // info not required.
-    if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.INFO) &&
-        !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
-        (isSingleEntityRead() || filters.getInfoFilters() == null)) {
+    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
       infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-          ApplicationColumnPrefix.INFO.getColumnPrefixBytes(""))));
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.NOT_EQUAL, ApplicationColumnPrefix.INFO));
     }
-    // is releated to not required.
-    if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.IS_RELATED_TO) &&
-        !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
-        (isSingleEntityRead() || filters.getIsRelatedTo() == null)) {
+    // is related to not required.
+    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO)) {
       infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-          ApplicationColumnPrefix.IS_RELATED_TO.getColumnPrefixBytes(""))));
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.NOT_EQUAL, ApplicationColumnPrefix.IS_RELATED_TO));
     }
     // relates to not required.
-    if (!dataToRetrieve.getFieldsToRetrieve().contains(Field.RELATES_TO) &&
-        !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL) &&
-        (isSingleEntityRead() || filters.getRelatesTo() == null)) {
+    if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
       infoColFamilyList.addFilter(
-          new QualifierFilter(CompareOp.NOT_EQUAL,
-          new BinaryPrefixComparator(
-          ApplicationColumnPrefix.RELATES_TO.getColumnPrefixBytes(""))));
+          TimelineFilterUtils.createHBaseQualifierFilter(
+              CompareOp.NOT_EQUAL, ApplicationColumnPrefix.RELATES_TO));
     }
-    list.addFilter(infoColFamilyList);
-    if ((dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS) ||
-        (!isSingleEntityRead() && filters.getConfigFilters() != null)) ||
-        (dataToRetrieve.getConfsToRetrieve() != null &&
-        !dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty())) {
-      FilterList filterCfg =
-          new FilterList(new FamilyFilter(CompareOp.EQUAL,
-          new BinaryComparator(ApplicationColumnFamily.CONFIGS.getBytes())));
-      if (dataToRetrieve.getConfsToRetrieve() != null &&
-          !dataToRetrieve.getConfsToRetrieve().getFilterList().isEmpty()) {
-        filterCfg.addFilter(TimelineFilterUtils.createHBaseFilterList(
-            ApplicationColumnPrefix.CONFIG,
-            dataToRetrieve.getConfsToRetrieve()));
-      }
-      list.addFilter(filterCfg);
+  }
+
+  /**
+   * Updates filter list based on fields for confs and metrics to retrieve.
+   *
+   * @param listBasedOnFields filter list based on fields.
+   * @throws IOException if any problem occurs while updating filter list.
+   */
+  private void updateFilterForConfsAndMetricsToRetrieve(
+      FilterList listBasedOnFields) throws IOException {
+    TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
+    // Please note that if confsToRetrieve is specified, we would have added
+    // CONFS to fields to retrieve in augmentParams() even if not specified.
+    if (dataToRetrieve.getFieldsToRetrieve().contains(Field.CONFIGS)) {
+      // Create a filter list for configs.
+      listBasedOnFields.addFilter(TimelineFilterUtils.
+          createFilterForConfsOrMetricsToRetrieve(
+              dataToRetrieve.getConfsToRetrieve(),
+              ApplicationColumnFamily.CONFIGS, ApplicationColumnPrefix.CONFIG));
     }
-    if ((dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS) ||
-        (!isSingleEntityRead() && filters.getMetricFilters() != null)) ||
-        (dataToRetrieve.getMetricsToRetrieve() != null &&
-        !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty())) {
-      FilterList filterMetrics =
-          new FilterList(new FamilyFilter(CompareOp.EQUAL,
-          new BinaryComparator(ApplicationColumnFamily.METRICS.getBytes())));
-      if (dataToRetrieve.getMetricsToRetrieve() != null &&
-          !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty()) {
-        filterMetrics.addFilter(TimelineFilterUtils.createHBaseFilterList(
-            ApplicationColumnPrefix.METRIC,
-            dataToRetrieve.getMetricsToRetrieve()));
-      }
-      list.addFilter(filterMetrics);
+
+    // Please note that if metricsToRetrieve is specified, we would have added
+    // METRICS to fields to retrieve in augmentParams() even if not specified.
+    if (dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS)) {
+      // Create a filter list for metrics.
+      listBasedOnFields.addFilter(TimelineFilterUtils.
+          createFilterForConfsOrMetricsToRetrieve(
+              dataToRetrieve.getMetricsToRetrieve(),
+              ApplicationColumnFamily.METRICS, ApplicationColumnPrefix.METRIC));
+    }
+  }
+
+  @Override
+  protected FilterList constructFilterListBasedOnFields() throws IOException {
+    if (!needCreateFilterListBasedOnFields()) {
+      // Fetch all the columns. No need of a filter.
+      return null;
+    }
+    FilterList listBasedOnFields = new FilterList(Operator.MUST_PASS_ONE);
+    FilterList infoColFamilyList = new FilterList();
+    // By default fetch everything in INFO column family.
+    FamilyFilter infoColumnFamily =
+        new FamilyFilter(CompareOp.EQUAL,
+            new BinaryComparator(ApplicationColumnFamily.INFO.getBytes()));
+    infoColFamilyList.addFilter(infoColumnFamily);
+    if (!isSingleEntityRead() && fetchPartialColsFromInfoFamily()) {
+      // We can fetch only some of the columns from info family.
+      infoColFamilyList.addFilter(createFilterListForColsOfInfoFamily());
+    } else {
+      // Exclude column prefixes in info column family which are not required
+      // based on fields to retrieve.
+      excludeFieldsFromInfoColFamily(infoColFamilyList);
     }
-    return list;
+    listBasedOnFields.addFilter(infoColFamilyList);
+
+    updateFilterForConfsAndMetricsToRetrieve(listBasedOnFields);
+    return listBasedOnFields;
   }
 
   @Override
@@ -182,6 +321,9 @@ class ApplicationEntityReader extends GenericEntityReader {
 
   @Override
   protected void validateParams() {
+    Preconditions.checkNotNull(getContext(), "context shouldn't be null");
+    Preconditions.checkNotNull(
+        getDataToRetrieve(), "data to retrieve shouldn't be null");
     Preconditions.checkNotNull(getContext().getClusterId(),
         "clusterId shouldn't be null");
     Preconditions.checkNotNull(getContext().getEntityType(),
@@ -202,6 +344,7 @@ class ApplicationEntityReader extends GenericEntityReader {
       throws IOException {
     TimelineReaderContext context = getContext();
     if (isSingleEntityRead()) {
+      // Get flow context information from AppToFlow table.
       if (context.getFlowName() == null || context.getFlowRunId() == null ||
           context.getUserId() == null) {
         FlowContext flowContext = lookupFlowContext(
@@ -211,7 +354,12 @@ class ApplicationEntityReader extends GenericEntityReader {
         context.setUserId(flowContext.getUserId());
       }
     }
+    // Add configs/metrics to fields to retrieve if confsToRetrieve and/or
+    // metricsToRetrieve are specified.
     getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve();
+    if (!isSingleEntityRead()) {
+      createFiltersIfNull();
+    }
   }
 
   @Override
@@ -252,114 +400,84 @@ class ApplicationEntityReader extends GenericEntityReader {
     Number createdTime =
         (Number)ApplicationColumn.CREATED_TIME.readResult(result);
     entity.setCreatedTime(createdTime.longValue());
-    if (!isSingleEntityRead() &&
-        (entity.getCreatedTime() < filters.getCreatedTimeBegin() ||
-        entity.getCreatedTime() > filters.getCreatedTimeEnd())) {
-      return null;
-    }
+
     EnumSet<Field> fieldsToRetrieve = getDataToRetrieve().getFieldsToRetrieve();
-    // fetch is related to entities
+    // fetch is related to entities and match isRelatedTo filter. If isRelatedTo
+    // filters do not match, entity would be dropped. We have to match filters
+    // locally as relevant HBase filters to filter out rows on the basis of
+    // isRelatedTo are not set in HBase scan.
     boolean checkIsRelatedTo =
-        filters != null && filters.getIsRelatedTo() != null &&
-        filters.getIsRelatedTo().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(), filters.getIsRelatedTo())) {
+        !isSingleEntityRead() && filters.getIsRelatedTo() != null &&
+        filters.getIsRelatedTo().getFilterList().size() > 0;
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.IS_RELATED_TO) ||
+        checkIsRelatedTo) {
+      TimelineStorageUtils.readRelationship(
+          entity, result, ApplicationColumnPrefix.IS_RELATED_TO, true);
+      if (checkIsRelatedTo && !TimelineStorageUtils.matchIsRelatedTo(entity,
+          filters.getIsRelatedTo())) {
         return null;
       }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.IS_RELATED_TO)) {
+      if (!TimelineStorageUtils.hasField(fieldsToRetrieve,
+          Field.IS_RELATED_TO)) {
         entity.getIsRelatedToEntities().clear();
       }
     }
 
-    // fetch relates to entities
+    // fetch relates to entities and match relatesTo filter. If relatesTo
+    // filters do not match, entity would be dropped. We have to match filters
+    // locally as relevant HBase filters to filter out rows on the basis of
+    // relatesTo are not set in HBase scan.
     boolean checkRelatesTo =
-        filters != null && filters.getRelatesTo() != null &&
-        filters.getRelatesTo().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(), filters.getRelatesTo())) {
+        !isSingleEntityRead() && filters.getRelatesTo() != null &&
+        filters.getRelatesTo().getFilterList().size() > 0;
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO) ||
+        checkRelatesTo) {
+      TimelineStorageUtils.readRelationship(
+          entity, result, ApplicationColumnPrefix.RELATES_TO, false);
+      if (checkRelatesTo && !TimelineStorageUtils.matchRelatesTo(entity,
+          filters.getRelatesTo())) {
         return null;
       }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.RELATES_TO)) {
+      if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.RELATES_TO)) {
         entity.getRelatesToEntities().clear();
       }
     }
 
-    // fetch info
-    boolean checkInfo = filters != null && filters.getInfoFilters() != null &&
-        filters.getInfoFilters().size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.INFO) || checkInfo) {
-      readKeyValuePairs(entity, result, ApplicationColumnPrefix.INFO, false);
-      if (checkInfo &&
-          !TimelineStorageUtils.matchFilters(
-          entity.getInfo(), filters.getInfoFilters())) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.INFO)) {
-        entity.getInfo().clear();
-      }
+    // fetch info if fieldsToRetrieve contains INFO or ALL.
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.INFO)) {
+      TimelineStorageUtils.readKeyValuePairs(
+          entity, result, ApplicationColumnPrefix.INFO, false);
     }
 
-    // fetch configs
-    boolean checkConfigs =
-        filters != null && filters.getConfigFilters() != null &&
-        filters.getConfigFilters().size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.CONFIGS) || checkConfigs) {
-      readKeyValuePairs(entity, result, ApplicationColumnPrefix.CONFIG, true);
-      if (checkConfigs && !TimelineStorageUtils.matchFilters(
-          entity.getConfigs(), filters.getConfigFilters())) {
-        return null;
-      }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.CONFIGS)) {
-        entity.getConfigs().clear();
-      }
+    // fetch configs if fieldsToRetrieve contains CONFIGS or ALL.
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.CONFIGS)) {
+      TimelineStorageUtils.readKeyValuePairs(
+          entity, result, ApplicationColumnPrefix.CONFIG, true);
     }
 
-    // fetch events
+    // fetch events and match event filters if they exist. If event filters do
+    // not match, entity would be dropped. We have to match filters locally
+    // as relevant HBase filters to filter out rows on the basis of events
+    // are not set in HBase scan.
     boolean checkEvents =
-        filters != null && filters.getEventFilters() != null &&
-        filters.getEventFilters().size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.EVENTS) || checkEvents) {
-      readEvents(entity, result, true);
-      if (checkEvents && !TimelineStorageUtils.matchEventFilters(
-          entity.getEvents(), filters.getEventFilters())) {
+        !isSingleEntityRead() && filters.getEventFilters() != null &&
+        filters.getEventFilters().getFilterList().size() > 0;
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS) ||
+        checkEvents) {
+      TimelineStorageUtils.readEvents(
+          entity, result, ApplicationColumnPrefix.EVENT);
+      if (checkEvents && !TimelineStorageUtils.matchEventFilters(entity,
+          filters.getEventFilters())) {
         return null;
       }
-      if (!fieldsToRetrieve.contains(Field.ALL) &&
-          !fieldsToRetrieve.contains(Field.EVENTS)) {
+      if (!TimelineStorageUtils.hasField(fieldsToRetrieve, Field.EVENTS)) {
         entity.getEvents().clear();
       }
     }
 
-    // fetch metrics
-    boolean checkMetrics =
-        filters != null && filters.getMetricFilters() != null &&
-        filters.getMetricFilters().size() > 0;
-    if (fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(Field.METRICS) || checkMetrics) {
+    // fetch metrics if fieldsToRetrieve contains METRICS or ALL.
+    if (TimelineStorageUtils.hasField(fieldsToRetrieve, Field.METRICS)) {
       readMetrics(entity, result, ApplicationColumnPrefix.METRIC);
-      if (checkMetrics && !TimelineStorageUtils.matchMetricFilters(
-          entity.getMetrics(), filters.getMetricFilters())) {
-        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/366eb54e/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
index 0d2bdd8..d8ca038 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/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
@@ -15,6 +15,7 @@
  * 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;
@@ -75,6 +76,12 @@ class FlowActivityEntityReader extends TimelineEntityReader {
   @Override
   protected void augmentParams(Configuration hbaseConf, Connection conn)
       throws IOException {
+    createFiltersIfNull();
+  }
+
+  @Override
+  protected FilterList constructFilterListBasedOnFilters() throws IOException {
+    return null;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
index 743315c..b2de2e5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/FlowRunEntityReader.java
@@ -38,9 +38,11 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 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;
@@ -76,6 +78,9 @@ class FlowRunEntityReader extends TimelineEntityReader {
 
   @Override
   protected void validateParams() {
+    Preconditions.checkNotNull(getContext(), "context shouldn't be null");
+    Preconditions.checkNotNull(
+        getDataToRetrieve(), "data to retrieve shouldn't be null");
     Preconditions.checkNotNull(getContext().getClusterId(),
         "clusterId shouldn't be null");
     Preconditions.checkNotNull(getContext().getUserId(),
@@ -90,37 +95,87 @@ class FlowRunEntityReader extends TimelineEntityReader {
 
   @Override
   protected void augmentParams(Configuration hbaseConf, Connection conn) {
+    // Add metrics to fields to retrieve if metricsToRetrieve is specified.
     getDataToRetrieve().addFieldsBasedOnConfsAndMetricsToRetrieve();
+    if (!isSingleEntityRead()) {
+      createFiltersIfNull();
+    }
+  }
+
+  protected FilterList constructFilterListBasedOnFilters() throws IOException {
+    FilterList listBasedOnFilters = new FilterList();
+    // Filter based on created time range.
+    Long createdTimeBegin = getFilters().getCreatedTimeBegin();
+    Long createdTimeEnd = getFilters().getCreatedTimeEnd();
+    if (createdTimeBegin != 0 || createdTimeEnd != Long.MAX_VALUE) {
+      listBasedOnFilters.addFilter(
+          TimelineFilterUtils.createSingleColValueFiltersByRange(
+          FlowRunColumn.MIN_START_TIME, createdTimeBegin, createdTimeEnd));
+    }
+    // Filter based on metric filters.
+    TimelineFilterList metricFilters = getFilters().getMetricFilters();
+    if (metricFilters != null && !metricFilters.getFilterList().isEmpty()) {
+      listBasedOnFilters.addFilter(
+          TimelineFilterUtils.createHBaseFilterList(
+              FlowRunColumnPrefix.METRIC, metricFilters));
+    }
+    return listBasedOnFilters;
+  }
+
+  /**
+   * Add {@link QualifierFilter} filters to filter list for each column of
+   * flow run table.
+   *
+   * @return filter list to which qualifier filters have been added.
+   */
+  private FilterList updateFixedColumns() {
+    FilterList columnsList = new FilterList(Operator.MUST_PASS_ONE);
+    for (FlowRunColumn column : FlowRunColumn.values()) {
+      columnsList.addFilter(new QualifierFilter(CompareOp.EQUAL,
+          new BinaryComparator(column.getColumnQualifierBytes())));
+    }
+    return columnsList;
   }
 
   @Override
-  protected FilterList constructFilterListBasedOnFields() {
+  protected FilterList constructFilterListBasedOnFields() throws IOException {
     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()));
     TimelineDataToRetrieve dataToRetrieve = getDataToRetrieve();
-    // Metrics not required.
-    if (!isSingleEntityRead() &&
-        !dataToRetrieve.getFieldsToRetrieve().contains(Field.METRICS) &&
-        !dataToRetrieve.getFieldsToRetrieve().contains(Field.ALL)) {
+    // If multiple entities have to be retrieved, check if metrics have to be
+    // retrieved and if not, add a filter so that metrics can be excluded.
+    // Metrics are always returned if we are reading a single entity.
+    if (!isSingleEntityRead() && !TimelineStorageUtils.hasField(
+        dataToRetrieve.getFieldsToRetrieve(), Field.METRICS)) {
       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 (dataToRetrieve.getMetricsToRetrieve() != null &&
-        !dataToRetrieve.getMetricsToRetrieve().getFilterList().isEmpty()) {
-      FilterList infoColFamilyList = new FilterList();
-      infoColFamilyList.addFilter(infoColumnFamily);
-      infoColFamilyList.addFilter(TimelineFilterUtils.createHBaseFilterList(
-          FlowRunColumnPrefix.METRIC, dataToRetrieve.getMetricsToRetrieve()));
+              FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(""))));
       list.addFilter(infoColFamilyList);
+    } else {
+      // Check if metricsToRetrieve are specified and if they are, create a
+      // filter list for info column family by adding flow run tables columns
+      // and a list for metrics to retrieve. Pls note that fieldsToRetrieve
+      // will have METRICS added to it if metricsToRetrieve are specified
+      // (in augmentParams()).
+      TimelineFilterList metricsToRetrieve =
+          dataToRetrieve.getMetricsToRetrieve();
+      if (metricsToRetrieve != null &&
+          !metricsToRetrieve.getFilterList().isEmpty()) {
+        FilterList infoColFamilyList = new FilterList();
+        infoColFamilyList.addFilter(infoColumnFamily);
+        FilterList columnsList = updateFixedColumns();
+        columnsList.addFilter(
+            TimelineFilterUtils.createHBaseFilterList(
+                FlowRunColumnPrefix.METRIC, metricsToRetrieve));
+        infoColFamilyList.addFilter(columnsList);
+        list.addFilter(infoColFamilyList);
+      }
     }
     return list;
   }
@@ -175,11 +230,6 @@ class FlowRunEntityReader extends TimelineEntityReader {
     if (startTime != null) {
       flowRun.setStartTime(startTime.longValue());
     }
-    if (!isSingleEntityRead() &&
-        (flowRun.getStartTime() < getFilters().getCreatedTimeBegin() ||
-        flowRun.getStartTime() > getFilters().getCreatedTimeEnd())) {
-      return null;
-    }
 
     // read the end time if available
     Long endTime = (Long)FlowRunColumn.MAX_END_TIME.readResult(result);
@@ -193,9 +243,10 @@ class FlowRunEntityReader extends TimelineEntityReader {
       flowRun.setVersion(version);
     }
 
-    // read metrics
-    if (isSingleEntityRead() ||
-        getDataToRetrieve().getFieldsToRetrieve().contains(Field.METRICS)) {
+    // read metrics if its a single entity query or if METRICS are part of
+    // fieldsToRetrieve.
+    if (isSingleEntityRead() || TimelineStorageUtils.hasField(
+        getDataToRetrieve().getFieldsToRetrieve(), Field.METRICS)) {
       readMetrics(flowRun, result, FlowRunColumnPrefix.METRIC);
     }
 


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


[47/50] [abbrv] hadoop git commit: YARN-5170. Eliminate singleton converters and static method access. (Joep Rottinghuis via Varun Saxena)

Posted by vr...@apache.org.
YARN-5170. Eliminate singleton converters and static method access. (Joep Rottinghuis 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/695e7edc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/695e7edc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/695e7edc

Branch: refs/heads/YARN-2928
Commit: 695e7edcb86b86260b4c5dc65aa6c3135c434a5f
Parents: a005fb8
Author: Varun Saxena <va...@apache.org>
Authored: Fri Jun 10 21:18:05 2016 +0530
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:14 2016 -0700

----------------------------------------------------------------------
 .../storage/TestHBaseTimelineStorage.java       |  43 ++-
 .../flow/TestHBaseStorageFlowActivity.java      |  11 +-
 .../storage/flow/TestHBaseStorageFlowRun.java   |  11 +-
 .../flow/TestHBaseStorageFlowRunCompaction.java |  54 +--
 .../reader/filter/TimelineFilterUtils.java      |  38 ---
 .../storage/HBaseTimelineWriterImpl.java        | 226 ++++++++-----
 .../storage/application/ApplicationColumn.java  |   2 +-
 .../application/ApplicationColumnPrefix.java    |   3 +-
 .../storage/application/ApplicationRowKey.java  | 173 +++++++---
 .../application/ApplicationRowKeyConverter.java | 130 --------
 .../application/ApplicationRowKeyPrefix.java    |  69 ++++
 .../storage/apptoflow/AppToFlowRowKey.java      |  93 +++++-
 .../apptoflow/AppToFlowRowKeyConverter.java     |  96 ------
 .../storage/common/AppIdKeyConverter.java       |  11 +-
 .../storage/common/EventColumnName.java         |  15 +
 .../common/EventColumnNameConverter.java        |  12 +-
 .../storage/common/LongConverter.java           |  27 +-
 .../storage/common/LongKeyConverter.java        |  14 +-
 .../storage/common/RowKeyPrefix.java            |  42 +++
 .../storage/common/StringKeyConverter.java      |   7 +-
 .../storage/common/TimelineStorageUtils.java    | 171 ----------
 .../storage/entity/EntityColumn.java            |   3 +-
 .../storage/entity/EntityColumnPrefix.java      |   3 +-
 .../storage/entity/EntityRowKey.java            | 187 ++++++++---
 .../storage/entity/EntityRowKeyConverter.java   | 143 --------
 .../storage/entity/EntityRowKeyPrefix.java      |  74 +++++
 .../storage/flow/FlowActivityRowKey.java        | 162 ++++++---
 .../flow/FlowActivityRowKeyConverter.java       | 115 -------
 .../storage/flow/FlowActivityRowKeyPrefix.java  |  60 ++++
 .../storage/flow/FlowRunColumn.java             |   6 +-
 .../storage/flow/FlowRunColumnPrefix.java       |   2 +-
 .../storage/flow/FlowRunRowKey.java             | 129 +++++--
 .../storage/flow/FlowRunRowKeyConverter.java    | 120 -------
 .../storage/flow/FlowRunRowKeyPrefix.java       |  54 +++
 .../storage/reader/ApplicationEntityReader.java | 103 +++---
 .../reader/FlowActivityEntityReader.java        |  26 +-
 .../storage/reader/FlowRunEntityReader.java     |  80 ++---
 .../storage/reader/GenericEntityReader.java     | 333 ++++++++++---------
 .../storage/reader/TimelineEntityReader.java    | 136 +++++++-
 .../storage/common/TestKeyConverters.java       | 287 ++++------------
 .../storage/common/TestRowKeys.java             | 244 ++++++++++++++
 41 files changed, 1883 insertions(+), 1632 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/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-hbase-tests/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-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index 7b647eb..fd5a7f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -70,12 +70,14 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
 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.EntityRowKeyPrefix;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -649,8 +651,9 @@ public class TestHBaseTimelineStorage {
 
       infoMap.putAll(infoMap1);
       // retrieve the row
-      byte[] rowKey =
-          ApplicationRowKey.getRowKey(cluster, user, flow, runid, appId);
+      ApplicationRowKey applicationRowKey =
+          new ApplicationRowKey(cluster, user, flow, runid, appId);
+      byte[] rowKey = applicationRowKey.getRowKey();
       Get get = new Get(rowKey);
       get.setMaxVersions(Integer.MAX_VALUE);
       Connection conn = ConnectionFactory.createConnection(c1);
@@ -674,7 +677,7 @@ public class TestHBaseTimelineStorage {
 
       Map<String, Object> infoColumns =
           ApplicationColumnPrefix.INFO.readResults(result,
-              StringKeyConverter.getInstance());
+              new StringKeyConverter());
       assertEquals(infoMap, infoColumns);
 
       // Remember isRelatedTo is of type Map<String, Set<String>>
@@ -710,15 +713,16 @@ public class TestHBaseTimelineStorage {
         }
       }
 
+      KeyConverter<String> stringKeyConverter = new StringKeyConverter();
       // Configuration
       Map<String, Object> configColumns =
-          ApplicationColumnPrefix.CONFIG.readResults(result,
-              StringKeyConverter.getInstance());
+          ApplicationColumnPrefix.CONFIG
+              .readResults(result, stringKeyConverter);
       assertEquals(conf, configColumns);
 
       NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
-          ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(
-              result, StringKeyConverter.getInstance());
+          ApplicationColumnPrefix.METRIC.readResultsWithTimestamps(result,
+              stringKeyConverter);
 
       NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
       matchMetrics(metricValues, metricMap);
@@ -908,7 +912,8 @@ public class TestHBaseTimelineStorage {
       // scan the table and see that entity exists
       Scan s = new Scan();
       byte[] startRow =
-          EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
+          new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
+              .getRowKeyPrefix();
       s.setStartRow(startRow);
       s.setMaxVersions(Integer.MAX_VALUE);
       Connection conn = ConnectionFactory.createConnection(c1);
@@ -916,6 +921,7 @@ public class TestHBaseTimelineStorage {
 
       int rowCount = 0;
       int colCount = 0;
+      KeyConverter<String> stringKeyConverter = new StringKeyConverter();
       for (Result result : scanner) {
         if (result != null && !result.isEmpty()) {
           rowCount++;
@@ -936,7 +942,7 @@ public class TestHBaseTimelineStorage {
 
           Map<String, Object> infoColumns =
               EntityColumnPrefix.INFO.readResults(result,
-                  StringKeyConverter.getInstance());
+                  new StringKeyConverter());
           assertEquals(infoMap, infoColumns);
 
           // Remember isRelatedTo is of type Map<String, Set<String>>
@@ -975,13 +981,12 @@ public class TestHBaseTimelineStorage {
 
           // Configuration
           Map<String, Object> configColumns =
-              EntityColumnPrefix.CONFIG.readResults(result,
-                  StringKeyConverter.getInstance());
+              EntityColumnPrefix.CONFIG.readResults(result, stringKeyConverter);
           assertEquals(conf, configColumns);
 
           NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
-              EntityColumnPrefix.METRIC.readResultsWithTimestamps(
-                  result, StringKeyConverter.getInstance());
+              EntityColumnPrefix.METRIC.readResultsWithTimestamps(result,
+                  stringKeyConverter);
 
           NavigableMap<Long, Number> metricMap = metricsResult.get(m1.getId());
           matchMetrics(metricValues, metricMap);
@@ -1116,8 +1121,9 @@ public class TestHBaseTimelineStorage {
       hbi.stop();
 
       // retrieve the row
-      byte[] rowKey =
-          ApplicationRowKey.getRowKey(cluster, user, flow, runid, appName);
+      ApplicationRowKey applicationRowKey =
+          new ApplicationRowKey(cluster, user, flow, runid, appName);
+      byte[] rowKey = applicationRowKey.getRowKey();
       Get get = new Get(rowKey);
       get.setMaxVersions(Integer.MAX_VALUE);
       Connection conn = ConnectionFactory.createConnection(c1);
@@ -1132,7 +1138,7 @@ public class TestHBaseTimelineStorage {
 
       Map<EventColumnName, Object> eventsResult =
           ApplicationColumnPrefix.EVENT.readResults(result,
-              EventColumnNameConverter.getInstance());
+              new EventColumnNameConverter());
       // there should be only one event
       assertEquals(1, eventsResult.size());
       for (Map.Entry<EventColumnName, Object> e : eventsResult.entrySet()) {
@@ -1212,7 +1218,8 @@ public class TestHBaseTimelineStorage {
       String appName = ApplicationId.newInstance(System.currentTimeMillis() +
           9000000L, 1).toString();
       byte[] startRow =
-          EntityRowKey.getRowKeyPrefix(cluster, user, flow, runid, appName);
+          new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
+              .getRowKeyPrefix();
       hbi.write(cluster, user, flow, flowVersion, runid, appName, entities);
       hbi.stop();
       // scan the table and see that entity exists
@@ -1234,7 +1241,7 @@ public class TestHBaseTimelineStorage {
 
           Map<EventColumnName, Object> eventsResult =
               EntityColumnPrefix.EVENT.readResults(result,
-                  EventColumnNameConverter.getInstance());
+                  new EventColumnNameConverter());
           // there should be only one event
           assertEquals(1, eventsResult.size());
           for (Map.Entry<EventColumnName, Object> e : eventsResult.entrySet()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
index 589b78d..37490ff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowActivity.java
@@ -158,7 +158,7 @@ public class TestHBaseStorageFlowActivity {
     Table table1 = conn.getTable(TableName
         .valueOf(FlowActivityTable.DEFAULT_TABLE_NAME));
     byte[] startRow =
-        FlowActivityRowKey.getRowKey(cluster, minStartTs, user, flow);
+        new FlowActivityRowKey(cluster, minStartTs, user, flow).getRowKey();
     Get g = new Get(startRow);
     Result r1 = table1.get(g);
     assertNotNull(r1);
@@ -278,11 +278,12 @@ public class TestHBaseStorageFlowActivity {
     Scan s = new Scan();
     s.addFamily(FlowActivityColumnFamily.INFO.getBytes());
     byte[] startRow =
-        FlowActivityRowKey.getRowKey(cluster, appCreatedTime, user, flow);
+        new FlowActivityRowKey(cluster, appCreatedTime, user, flow).getRowKey();
     s.setStartRow(startRow);
     String clusterStop = cluster + "1";
     byte[] stopRow =
-        FlowActivityRowKey.getRowKey(clusterStop, appCreatedTime, user, flow);
+        new FlowActivityRowKey(clusterStop, appCreatedTime, user, flow)
+            .getRowKey();
     s.setStopRow(stopRow);
     Connection conn = ConnectionFactory.createConnection(c1);
     Table table1 = conn.getTable(TableName
@@ -420,11 +421,11 @@ public class TestHBaseStorageFlowActivity {
     Scan s = new Scan();
     s.addFamily(FlowActivityColumnFamily.INFO.getBytes());
     byte[] startRow =
-        FlowActivityRowKey.getRowKey(cluster, appCreatedTime, user, flow);
+        new FlowActivityRowKey(cluster, appCreatedTime, user, flow).getRowKey();
     s.setStartRow(startRow);
     String clusterStop = cluster + "1";
     byte[] stopRow =
-        FlowActivityRowKey.getRowKey(clusterStop, appCreatedTime, user, flow);
+        new FlowActivityRowKey(clusterStop, appCreatedTime, user, flow).getRowKey();
     s.setStopRow(stopRow);
     Connection conn = ConnectionFactory.createConnection(c1);
     Table table1 = conn.getTable(TableName

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
index a443b50..328b25a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
@@ -59,8 +59,8 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilte
 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.TimelineSchemaCreator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
@@ -224,7 +224,7 @@ public class TestHBaseStorageFlowRun {
         .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);
+    byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
     Get g = new Get(startRow);
     g.addColumn(FlowRunColumnFamily.INFO.getBytes(),
         FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes());
@@ -354,10 +354,11 @@ public class TestHBaseStorageFlowRun {
       long runid, Configuration c1) throws IOException {
     Scan s = new Scan();
     s.addFamily(FlowRunColumnFamily.INFO.getBytes());
-    byte[] startRow = FlowRunRowKey.getRowKey(cluster, user, flow, runid);
+    byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
     s.setStartRow(startRow);
     String clusterStop = cluster + "1";
-    byte[] stopRow = FlowRunRowKey.getRowKey(clusterStop, user, flow, runid);
+    byte[] stopRow =
+        new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey();
     s.setStopRow(stopRow);
     Connection conn = ConnectionFactory.createConnection(c1);
     Table table1 = conn.getTable(TableName
@@ -629,7 +630,7 @@ public class TestHBaseStorageFlowRun {
         .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);
+    byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
     Get g = new Get(startRow);
     g.addColumn(FlowRunColumnFamily.INFO.getBytes(),
         FlowRunColumn.MIN_START_TIME.getColumnQualifierBytes());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
index 6b0ee5c..e1bef53 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
@@ -19,24 +19,24 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertNotEquals;
 
 import java.io.IOException;
-import java.util.Map;
+import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.ArrayList;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Get;
@@ -46,21 +46,21 @@ 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.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 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.TimestampGenerator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 
 /**
  * Tests the FlowRun and FlowActivity Tables
@@ -194,10 +194,11 @@ public class TestHBaseStorageFlowRunCompaction {
       long runid, Configuration c1, int valueCount) throws IOException {
     Scan s = new Scan();
     s.addFamily(FlowRunColumnFamily.INFO.getBytes());
-    byte[] startRow = FlowRunRowKey.getRowKey(cluster, user, flow, runid);
+    byte[] startRow = new FlowRunRowKey(cluster, user, flow, runid).getRowKey();
     s.setStartRow(startRow);
     String clusterStop = cluster + "1";
-    byte[] stopRow = FlowRunRowKey.getRowKey(clusterStop, user, flow, runid);
+    byte[] stopRow =
+        new FlowRunRowKey(clusterStop, user, flow, runid).getRowKey();
     s.setStopRow(stopRow);
     Connection conn = ConnectionFactory.createConnection(c1);
     Table table1 = conn.getTable(TableName
@@ -302,8 +303,9 @@ public class TestHBaseStorageFlowRunCompaction {
         cell4Ts, Bytes.toBytes(cellValue4), tagByteArray);
     currentColumnCells.add(c4);
 
-    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
+    List<Cell> cells =
+        fs.processSummationMajorCompaction(currentColumnCells,
+            new LongConverter(), currentTimestamp);
     assertNotNull(cells);
 
     // we should be getting back 4 cells
@@ -387,8 +389,9 @@ public class TestHBaseStorageFlowRunCompaction {
       cellTsNotFinal++;
     }
 
-    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
+    List<Cell> cells =
+        fs.processSummationMajorCompaction(currentColumnCells,
+            new LongConverter(), currentTimestamp);
     assertNotNull(cells);
 
     // we should be getting back count + 1 cells
@@ -489,8 +492,9 @@ public class TestHBaseStorageFlowRunCompaction {
       cellTsNotFinal++;
     }
 
-    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
+    List<Cell> cells =
+        fs.processSummationMajorCompaction(currentColumnCells,
+            new LongConverter(), currentTimestamp);
     assertNotNull(cells);
 
     // we should be getting back
@@ -554,7 +558,7 @@ public class TestHBaseStorageFlowRunCompaction {
         130L, Bytes.toBytes(cellValue2), tagByteArray);
     currentColumnCells.add(c2);
     List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
+        new LongConverter(), currentTimestamp);
     assertNotNull(cells);
 
     // we should be getting back two cells
@@ -602,7 +606,7 @@ public class TestHBaseStorageFlowRunCompaction {
     currentColumnCells.add(c1);
 
     List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
+        new LongConverter(), currentTimestamp);
     assertNotNull(cells);
     // we should not get the same cell back
     // but we get back the flow cell
@@ -639,7 +643,7 @@ public class TestHBaseStorageFlowRunCompaction {
         currentTimestamp, Bytes.toBytes(1110L), tagByteArray);
     currentColumnCells.add(c1);
     List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
+        new LongConverter(), currentTimestamp);
     assertNotNull(cells);
     // we expect the same cell back
     assertEquals(1, cells.size());
@@ -653,15 +657,19 @@ public class TestHBaseStorageFlowRunCompaction {
     FlowScanner fs = getFlowScannerForTestingCompaction();
     long currentTimestamp = System.currentTimeMillis();
 
+    LongConverter longConverter = new LongConverter();
+
     SortedSet<Cell> currentColumnCells = null;
-    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
+    List<Cell> cells =
+        fs.processSummationMajorCompaction(currentColumnCells, longConverter,
+            currentTimestamp);
     assertNotNull(cells);
     assertEquals(0, cells.size());
 
     currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
-    cells = fs.processSummationMajorCompaction(currentColumnCells,
-        LongConverter.getInstance(), currentTimestamp);
+    cells =
+        fs.processSummationMajorCompaction(currentColumnCells, longConverter,
+            currentTimestamp);
     assertNotNull(cells);
     assertEquals(0, cells.size());
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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
index 036746b..cccae26 100644
--- 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
@@ -31,14 +31,9 @@ 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.application.ApplicationColumnPrefix;
 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.ColumnPrefix;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityColumnPrefix;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 
@@ -209,39 +204,6 @@ public final class TimelineFilterUtils {
     return singleColValFilter;
   }
 
-  private static <T> byte[] createColQualifierPrefix(ColumnPrefix<T> colPrefix,
-      String column) {
-    if (colPrefix == ApplicationColumnPrefix.EVENT ||
-        colPrefix == EntityColumnPrefix.EVENT) {
-      return EventColumnNameConverter.getInstance().encode(
-          new EventColumnName(column, null, null));
-    } else {
-      return StringKeyConverter.getInstance().encode(column);
-    }
-  }
-
-  /**
-   * Create a filter list of qualifier filters based on passed set of columns.
-   *
-   * @param <T> Describes the type of column prefix.
-   * @param colPrefix Column Prefix.
-   * @param columns set of column qualifiers.
-   * @return filter list.
-   */
-  public static <T> FilterList createFiltersFromColumnQualifiers(
-      ColumnPrefix<T> colPrefix, Set<String> columns) {
-    FilterList list = new FilterList(Operator.MUST_PASS_ONE);
-    for (String column : columns) {
-      // For columns which have compound column qualifiers (eg. events), we need
-      // to include the required separator.
-      byte[] compoundColQual = createColQualifierPrefix(colPrefix, column);
-      list.addFilter(new QualifierFilter(CompareOp.EQUAL,
-          new BinaryPrefixComparator(
-              colPrefix.getColumnPrefixBytes(compoundColQual))));
-    }
-    return list;
-  }
-
   /**
    * Fetch columns from filter list containing exists and multivalue equality
    * filters. This is done to fetch only required columns from back-end and

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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 f8b5a65..3511a2f 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
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 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;
@@ -45,11 +46,10 @@ 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.EventColumnName;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongKeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
-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;
@@ -86,6 +86,17 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   private TypedBufferedMutator<FlowActivityTable> flowActivityTable;
   private TypedBufferedMutator<FlowRunTable> flowRunTable;
 
+  /**
+   * Used to convert strings key components to and from storage format.
+   */
+  private final KeyConverter<String> stringKeyConverter =
+      new StringKeyConverter();
+
+  /**
+   * Used to convert Long key components to and from storage format.
+   */
+  private final KeyConverter<Long> longKeyConverter = new LongKeyConverter();
+
   public HBaseTimelineWriterImpl() {
     super(HBaseTimelineWriterImpl.class.getName());
   }
@@ -138,12 +149,19 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
 
       // if the entity is the application, the destination is the application
       // table
-      boolean isApplication = TimelineStorageUtils.isApplicationEntity(te);
-      byte[] rowKey = isApplication ?
-          ApplicationRowKey.getRowKey(clusterId, userId, flowName, flowRunId,
-              appId) :
-          EntityRowKey.getRowKey(clusterId, userId, flowName, flowRunId, appId,
-              te.getType(), te.getId());
+      boolean isApplication = isApplicationEntity(te);
+      byte[] rowKey;
+      if (isApplication) {
+        ApplicationRowKey applicationRowKey =
+            new ApplicationRowKey(clusterId, userId, flowName, flowRunId,
+                appId);
+        rowKey = applicationRowKey.getRowKey();
+      } else {
+        EntityRowKey entityRowKey =
+            new EntityRowKey(clusterId, userId, flowName, flowRunId, appId,
+                te.getType(), te.getId());
+        rowKey = entityRowKey.getRowKey();
+      }
 
       storeInfo(rowKey, te, flowVersion, isApplication);
       storeEvents(rowKey, te.getEvents(), isApplication);
@@ -152,102 +170,101 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       storeRelations(rowKey, te, isApplication);
 
       if (isApplication) {
-        TimelineEvent event = TimelineStorageUtils.getApplicationEvent(te,
-            ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+        TimelineEvent event =
+            getApplicationEvent(te,
+                ApplicationMetricsConstants.CREATED_EVENT_TYPE);
+        FlowRunRowKey flowRunRowKey =
+            new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
         if (event != null) {
-          onApplicationCreated(clusterId, userId, flowName, flowVersion,
-              flowRunId, appId, te, event.getTimestamp());
+          AppToFlowRowKey appToFlowRowKey =
+              new AppToFlowRowKey(clusterId, appId);
+          onApplicationCreated(flowRunRowKey, appToFlowRowKey, appId, userId,
+              flowVersion, te, event.getTimestamp());
         }
         // if it's an application entity, store metrics
-        storeFlowMetricsAppRunning(clusterId, userId, flowName, flowRunId,
-            appId, te);
+        storeFlowMetricsAppRunning(flowRunRowKey, appId, te);
         // if application has finished, store it's finish time and write final
         // values of all metrics
-        event = TimelineStorageUtils.getApplicationEvent(te,
+        event = getApplicationEvent(te,
             ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
         if (event != null) {
-          onApplicationFinished(clusterId, userId, flowName, flowVersion,
-              flowRunId, appId, te, event.getTimestamp());
+          onApplicationFinished(flowRunRowKey, flowVersion, appId, te,
+              event.getTimestamp());
         }
       }
     }
     return putStatus;
   }
 
-  private void onApplicationCreated(String clusterId, String userId,
-      String flowName, String flowVersion, long flowRunId, String appId,
-      TimelineEntity te, long appCreatedTimeStamp) throws IOException {
+  private void onApplicationCreated(FlowRunRowKey flowRunRowKey,
+      AppToFlowRowKey appToFlowRowKey, String appId, String userId,
+      String flowVersion, TimelineEntity te, long appCreatedTimeStamp)
+      throws IOException {
+
+    String flowName = flowRunRowKey.getFlowName();
+    Long flowRunId = flowRunRowKey.getFlowRunId();
+
     // store in App to flow table
-    storeInAppToFlowTable(clusterId, userId, flowName, flowRunId, appId, te);
+    byte[] rowKey = appToFlowRowKey.getRowKey();
+    AppToFlowColumn.FLOW_ID.store(rowKey, appToFlowTable, null, flowName);
+    AppToFlowColumn.FLOW_RUN_ID.store(rowKey, appToFlowTable, null, flowRunId);
+    AppToFlowColumn.USER_ID.store(rowKey, appToFlowTable, null, userId);
+
     // 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, appCreatedTimeStamp);
-  }
+    storeAppCreatedInFlowRunTable(flowRunRowKey, appId, te);
 
-  /*
-   * updates the {@link FlowActivityTable} with the Application TimelineEntity
-   * information
-   */
-  private void storeInFlowActivityTable(String clusterId, String userId,
-      String flowName, String flowVersion, long flowRunId, String appId,
-      long activityTimeStamp) throws IOException {
-    byte[] rowKey = FlowActivityRowKey.getRowKey(clusterId, activityTimeStamp,
-        userId, flowName);
-    byte[] qualifier = LongKeyConverter.getInstance().encode(flowRunId);
-    FlowActivityColumnPrefix.RUN_ID.store(rowKey, flowActivityTable, qualifier,
-        null, flowVersion,
+    // store in flow activity table
+    byte[] flowActivityRowKeyBytes =
+        new FlowActivityRowKey(flowRunRowKey.getClusterId(),
+            appCreatedTimeStamp, flowRunRowKey.getUserId(), flowName)
+            .getRowKey();
+    byte[] qualifier = longKeyConverter.encode(flowRunRowKey.getFlowRunId());
+    FlowActivityColumnPrefix.RUN_ID.store(flowActivityRowKeyBytes,
+        flowActivityTable, qualifier, null, flowVersion,
         AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
   }
 
   /*
    * 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);
+  private void storeAppCreatedInFlowRunTable(FlowRunRowKey flowRunRowKey,
+      String appId, TimelineEntity te) throws IOException {
+    byte[] rowKey = flowRunRowKey.getRowKey();
     FlowRunColumn.MIN_START_TIME.store(rowKey, flowRunTable, null,
         te.getCreatedTime(),
         AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
   }
 
-  private void storeInAppToFlowTable(String clusterId, String userId,
-      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);
-  }
 
   /*
    * 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, long appFinishedTimeStamp) throws IOException {
+  private void onApplicationFinished(FlowRunRowKey flowRunRowKey,
+      String flowVersion, String appId, TimelineEntity te,
+      long appFinishedTimeStamp) throws IOException {
     // store in flow run table
-    storeAppFinishedInFlowRunTable(clusterId, userId, flowName, flowRunId,
-        appId, te, appFinishedTimeStamp);
+    storeAppFinishedInFlowRunTable(flowRunRowKey, appId, te,
+        appFinishedTimeStamp);
 
     // indicate in the flow activity table that the app has finished
-    storeInFlowActivityTable(clusterId, userId, flowName, flowVersion,
-        flowRunId, appId, appFinishedTimeStamp);
+    byte[] rowKey =
+        new FlowActivityRowKey(flowRunRowKey.getClusterId(),
+            appFinishedTimeStamp, flowRunRowKey.getUserId(),
+            flowRunRowKey.getFlowName()).getRowKey();
+    byte[] qualifier = longKeyConverter.encode(flowRunRowKey.getFlowRunId());
+    FlowActivityColumnPrefix.RUN_ID.store(rowKey, flowActivityTable, qualifier,
+        null, flowVersion,
+        AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId));
   }
 
   /*
    * Update the {@link FlowRunTable} with Application Finished information
    */
-  private void storeAppFinishedInFlowRunTable(String clusterId, String userId,
-      String flowName, long flowRunId, String appId, TimelineEntity te,
-      long appFinishedTimeStamp) throws IOException {
-    byte[] rowKey =
-        FlowRunRowKey.getRowKey(clusterId, userId, flowName, flowRunId);
+  private void storeAppFinishedInFlowRunTable(FlowRunRowKey flowRunRowKey,
+      String appId, TimelineEntity te, long appFinishedTimeStamp)
+      throws IOException {
+    byte[] rowKey = flowRunRowKey.getRowKey();
     Attribute attributeAppId =
         AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId);
     FlowRunColumn.MAX_END_TIME.store(rowKey, flowRunTable, null,
@@ -264,13 +281,11 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   /*
    * Updates the {@link FlowRunTable} with Application Metrics
    */
-  private void storeFlowMetricsAppRunning(String clusterId, String userId,
-      String flowName, long flowRunId, String appId, TimelineEntity te)
-      throws IOException {
+  private void storeFlowMetricsAppRunning(FlowRunRowKey flowRunRowKey,
+      String appId, TimelineEntity te) throws IOException {
     Set<TimelineMetric> metrics = te.getMetrics();
     if (metrics != null) {
-      byte[] rowKey = FlowRunRowKey.getRowKey(clusterId, userId, flowName,
-          flowRunId);
+      byte[] rowKey = flowRunRowKey.getRowKey();
       storeFlowMetrics(rowKey, metrics,
           AggregationCompactionDimension.APPLICATION_ID.getAttribute(appId),
           AggregationOperation.SUM.getAttribute());
@@ -280,8 +295,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
   private void storeFlowMetrics(byte[] rowKey, Set<TimelineMetric> metrics,
       Attribute... attributes) throws IOException {
     for (TimelineMetric metric : metrics) {
-      byte[] metricColumnQualifier =
-          StringKeyConverter.getInstance().encode(metric.getId());
+      byte[] metricColumnQualifier = stringKeyConverter.encode(metric.getId());
       Map<Long, Number> timeseries = metric.getValues();
       for (Map.Entry<Long, Number> timeseriesEntry : timeseries.entrySet()) {
         Long timestamp = timeseriesEntry.getKey();
@@ -320,8 +334,8 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       String compoundValue =
           Separator.VALUES.joinEncoded(connectedEntity.getValue());
       columnPrefix.store(rowKey, table,
-          StringKeyConverter.getInstance().encode(connectedEntity.getKey()),
-          null, compoundValue);
+          stringKeyConverter.encode(connectedEntity.getKey()), null,
+          compoundValue);
     }
   }
 
@@ -341,7 +355,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       if (info != null) {
         for (Map.Entry<String, Object> entry : info.entrySet()) {
           ApplicationColumnPrefix.INFO.store(rowKey, applicationTable,
-              StringKeyConverter.getInstance().encode(entry.getKey()), null,
+              stringKeyConverter.encode(entry.getKey()), null,
               entry.getValue());
         }
       }
@@ -355,7 +369,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       if (info != null) {
         for (Map.Entry<String, Object> entry : info.entrySet()) {
           EntityColumnPrefix.INFO.store(rowKey, entityTable,
-              StringKeyConverter.getInstance().encode(entry.getKey()), null,
+              stringKeyConverter.encode(entry.getKey()), null,
               entry.getValue());
         }
       }
@@ -371,8 +385,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       return;
     }
     for (Map.Entry<String, String> entry : config.entrySet()) {
-      byte[] configKey =
-          StringKeyConverter.getInstance().encode(entry.getKey());
+      byte[] configKey = stringKeyConverter.encode(entry.getKey());
       if (isApplication) {
         ApplicationColumnPrefix.CONFIG.store(rowKey, applicationTable,
             configKey, null, entry.getValue());
@@ -392,7 +405,7 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     if (metrics != null) {
       for (TimelineMetric metric : metrics) {
         byte[] metricColumnQualifier =
-            StringKeyConverter.getInstance().encode(metric.getId());
+            stringKeyConverter.encode(metric.getId());
         Map<Long, Number> timeseries = metric.getValues();
         for (Map.Entry<Long, Number> timeseriesEntry : timeseries.entrySet()) {
           Long timestamp = timeseriesEntry.getKey();
@@ -425,12 +438,11 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
                   "! Using the current timestamp");
               eventTimestamp = System.currentTimeMillis();
             }
-            EventColumnNameConverter converter =
-                EventColumnNameConverter.getInstance();
             Map<String, Object> eventInfo = event.getInfo();
             if ((eventInfo == null) || (eventInfo.size() == 0)) {
-              byte[] columnQualifierBytes = converter.encode(
-                  new EventColumnName(eventId, eventTimestamp, null));
+              byte[] columnQualifierBytes =
+                  new EventColumnName(eventId, eventTimestamp, null)
+                      .getColumnQualifier();
               if (isApplication) {
                 ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
                     columnQualifierBytes, null, Separator.EMPTY_BYTES);
@@ -441,9 +453,9 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
             } else {
               for (Map.Entry<String, Object> info : eventInfo.entrySet()) {
                 // eventId=infoKey
-                byte[] columnQualifierBytes = converter.encode(
-                    new EventColumnName(eventId, eventTimestamp,
-                        info.getKey()));
+                byte[] columnQualifierBytes =
+                    new EventColumnName(eventId, eventTimestamp, info.getKey())
+                        .getColumnQualifier();
                 if (isApplication) {
                   ApplicationColumnPrefix.EVENT.store(rowKey, applicationTable,
                       columnQualifierBytes, null, info.getValue());
@@ -459,12 +471,56 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
     }
   }
 
+  /**
+   * Checks if the input TimelineEntity object is an ApplicationEntity.
+   *
+   * @param te TimelineEntity object.
+   * @return true if input is an ApplicationEntity, false otherwise
+   */
+  static boolean isApplicationEntity(TimelineEntity te) {
+    return te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString());
+  }
+
+  /**
+   * @param te TimelineEntity object.
+   * @param eventId event with this id needs to be fetched
+   * @return TimelineEvent if TimelineEntity contains the desired event.
+   */
+  private static TimelineEvent getApplicationEvent(TimelineEntity te,
+      String eventId) {
+    if (isApplicationEntity(te)) {
+      for (TimelineEvent event : te.getEvents()) {
+        if (event.getId().equals(eventId)) {
+          return event;
+        }
+      }
+    }
+    return null;
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage
+   * .TimelineWriter#aggregate
+   * (org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity,
+   * org.apache
+   * .hadoop.yarn.server.timelineservice.storage.TimelineAggregationTrack)
+   */
   @Override
   public TimelineWriteResponse aggregate(TimelineEntity data,
       TimelineAggregationTrack track) throws IOException {
     return null;
   }
 
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.TimelineWriter#flush
+   * ()
+   */
   @Override
   public void flush() throws IOException {
     // flush all buffered mutators

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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 80fcf8c..dde3911 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
@@ -45,7 +45,7 @@ public enum ApplicationColumn implements Column<ApplicationTable> {
    * When the application was created.
    */
   CREATED_TIME(ApplicationColumnFamily.INFO, "created_time",
-      LongConverter.getInstance()),
+      new LongConverter()),
 
   /**
    * The version of the flow that this app belongs to.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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 0febc67..42488f4 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
@@ -67,8 +67,7 @@ public enum ApplicationColumnPrefix implements ColumnPrefix<ApplicationTable> {
   /**
    * Metrics are stored with the metric name as the column name.
    */
-  METRIC(ApplicationColumnFamily.METRICS, null,
-      LongConverter.getInstance());
+  METRIC(ApplicationColumnFamily.METRICS, null, new LongConverter());
 
   private final ColumnHelper<ApplicationTable> column;
   private final ColumnFamily<ApplicationTable> columnFamily;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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 e476b21..da62fdf 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
@@ -18,6 +18,12 @@
 
 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.AppIdKeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+
 /**
  * Represents a rowkey for the application table.
  */
@@ -27,6 +33,8 @@ public class ApplicationRowKey {
   private final String flowName;
   private final Long flowRunId;
   private final String appId;
+  private final KeyConverter<ApplicationRowKey> appRowKeyConverter =
+      new ApplicationRowKeyConverter();
 
   public ApplicationRowKey(String clusterId, String userId, String flowName,
       Long flowRunId, String appId) {
@@ -58,60 +66,141 @@ public class ApplicationRowKey {
   }
 
   /**
-   * Constructs a row key prefix for the application table as follows:
-   * {@code clusterId!userName!flowName!}.
-   *
-   * @param clusterId Cluster Id.
-   * @param userId User Id.
-   * @param flowName Flow Name.
-   * @return byte array with the row key prefix
-   */
-  public static byte[] getRowKeyPrefix(String clusterId, String userId,
-      String flowName) {
-    return ApplicationRowKeyConverter.getInstance().encode(
-        new ApplicationRowKey(clusterId, userId, flowName, null, null));
-  }
-
-  /**
-   * Constructs a row key prefix for the application table as follows:
-   * {@code clusterId!userName!flowName!flowRunId!}.
-   *
-   * @param clusterId Cluster Id.
-   * @param userId User Id.
-   * @param flowName Flow Name.
-   * @param flowRunId Run Id for the flow.
-   * @return byte array with the row key prefix
-   */
-  public static byte[] getRowKeyPrefix(String clusterId, String userId,
-      String flowName, Long flowRunId) {
-    return ApplicationRowKeyConverter.getInstance().encode(
-        new ApplicationRowKey(clusterId, userId, flowName, flowRunId, null));
-  }
-
-  /**
    * Constructs a row key for the application table as follows:
    * {@code clusterId!userName!flowName!flowRunId!AppId}.
    *
-   * @param clusterId Cluster Id.
-   * @param userId User Id.
-   * @param flowName Flow Name.
-   * @param flowRunId Run Id for the flow.
-   * @param appId App Id.
    * @return byte array with the row key
    */
-  public static byte[] getRowKey(String clusterId, String userId,
-      String flowName, Long flowRunId, String appId) {
-    return ApplicationRowKeyConverter.getInstance().encode(
-        new ApplicationRowKey(clusterId, userId, flowName, flowRunId, appId));
+  public byte[] getRowKey() {
+    return appRowKeyConverter.encode(this);
   }
 
   /**
    * Given the raw row key as bytes, returns the row key as an object.
    *
-   * @param rowKey Byte representation  of row key.
+   * @param rowKey Byte representation of row key.
    * @return An <cite>ApplicationRowKey</cite> object.
    */
   public static ApplicationRowKey parseRowKey(byte[] rowKey) {
-    return ApplicationRowKeyConverter.getInstance().decode(rowKey);
+    return new ApplicationRowKeyConverter().decode(rowKey);
   }
+
+  /**
+   * Encodes and decodes row key for application table. The row key is of the
+   * form: clusterId!userName!flowName!flowRunId!appId. flowRunId is a long,
+   * appId is encoded and decoded using {@link AppIdKeyConverter} and rest are
+   * strings.
+   * <p>
+   */
+  final private static class ApplicationRowKeyConverter implements
+      KeyConverter<ApplicationRowKey> {
+
+    private final KeyConverter<String> appIDKeyConverter =
+        new AppIdKeyConverter();
+
+    /**
+     * Intended for use in ApplicationRowKey only.
+     */
+    private ApplicationRowKeyConverter() {
+    }
+
+    /**
+     * Application row key is of the form
+     * clusterId!userName!flowName!flowRunId!appId with each segment separated
+     * by !. The sizes below indicate sizes of each one of these segements in
+     * sequence. clusterId, userName and flowName are strings. flowrunId is a
+     * long hence 8 bytes in size. app id is represented as 12 bytes with
+     * cluster timestamp part of appid takes 8 bytes(long) and seq id takes 4
+     * bytes(int). Strings are variable in size (i.e. end whenever separator is
+     * encountered). This is used while decoding and helps in determining where
+     * to split.
+     */
+    private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE,
+        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+        AppIdKeyConverter.getKeySize() };
+
+    /*
+     * (non-Javadoc)
+     *
+     * Encodes ApplicationRowKey object into a byte array with each
+     * component/field in ApplicationRowKey separated by Separator#QUALIFIERS.
+     * This leads to an application table row key of the form
+     * clusterId!userName!flowName!flowRunId!appId If flowRunId in passed
+     * ApplicationRowKey object is null (and the fields preceding it i.e.
+     * clusterId, userId and flowName are not null), this returns a row key
+     * prefix of the form clusterId!userName!flowName! and if appId in
+     * ApplicationRowKey is null (other 4 components all are not null), this
+     * returns a row key prefix of the form
+     * clusterId!userName!flowName!flowRunId! flowRunId is inverted while
+     * encoding as it helps maintain a descending order for row keys in the
+     * application table.
+     *
+     * @see
+     * org.apache.hadoop.yarn.server.timelineservice.storage.common
+     * .KeyConverter#encode(java.lang.Object)
+     */
+    @Override
+    public byte[] encode(ApplicationRowKey rowKey) {
+      byte[] cluster =
+          Separator.encode(rowKey.getClusterId(), Separator.SPACE,
+              Separator.TAB, Separator.QUALIFIERS);
+      byte[] user =
+          Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
+              Separator.QUALIFIERS);
+      byte[] flow =
+          Separator.encode(rowKey.getFlowName(), Separator.SPACE,
+              Separator.TAB, Separator.QUALIFIERS);
+      byte[] first = Separator.QUALIFIERS.join(cluster, user, flow);
+      // Note that flowRunId is a long, so we can't encode them all at the same
+      // time.
+      if (rowKey.getFlowRunId() == null) {
+        return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES);
+      }
+      byte[] second =
+          Bytes.toBytes(LongConverter.invertLong(
+              rowKey.getFlowRunId()));
+      if (rowKey.getAppId() == null || rowKey.getAppId().isEmpty()) {
+        return Separator.QUALIFIERS.join(first, second, Separator.EMPTY_BYTES);
+      }
+      byte[] third = appIDKeyConverter.encode(rowKey.getAppId());
+      return Separator.QUALIFIERS.join(first, second, third);
+    }
+
+    /*
+     * (non-Javadoc)
+     *
+     * Decodes an application row key of the form
+     * clusterId!userName!flowName!flowRunId!appId represented in byte format
+     * and converts it into an ApplicationRowKey object.flowRunId is inverted
+     * while decoding as it was inverted while encoding.
+     *
+     * @see
+     * org.apache.hadoop.yarn.server.timelineservice.storage.common
+     * .KeyConverter#decode(byte[])
+     */
+    @Override
+    public ApplicationRowKey decode(byte[] rowKey) {
+      byte[][] rowKeyComponents =
+          Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
+      if (rowKeyComponents.length != 5) {
+        throw new IllegalArgumentException("the row key is not valid for "
+            + "an application");
+      }
+      String clusterId =
+          Separator.decode(Bytes.toString(rowKeyComponents[0]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      String userId =
+          Separator.decode(Bytes.toString(rowKeyComponents[1]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      String flowName =
+          Separator.decode(Bytes.toString(rowKeyComponents[2]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      Long flowRunId =
+          LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3]));
+      String appId = appIDKeyConverter.decode(rowKeyComponents[4]);
+      return new ApplicationRowKey(clusterId, userId, flowName, flowRunId,
+          appId);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyConverter.java
----------------------------------------------------------------------
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/ApplicationRowKeyConverter.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/ApplicationRowKeyConverter.java
deleted file mode 100644
index 3b054a5..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyConverter.java
+++ /dev/null
@@ -1,130 +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.application;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
-
-/**
- * Encodes and decodes row key for application table.
- * The row key is of the form : clusterId!userName!flowName!flowRunId!appId.
- * flowRunId is a long, appId is encoded and decoded using
- * {@link AppIdKeyConverter} and rest are strings.
- */
-public final class ApplicationRowKeyConverter implements
-    KeyConverter<ApplicationRowKey> {
-  private static final ApplicationRowKeyConverter INSTANCE =
-      new ApplicationRowKeyConverter();
-
-  public static ApplicationRowKeyConverter getInstance() {
-    return INSTANCE;
-  }
-
-  private ApplicationRowKeyConverter() {
-  }
-
-  // Application row key is of the form
-  // clusterId!userName!flowName!flowRunId!appId with each segment separated
-  // by !. The sizes below indicate sizes of each one of these segements in
-  // sequence. clusterId, userName and flowName are strings. flowrunId is a long
-  // hence 8 bytes in size. app id is represented as 12 bytes with cluster
-  // timestamp part of appid being 8 bytes(long) and seq id being 4 bytes(int).
-  // Strings are variable in size (i.e. end whenever separator is encountered).
-  // This is used while decoding and helps in determining where to split.
-  private static final int[] SEGMENT_SIZES = {
-      Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
-      Bytes.SIZEOF_LONG, AppIdKeyConverter.getKeySize() };
-
-  /*
-   * (non-Javadoc)
-   *
-   * Encodes ApplicationRowKey object into a byte array with each
-   * component/field in ApplicationRowKey separated by Separator#QUALIFIERS.
-   * This leads to an application table row key of the form
-   * clusterId!userName!flowName!flowRunId!appId
-   * If flowRunId in passed ApplicationRowKey object is null (and the fields
-   * preceding it i.e. clusterId, userId and flowName are not null), this
-   * returns a row key prefix of the form clusterId!userName!flowName! and if
-   * appId in ApplicationRowKey is null (other 4 components are not null), this
-   * returns a row key prefix of the form clusterId!userName!flowName!flowRunId!
-   * flowRunId is inverted while encoding as it helps maintain a descending
-   * order for row keys in application table.
-   *
-   * @see
-   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
-   * #encode(java.lang.Object)
-   */
-  @Override
-  public byte[] encode(ApplicationRowKey rowKey) {
-    byte[] cluster = Separator.encode(rowKey.getClusterId(),
-        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
-    byte[] user = Separator.encode(rowKey.getUserId(),
-        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
-    byte[] flow = Separator.encode(rowKey.getFlowName(),
-        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
-    byte[] first = Separator.QUALIFIERS.join(cluster, user, flow);
-    // Note that flowRunId is a long, so we can't encode them all at the same
-    // time.
-    if (rowKey.getFlowRunId() == null) {
-      return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES);
-    }
-    byte[] second = Bytes.toBytes(
-        TimelineStorageUtils.invertLong(rowKey.getFlowRunId()));
-    if (rowKey.getAppId() == null || rowKey.getAppId().isEmpty()) {
-      return Separator.QUALIFIERS.join(first, second, Separator.EMPTY_BYTES);
-    }
-    byte[] third = AppIdKeyConverter.getInstance().encode(rowKey.getAppId());
-    return Separator.QUALIFIERS.join(first, second, third);
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * Decodes an application row key of the form
-   * clusterId!userName!flowName!flowRunId!appId represented in byte format and
-   * converts it into an ApplicationRowKey object.flowRunId is inverted while
-   * decoding as it was inverted while encoding.
-   *
-   * @see
-   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
-   * #decode(byte[])
-   */
-  @Override
-  public ApplicationRowKey decode(byte[] rowKey) {
-    byte[][] rowKeyComponents =
-        Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
-    if (rowKeyComponents.length != 5) {
-      throw new IllegalArgumentException("the row key is not valid for " +
-          "an application");
-    }
-    String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    String userId = Separator.decode(Bytes.toString(rowKeyComponents[1]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    String flowName = Separator.decode(Bytes.toString(rowKeyComponents[2]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    Long flowRunId =
-        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
-    String appId = AppIdKeyConverter.getInstance().decode(rowKeyComponents[4]);
-    return new ApplicationRowKey(clusterId, userId, flowName, flowRunId, appId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/application/ApplicationRowKeyPrefix.java
----------------------------------------------------------------------
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/ApplicationRowKeyPrefix.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/ApplicationRowKeyPrefix.java
new file mode 100644
index 0000000..f61b0e9
--- /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/ApplicationRowKeyPrefix.java
@@ -0,0 +1,69 @@
+/**
+ * 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.yarn.server.timelineservice.storage.common.RowKeyPrefix;
+
+/**
+ * Represents a partial rowkey (without flowName or without flowName and
+ * flowRunId) for the application table.
+ */
+public class ApplicationRowKeyPrefix extends ApplicationRowKey implements
+    RowKeyPrefix<ApplicationRowKey> {
+
+  /**
+   * Creates a prefix which generates the following rowKeyPrefixes for the
+   * application table: {@code clusterId!userName!flowName!}.
+   *
+   * @param clusterId the cluster on which applications ran
+   * @param userId the user that ran applications
+   * @param flowName the name of the flow that was run by the user on the
+   *          cluster
+   */
+  public ApplicationRowKeyPrefix(String clusterId, String userId,
+      String flowName) {
+    super(clusterId, userId, flowName, null, null);
+  }
+
+  /**
+   * Creates a prefix which generates the following rowKeyPrefixes for the
+   * application table: {@code clusterId!userName!flowName!flowRunId!}.
+   *
+   * @param clusterId identifying the cluster
+   * @param userId identifying the user
+   * @param flowName identifying the flow
+   * @param flowRunId identifying the instance of this flow
+   */
+  public ApplicationRowKeyPrefix(String clusterId, String userId,
+      String flowName, Long flowRunId) {
+    super(clusterId, userId, flowName, flowRunId, null);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.application.
+   * RowKeyPrefix#getRowKeyPrefix()
+   */
+  @Override
+  public byte[] getRowKeyPrefix() {
+    return super.getRowKey();
+  }
+
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKeyConverter.java
----------------------------------------------------------------------
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/AppToFlowRowKeyConverter.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/AppToFlowRowKeyConverter.java
deleted file mode 100644
index 0f0b879d..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowRowKeyConverter.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-
-/**
- * Encodes and decodes row key for app_flow table.
- * The row key is of the form : clusterId!appId.
- * clusterId is a string and appId is encoded/decoded using
- * {@link AppIdKeyConverter}.
- */
-public final class AppToFlowRowKeyConverter
-    implements KeyConverter<AppToFlowRowKey> {
-  private static final AppToFlowRowKeyConverter INSTANCE =
-      new AppToFlowRowKeyConverter();
-
-  public static AppToFlowRowKeyConverter getInstance() {
-    return INSTANCE;
-  }
-
-  private AppToFlowRowKeyConverter() {
-  }
-
-  // App to flow row key is of the form clusterId!appId with the 2 segments
-  // separated by !. The sizes below indicate sizes of both of these segments
-  // in sequence. clusterId is a string. appId is represented as 12 bytes with
-  // cluster Timestamp part of appid being 8 bytes(long) and seq id being 4
-  // bytes(int).
-  // Strings are variable in size (i.e. end whenever separator is encountered).
-  // This is used while decoding and helps in determining where to split.
-  private static final int[] SEGMENT_SIZES = {
-      Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG + Bytes.SIZEOF_INT };
-
-  /*
-   * (non-Javadoc)
-   *
-   * Encodes AppToFlowRowKey object into a byte array with each component/field
-   * in AppToFlowRowKey separated by Separator#QUALIFIERS. This leads to an
-   * app to flow table row key of the form clusterId!appId
-   *
-   * @see
-   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
-   * #encode(java.lang.Object)
-   */
-  @Override
-  public byte[] encode(AppToFlowRowKey rowKey) {
-    byte[] first = Separator.encode(rowKey.getClusterId(),
-        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
-    byte[] second = AppIdKeyConverter.getInstance().encode(rowKey.getAppId());
-    return Separator.QUALIFIERS.join(first, second);
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * Decodes an app to flow row key of the form clusterId!appId represented in
-   * byte format and converts it into an AppToFlowRowKey object.
-   *
-   * @see
-   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
-   * #decode(byte[])
-   */
-  @Override
-  public AppToFlowRowKey decode(byte[] rowKey) {
-    byte[][] rowKeyComponents =
-        Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
-    if (rowKeyComponents.length != 2) {
-      throw new IllegalArgumentException("the row key is not valid for " +
-          "the app-to-flow table");
-    }
-    String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    String appId = AppIdKeyConverter.getInstance().decode(rowKeyComponents[1]);
-    return new AppToFlowRowKey(clusterId, appId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
----------------------------------------------------------------------
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/AppIdKeyConverter.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/AppIdKeyConverter.java
index a173b0f..f5f7aa6 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/AppIdKeyConverter.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/AppIdKeyConverter.java
@@ -28,13 +28,8 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
  * (long - 8 bytes) followed by sequence id section of app id (int - 4 bytes).
  */
 public final class AppIdKeyConverter implements KeyConverter<String> {
-  private static final AppIdKeyConverter INSTANCE = new AppIdKeyConverter();
 
-  public static AppIdKeyConverter getInstance() {
-    return INSTANCE;
-  }
-
-  private AppIdKeyConverter() {
+  public AppIdKeyConverter() {
   }
 
   /*
@@ -58,7 +53,7 @@ public final class AppIdKeyConverter implements KeyConverter<String> {
     ApplicationId appId = ConverterUtils.toApplicationId(appIdStr);
     byte[] appIdBytes = new byte[getKeySize()];
     byte[] clusterTs = Bytes.toBytes(
-        TimelineStorageUtils.invertLong(appId.getClusterTimestamp()));
+        LongConverter.invertLong(appId.getClusterTimestamp()));
     System.arraycopy(clusterTs, 0, appIdBytes, 0, Bytes.SIZEOF_LONG);
     byte[] seqId = Bytes.toBytes(TimelineStorageUtils.invertInt(appId.getId()));
     System.arraycopy(seqId, 0, appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT);
@@ -83,7 +78,7 @@ public final class AppIdKeyConverter implements KeyConverter<String> {
     if (appIdBytes.length != getKeySize()) {
       throw new IllegalArgumentException("Invalid app id in byte format");
     }
-    long clusterTs = TimelineStorageUtils.invertLong(
+    long clusterTs = LongConverter.invertLong(
         Bytes.toLong(appIdBytes, 0, Bytes.SIZEOF_LONG));
     int seqId = TimelineStorageUtils.invertInt(
         Bytes.toInt(appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnName.java
----------------------------------------------------------------------
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/EventColumnName.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/EventColumnName.java
index 6018f86..8445575 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/EventColumnName.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/EventColumnName.java
@@ -26,6 +26,8 @@ public class EventColumnName {
   private final String id;
   private final Long timestamp;
   private final String infoKey;
+  private final KeyConverter<EventColumnName> eventColumnNameConverter =
+      new EventColumnNameConverter();
 
   public EventColumnName(String id, Long timestamp, String infoKey) {
     this.id = id;
@@ -45,4 +47,17 @@ public class EventColumnName {
     return infoKey;
   }
 
+  /**
+   * @return a byte array with each components/fields separated by
+   *         Separator#VALUES. This leads to an event column name of the form
+   *         eventId=timestamp=infokey. If both timestamp and infokey are null,
+   *         then a qualifier of the form eventId=timestamp= is returned. If
+   *         only infokey is null, then a qualifier of the form eventId= is
+   *         returned. These prefix forms are useful for queries that intend to
+   *         retrieve more than one specific column name.
+   */
+  public byte[] getColumnQualifier() {
+    return eventColumnNameConverter.encode(this);
+  }
+
 }


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


[42/50] [abbrv] hadoop git commit: YARN-5210 NPE in Distributed Shell while publishing DS_CONTAINER_START event (Varun Saxena via Vrushali C)

Posted by vr...@apache.org.
YARN-5210 NPE in Distributed Shell while publishing DS_CONTAINER_START event (Varun Saxena via Vrushali C)


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

Branch: refs/heads/YARN-2928
Commit: a005fb8c0db219e1f7941bccb0d4845e9f348c85
Parents: 2cab3fc
Author: Vrushali Channapattan <vr...@apache.org>
Authored: Thu Jun 9 11:36:52 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:14 2016 -0700

----------------------------------------------------------------------
 .../distributedshell/ApplicationMaster.java     | 23 ++++++++++----------
 1 file changed, 12 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a005fb8c/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 08d5364..fb2ae6f 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
@@ -978,16 +978,13 @@ public class ApplicationMaster {
       }
       Container container = containers.get(containerId);
       if (container != null) {
-        applicationMaster.nmClientAsync.getContainerStatusAsync(containerId, container.getNodeId());
+        applicationMaster.nmClientAsync.getContainerStatusAsync(
+            containerId, container.getNodeId());
       }
       if(applicationMaster.timelineClient != null) {
-        applicationMaster.publishContainerStartEvent(
-            applicationMaster.timelineClient, container,
-            applicationMaster.domainId, applicationMaster.appSubmitterUgi);
-
         if (applicationMaster.timelineServiceV2) {
-            applicationMaster.publishContainerStartEventOnTimelineServiceV2(
-                container);
+          applicationMaster.publishContainerStartEventOnTimelineServiceV2(
+              container);
         } else {
           applicationMaster.publishContainerStartEvent(
             applicationMaster.timelineClient, container,
@@ -1354,12 +1351,13 @@ public class ApplicationMaster {
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
     entity.setId(container.getId().toString());
     entity.setType(DSEntity.DS_CONTAINER.toString());
-    //entity.setDomainId(domainId);
+    long ts = System.currentTimeMillis();
+    entity.setCreatedTime(ts);
     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());
+    event.setTimestamp(ts);
     event.setId(DSEvent.DS_CONTAINER_START.toString());
     event.addInfo("Node", container.getNodeId().toString());
     event.addInfo("Resources", container.getResource().toString());
@@ -1417,12 +1415,15 @@ public class ApplicationMaster {
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
     entity.setId(appAttemptID.toString());
     entity.setType(DSEntity.DS_APP_ATTEMPT.toString());
-    //entity.setDomainId(domainId);
+    long ts = System.currentTimeMillis();
+    if (appEvent == DSEvent.DS_APP_ATTEMPT_START) {
+      entity.setCreatedTime(ts);
+    }
     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());
-    event.setTimestamp(System.currentTimeMillis());
+    event.setTimestamp(ts);
     entity.addEvent(event);
 
     try {


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


[07/50] [abbrv] hadoop git commit: YARN-4986. Add a check in the coprocessor for table to operated on (Vrushali C via sjlee)

Posted by vr...@apache.org.
YARN-4986. Add a check in the coprocessor for table to operated on (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/4caa1461
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4caa1461
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4caa1461

Branch: refs/heads/YARN-2928
Commit: 4caa1461bf526938c14f7b305595112f02340d26
Parents: 78ffdf0
Author: Sangjin Lee <sj...@apache.org>
Authored: Fri Apr 29 17:13:32 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:06 2016 -0700

----------------------------------------------------------------------
 .../storage/common/TimelineStorageUtils.java    | 20 +++++++
 .../storage/entity/EntityTable.java             |  2 +-
 .../storage/flow/FlowRunCoprocessor.java        | 39 +++++++++++--
 .../storage/flow/FlowScanner.java               | 13 +++--
 .../storage/flow/TestHBaseStorageFlowRun.java   | 61 ++++++++++++++++++++
 .../flow/TestHBaseStorageFlowRunCompaction.java | 36 ++++++++++++
 6 files changed, 160 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4caa1461/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 2d85bab..18f975a 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
@@ -32,8 +32,10 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
 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.client.Result;
@@ -56,6 +58,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Fiel
 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.server.timelineservice.storage.flow.FlowRunTable;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 /**
@@ -887,4 +890,21 @@ public final class TimelineStorageUtils {
     Set<TimelineEvent> eventsSet = new HashSet<>(eventsMap.values());
     entity.addEvents(eventsSet);
   }
+
+  public static boolean isFlowRunTable(HRegionInfo hRegionInfo,
+      Configuration conf) {
+    String regionTableName = hRegionInfo.getTable().getNameAsString();
+    String flowRunTableName = conf.get(FlowRunTable.TABLE_NAME_CONF_NAME,
+        FlowRunTable.DEFAULT_TABLE_NAME);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("regionTableName=" + regionTableName);
+    }
+    if (flowRunTableName.equalsIgnoreCase(regionTableName)) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(" table is the flow run table!! " + flowRunTableName);
+      }
+      return true;
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4caa1461/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 3e3e3ab..b194f07 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
@@ -84,7 +84,7 @@ public class EntityTable extends BaseTable<EntityTable> {
       + ".table.metrics.ttl";
 
   /** default value for entity table name. */
-  private static final String DEFAULT_TABLE_NAME = "timelineservice.entity";
+  public static final String DEFAULT_TABLE_NAME = "timelineservice.entity";
 
   /** default TTL is 30 days for metrics timeseries. */
   private static final int DEFAULT_METRICS_TTL = 2592000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4caa1461/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 450640a..8ea51a1 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
@@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGen
 public class FlowRunCoprocessor extends BaseRegionObserver {
 
   private static final Log LOG = LogFactory.getLog(FlowRunCoprocessor.class);
+  private boolean isFlowRunRegion = false;
 
   private HRegion region;
   /**
@@ -70,9 +71,15 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
     if (e instanceof RegionCoprocessorEnvironment) {
       RegionCoprocessorEnvironment env = (RegionCoprocessorEnvironment) e;
       this.region = env.getRegion();
+      isFlowRunRegion = TimelineStorageUtils.isFlowRunTable(
+          region.getRegionInfo(), env.getConfiguration());
     }
   }
 
+  public boolean isFlowRunRegion() {
+    return isFlowRunRegion;
+  }
+
   /*
    * (non-Javadoc)
    *
@@ -93,6 +100,9 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
       WALEdit edit, Durability durability) throws IOException {
     Map<String, byte[]> attributes = put.getAttributesMap();
 
+    if (!isFlowRunRegion) {
+      return;
+    }
     // Assumption is that all the cells in a put are the same operation.
     List<Tag> tags = new ArrayList<>();
     if ((attributes != null) && (attributes.size() > 0)) {
@@ -160,6 +170,10 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
   @Override
   public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e,
       Get get, List<Cell> results) throws IOException {
+    if (!isFlowRunRegion) {
+      return;
+    }
+
     Scan scan = new Scan(get);
     scan.setMaxVersions();
     RegionScanner scanner = null;
@@ -190,11 +204,14 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
   @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;
+      RegionScanner scanner) throws IOException {
+
+    if (isFlowRunRegion) {
+      // set max versions for scan to see all
+      // versions to aggregate for metrics
+      scan.setMaxVersions();
+    }
+    return scanner;
   }
 
   /*
@@ -213,6 +230,9 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
   public RegionScanner postScannerOpen(
       ObserverContext<RegionCoprocessorEnvironment> e, Scan scan,
       RegionScanner scanner) throws IOException {
+    if (!isFlowRunRegion) {
+      return scanner;
+    }
     return new FlowScanner(e.getEnvironment(), scan.getBatch(),
         scanner, FlowScannerOperation.READ);
   }
@@ -221,6 +241,9 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
   public InternalScanner preFlush(
       ObserverContext<RegionCoprocessorEnvironment> c, Store store,
       InternalScanner scanner) throws IOException {
+    if (!isFlowRunRegion) {
+      return scanner;
+    }
     if (LOG.isDebugEnabled()) {
       if (store != null) {
         LOG.debug("preFlush store = " + store.getColumnFamilyName()
@@ -241,6 +264,9 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
   @Override
   public void postFlush(ObserverContext<RegionCoprocessorEnvironment> c,
       Store store, StoreFile resultFile) {
+    if (!isFlowRunRegion) {
+      return;
+    }
     if (LOG.isDebugEnabled()) {
       if (store != null) {
         LOG.debug("postFlush store = " + store.getColumnFamilyName()
@@ -262,6 +288,9 @@ public class FlowRunCoprocessor extends BaseRegionObserver {
       InternalScanner scanner, ScanType scanType, CompactionRequest request)
       throws IOException {
 
+    if (!isFlowRunRegion) {
+      return scanner;
+    }
     FlowScannerOperation requestOp = FlowScannerOperation.MINOR_COMPACTION;
     if (request != null) {
       requestOp = (request.isMajor() ? FlowScannerOperation.MAJOR_COMPACTION

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4caa1461/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 0ace529..398d7b4 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
@@ -210,7 +210,7 @@ class FlowScanner implements RegionScanner, Closeable {
       if (comp.compare(currentColumnQualifier, newColumnQualifier) != 0) {
         if (converter != null && isNumericConverter(converter)) {
           addedCnt += emitCells(cells, currentColumnCells, currentAggOp,
-              (NumericValueConverter)converter, currentTimestamp);
+              converter, currentTimestamp);
         }
         resetState(currentColumnCells, alreadySeenAggDim);
         currentColumnQualifier = newColumnQualifier;
@@ -219,6 +219,7 @@ class FlowScanner implements RegionScanner, Closeable {
       }
       // No operation needs to be performed on non numeric converters.
       if (!isNumericConverter(converter)) {
+        currentColumnCells.add(cell);
         nextCell(cellLimit);
         continue;
       }
@@ -228,7 +229,7 @@ class FlowScanner implements RegionScanner, Closeable {
     }
     if (!currentColumnCells.isEmpty()) {
       addedCnt += emitCells(cells, currentColumnCells, currentAggOp,
-          (NumericValueConverter)converter, currentTimestamp);
+          converter, currentTimestamp);
       if (LOG.isDebugEnabled()) {
         if (addedCnt > 0) {
           LOG.debug("emitted cells. " + addedCnt + " for " + this.action
@@ -345,7 +346,7 @@ class FlowScanner implements RegionScanner, Closeable {
    * parameter.
    */
   private int emitCells(List<Cell> cells, SortedSet<Cell> currentColumnCells,
-      AggregationOperation currentAggOp, NumericValueConverter converter,
+      AggregationOperation currentAggOp, ValueConverter converter,
       long currentTimestamp) throws IOException {
     if ((currentColumnCells == null) || (currentColumnCells.size() == 0)) {
       return 0;
@@ -372,12 +373,14 @@ class FlowScanner implements RegionScanner, Closeable {
         cells.addAll(currentColumnCells);
         return currentColumnCells.size();
       case READ:
-        Cell sumCell = processSummation(currentColumnCells, converter);
+        Cell sumCell = processSummation(currentColumnCells,
+            (NumericValueConverter) converter);
         cells.add(sumCell);
         return 1;
       case MAJOR_COMPACTION:
         List<Cell> finalCells = processSummationMajorCompaction(
-            currentColumnCells, converter, currentTimestamp);
+            currentColumnCells, (NumericValueConverter) converter,
+            currentTimestamp);
         cells.addAll(finalCells);
         return finalCells.size();
       default:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4caa1461/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 a724db2..801d43c 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
@@ -19,18 +19,21 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage.flow;
 
 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 static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
 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;
@@ -38,6 +41,8 @@ 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.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 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;
@@ -57,6 +62,8 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriter
 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.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityTable;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -84,6 +91,60 @@ public class TestHBaseStorageFlowRun {
     TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
   }
 
+  @Test
+  public void checkCoProcessorOff() throws IOException, InterruptedException {
+    Configuration hbaseConf = util.getConfiguration();
+    TableName table = TableName.valueOf(hbaseConf.get(
+        FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+    Connection conn = null;
+    conn = ConnectionFactory.createConnection(hbaseConf);
+    Admin admin = conn.getAdmin();
+    if (admin == null) {
+      throw new IOException("Can't check tables since admin is null");
+    }
+    if (admin.tableExists(table)) {
+      // check the regions.
+      // check in flow run table
+      util.waitUntilAllRegionsAssigned(table);
+      HRegionServer server = util.getRSForFirstRegionInTable(table);
+      List<HRegion> regions = server.getOnlineRegions(table);
+      for (HRegion region : regions) {
+        assertTrue(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
+            hbaseConf));
+      }
+    }
+
+    table = TableName.valueOf(hbaseConf.get(
+        FlowActivityTable.TABLE_NAME_CONF_NAME,
+        FlowActivityTable.DEFAULT_TABLE_NAME));
+    if (admin.tableExists(table)) {
+      // check the regions.
+      // check in flow activity table
+      util.waitUntilAllRegionsAssigned(table);
+      HRegionServer server = util.getRSForFirstRegionInTable(table);
+      List<HRegion> regions = server.getOnlineRegions(table);
+      for (HRegion region : regions) {
+        assertFalse(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
+            hbaseConf));
+      }
+    }
+
+    table = TableName.valueOf(hbaseConf.get(
+        EntityTable.TABLE_NAME_CONF_NAME,
+        EntityTable.DEFAULT_TABLE_NAME));
+    if (admin.tableExists(table)) {
+      // check the regions.
+      // check in entity run table
+      util.waitUntilAllRegionsAssigned(table);
+      HRegionServer server = util.getRSForFirstRegionInTable(table);
+      List<HRegion> regions = server.getOnlineRegions(table);
+      for (HRegion region : regions) {
+        assertFalse(TimelineStorageUtils.isFlowRunTable(region.getRegionInfo(),
+            hbaseConf));
+      }
+    }
+  }
+
   /**
    * Writes 4 timeline entities belonging to one flow run through the
    * {@link HBaseTimelineWriterImpl}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4caa1461/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
index 2738e6a..e7e7ba4 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/TestHBaseStorageFlowRunCompaction.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/TestHBaseStorageFlowRunCompaction.java
@@ -39,6 +39,8 @@ import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.KeyValue;
 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.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -87,6 +89,40 @@ public class TestHBaseStorageFlowRunCompaction {
     TimelineSchemaCreator.createAllTables(util.getConfiguration(), false);
   }
 
+  /** writes non numeric data into flow run table
+   * reads it back
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testWriteNonNumericData() throws Exception {
+    String rowKey = "nonNumericRowKey";
+    String column = "nonNumericColumnName";
+    String value = "nonNumericValue";
+    byte[] rowKeyBytes = Bytes.toBytes(rowKey);
+    byte[] columnNameBytes = Bytes.toBytes(column);
+    byte[] valueBytes = Bytes.toBytes(value);
+    Put p = new Put(rowKeyBytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnNameBytes,
+        valueBytes);
+    Configuration hbaseConf = util.getConfiguration();
+    TableName table = TableName.valueOf(hbaseConf.get(
+        FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+    Connection conn = null;
+    conn = ConnectionFactory.createConnection(hbaseConf);
+    Table flowRunTable = conn.getTable(table);
+    flowRunTable.put(p);
+
+    Get g = new Get(rowKeyBytes);
+    Result r = flowRunTable.get(g);
+    assertNotNull(r);
+    assertTrue(r.size() >= 1);
+    Cell actualValue = r.getColumnLatestCell(
+        FlowRunColumnFamily.INFO.getBytes(), columnNameBytes);
+    assertNotNull(CellUtil.cloneValue(actualValue));
+    assertEquals(Bytes.toString(CellUtil.cloneValue(actualValue)), value);
+  }
+
   @Test
   public void testWriteFlowRunCompaction() throws Exception {
     String cluster = "kompaction_cluster1";


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


[19/50] [abbrv] hadoop git commit: YARN-5045. hbase unit tests fail due to dependency issues. (Sangjin Lee via varunsaxena)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/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
deleted file mode 100644
index ca80ed5..0000000
--- 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
+++ /dev/null
@@ -1,2008 +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.text.DateFormat;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-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.server.timelineservice.storage.common.TimelineStorageUtils;
-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.google.common.collect.ImmutableMap;
-import com.google.common.collect.Sets;
-import com.sun.jersey.api.client.Client;
-import com.sun.jersey.api.client.ClientResponse;
-import com.sun.jersey.api.client.ClientResponse.Status;
-import com.sun.jersey.api.client.GenericType;
-import com.sun.jersey.api.client.config.ClientConfig;
-import com.sun.jersey.api.client.config.DefaultClientConfig;
-import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
-import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
-
-public class TestTimelineReaderWebServicesHBaseStorage {
-  private int serverPort;
-  private TimelineReaderServer server;
-  private static HBaseTestingUtility util;
-  private static long ts = System.currentTimeMillis();
-  private static long dayTs =
-      TimelineStorageUtils.getTopOfTheDayTimestamp(ts);
-
-  @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 =
-        ImmutableMap.of(ts - 100000, (Number)2, ts - 80000, 40);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    m1 = new TimelineMetric();
-    m1.setId("MAP1_SLOT_MILLIS");
-    metricValues = ImmutableMap.of(ts - 100000, (Number)2, ts - 80000, 40);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    m1 = new TimelineMetric();
-    m1.setId("HDFS_BYTES_READ");
-    metricValues = ImmutableMap.of(ts - 100000, (Number)31, 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);
-    event.setTimestamp(cTime);
-    String expKey = "foo_event";
-    Object expVal = "test";
-    event.addInfo(expKey, expVal);
-    entity.addEvent(event);
-    TimelineEvent event11 = new TimelineEvent();
-    event11.setId(ApplicationMetricsConstants.FINISHED_EVENT_TYPE);
-    Long expTs = 1425019501000L;
-    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(cTime);
-    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 = 1425016501037L;
-    entity3.setCreatedTime(cTime);
-    TimelineEvent event2 = new TimelineEvent();
-    event2.setId(ApplicationMetricsConstants.CREATED_EVENT_TYPE);
-    event2.setTimestamp(cTime);
-    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(cTime);
-    event4.addInfo("foo_event", "test");
-    entity4.addEvent(event4);
-    metrics.clear();
-    m2 = new TimelineMetric();
-    m2.setId("MAP_SLOT_MILLIS");
-    metricValues = ImmutableMap.of(ts - 100000, (Number)5L, ts - 80000, 101L);
-    m2.setType(Type.TIME_SERIES);
-    m2.setValues(metricValues);
-    metrics.add(m2);
-    entity4.addMetrics(metrics);
-    te4.addEntity(entity4);
-
-    TimelineEntities te5 = new TimelineEntities();
-    TimelineEntity entity5 = new TimelineEntity();
-    entity5.setId("entity1");
-    entity5.setType("type1");
-    entity5.setCreatedTime(1425016501034L);
-    // add some config entries
-    entity5.addConfigs(ImmutableMap.of("config_param1", "value1",
-        "config_param2", "value2", "cfg_param1", "value3"));
-    entity5.addInfo(ImmutableMap.of("info1", (Object)"cluster1",
-        "info2", 2.0, "info3", 35000, "info4", 36000));
-    metrics = new HashSet<>();
-    m1 = new TimelineMetric();
-    m1.setId("MAP_SLOT_MILLIS");
-    metricValues = ImmutableMap.of(ts - 100000, (Number)2, ts - 80000, 40);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    m1 = new TimelineMetric();
-    m1.setId("HDFS_BYTES_READ");
-    metricValues = ImmutableMap.of(ts - 100000, (Number)31, ts - 80000, 57);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    entity5.addMetrics(metrics);
-    TimelineEvent event51 = new TimelineEvent();
-    event51.setId("event1");
-    event51.setTimestamp(cTime);
-    entity5.addEvent(event51);
-    TimelineEvent event52 = new TimelineEvent();
-    event52.setId("event2");
-    event52.setTimestamp(cTime);
-    entity5.addEvent(event52);
-    TimelineEvent event53 = new TimelineEvent();
-    event53.setId("event3");
-    event53.setTimestamp(cTime);
-    entity5.addEvent(event53);
-    TimelineEvent event54 = new TimelineEvent();
-    event54.setId("event4");
-    event54.setTimestamp(cTime);
-    entity5.addEvent(event54);
-    Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
-    isRelatedTo1.put("type2",
-        Sets.newHashSet("entity21","entity22","entity23","entity24"));
-    isRelatedTo1.put("type4", Sets.newHashSet("entity41","entity42"));
-    isRelatedTo1.put("type1", Sets.newHashSet("entity14","entity15"));
-    isRelatedTo1.put("type3",
-        Sets.newHashSet("entity31", "entity35", "entity32", "entity33"));
-    entity5.addIsRelatedToEntities(isRelatedTo1);
-    Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
-    relatesTo1.put("type2",
-        Sets.newHashSet("entity21","entity22","entity23","entity24"));
-    relatesTo1.put("type4", Sets.newHashSet("entity41","entity42"));
-    relatesTo1.put("type1", Sets.newHashSet("entity14","entity15"));
-    relatesTo1.put("type3",
-        Sets.newHashSet("entity31", "entity35", "entity32", "entity33"));
-    entity5.addRelatesToEntities(relatesTo1);
-    te5.addEntity(entity5);
-
-    TimelineEntity entity6 = new TimelineEntity();
-    entity6.setId("entity2");
-    entity6.setType("type1");
-    entity6.setCreatedTime(1425016501034L);
-    entity6.addConfigs(ImmutableMap.of("cfg_param3", "value1",
-        "configuration_param2", "value2", "config_param1", "value3"));
-    entity6.addInfo(ImmutableMap.of("info1", (Object)"cluster2",
-        "info2", 2.0, "info4", 35000));
-    metrics = new HashSet<>();
-    m1 = new TimelineMetric();
-    m1.setId("MAP1_SLOT_MILLIS");
-    metricValues = ImmutableMap.of(ts - 100000, (Number)12, ts - 80000, 140);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    m1 = new TimelineMetric();
-    m1.setId("HDFS_BYTES_READ");
-    metricValues = ImmutableMap.of(ts - 100000, (Number)78, ts - 80000, 157);
-    m1.setType(Type.TIME_SERIES);
-    m1.setValues(metricValues);
-    metrics.add(m1);
-    m1 = new TimelineMetric();
-    m1.setId("MAP11_SLOT_MILLIS");
-    m1.setType(Type.SINGLE_VALUE);
-    m1.addValue(ts - 100000, 122);
-    metrics.add(m1);
-    entity6.addMetrics(metrics);
-    TimelineEvent event61 = new TimelineEvent();
-    event61.setId("event1");
-    event61.setTimestamp(cTime);
-    entity6.addEvent(event61);
-    TimelineEvent event62 = new TimelineEvent();
-    event62.setId("event5");
-    event62.setTimestamp(cTime);
-    entity6.addEvent(event62);
-    TimelineEvent event63 = new TimelineEvent();
-    event63.setId("event3");
-    event63.setTimestamp(cTime);
-    entity6.addEvent(event63);
-    TimelineEvent event64 = new TimelineEvent();
-    event64.setId("event6");
-    event64.setTimestamp(cTime);
-    entity6.addEvent(event64);
-    Map<String, Set<String>> isRelatedTo2 = new HashMap<String, Set<String>>();
-    isRelatedTo2.put("type2",
-        Sets.newHashSet("entity21","entity22","entity23","entity24"));
-    isRelatedTo2.put("type5", Sets.newHashSet("entity51","entity52"));
-    isRelatedTo2.put("type6", Sets.newHashSet("entity61","entity66"));
-    isRelatedTo2.put("type3", Sets.newHashSet("entity31"));
-    entity6.addIsRelatedToEntities(isRelatedTo2);
-    Map<String, Set<String>> relatesTo2 = new HashMap<String, Set<String>>();
-    relatesTo2.put("type2",
-        Sets.newHashSet("entity21","entity22","entity23","entity24"));
-    relatesTo2.put("type5", Sets.newHashSet("entity51","entity52"));
-    relatesTo2.put("type6", Sets.newHashSet("entity61","entity66"));
-    relatesTo2.put("type3", Sets.newHashSet("entity31"));
-    entity6.addRelatesToEntities(relatesTo2);
-    te5.addEntity(entity6);
-
-    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.write(cluster, user, flow, flowVersion, runid,
-          "application_1111111111_1111", te5);
-      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.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
-      config.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
-      config.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS,
-          "localhost:0");
-      config.set(YarnConfiguration.RM_CLUSTER_ID, "cluster1");
-      config.set(YarnConfiguration.TIMELINE_SERVICE_READER_CLASS,
-          "org.apache.hadoop.yarn.server.timelineservice.storage." +
-              "HBaseTimelineReaderImpl");
-      config.setInt("hfile.format.version", 3);
-      server = new TimelineReaderServer();
-      server.init(config);
-      server.start();
-      serverPort = server.getWebServerPort();
-    } catch (Exception e) {
-      Assert.fail("Web server failed to start");
-    }
-  }
-
-  private static Client createClient() {
-    ClientConfig cfg = new DefaultClientConfig();
-    cfg.getClasses().add(YarnJacksonJaxbJsonProvider.class);
-    return new Client(new URLConnectionClientHandler(
-        new DummyURLConnectionFactory()), cfg);
-  }
-
-  private static ClientResponse getResponse(Client client, URI uri)
-      throws Exception {
-    ClientResponse resp =
-        client.resource(uri).accept(MediaType.APPLICATION_JSON)
-        .type(MediaType.APPLICATION_JSON).get(ClientResponse.class);
-    if (resp == null ||
-        resp.getClientResponseStatus() != ClientResponse.Status.OK) {
-      String msg = 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/clusters/cluster1/users/user1/flows/flow_name/runs/" +
-          "1002345678919");
-      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(3, 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);
-      TimelineMetric m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
-          "MAP1_SLOT_MILLIS", ts - 80000, 40L);
-      for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue(verifyMetrics(metric, m1, m2, m3));
-      }
-
-      // Query without specifying cluster ID.
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/users/user1/flows/flow_name/runs/1002345678919");
-      resp = getResponse(client, uri);
-      entity = resp.getEntity(FlowRunEntity.class);
-      assertNotNull(entity);
-      assertEquals("user1@flow_name/1002345678919", entity.getId());
-      assertEquals(3, 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);
-      m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
-          "MAP1_SLOT_MILLIS", ts - 80000, 40L);
-      for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue(verifyMetrics(metric, m1, m2, m3));
-      }
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetFlowRuns() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs");
-      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/" +
-          "clusters/cluster1/users/user1/flows/flow_name/runs?limit=1");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
-      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/clusters/cluster1/users/user1/flows/flow_name/runs?" +
-          "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/clusters/cluster1/users/user1/flows/flow_name/runs?" +
-          "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/clusters/cluster1/users/user1/flows/flow_name/runs?" +
-          "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/clusters/cluster1/users/user1/flows/flow_name/runs?" +
-          "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() == 3)) ||
-            ((entity.getId().equals("user1@flow_name/1002345678920")) &&
-            (entity.getRunId() == 1002345678920L) &&
-            (entity.getStartTime() == 1425016501034L) &&
-            (entity.getMetrics().size() == 1)));
-      }
-    } finally {
-      client.destroy();
-    }
-  }
-
-
-  @Test
-  public void testGetFlowRunsMetricsToRetrieve() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
-          "metricstoretrieve=MAP_,HDFS_");
-      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());
-      int metricCnt = 0;
-      for (FlowRunEntity entity : entities) {
-        metricCnt += entity.getMetrics().size();
-        for (TimelineMetric metric : entity.getMetrics()) {
-          assertTrue(metric.getId().startsWith("MAP_") ||
-              metric.getId().startsWith("HDFS_"));
-        }
-      }
-      assertEquals(3, metricCnt);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
-          "metricstoretrieve=!(MAP_,HDFS_)");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
-      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-      metricCnt = 0;
-      for (FlowRunEntity entity : entities) {
-        metricCnt += entity.getMetrics().size();
-        for (TimelineMetric metric : entity.getMetrics()) {
-          assertTrue(metric.getId().startsWith("MAP1_"));
-        }
-      }
-      assertEquals(1, metricCnt);
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetEntitiesByUID() throws Exception {
-    Client client = createClient();
-    try {
-      // Query all flows.
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flows");
-      ClientResponse resp = getResponse(client, uri);
-      Set<FlowActivityEntity> flowEntities =
-          resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(flowEntities);
-      assertEquals(2, flowEntities.size());
-      List<String> listFlowUIDs = new ArrayList<String>();
-      for (FlowActivityEntity entity : flowEntities) {
-        String flowUID =
-            (String)entity.getInfo().get(TimelineReaderManager.UID_KEY);
-        listFlowUIDs.add(flowUID);
-        assertEquals(TimelineUIDConverter.FLOW_UID.encodeUID(
-            new TimelineReaderContext(entity.getCluster(), entity.getUser(),
-            entity.getFlowName(), null, null, null, null)), flowUID);
-        assertTrue((entity.getId().endsWith("@flow_name") &&
-            entity.getFlowRuns().size() == 2) ||
-            (entity.getId().endsWith("@flow_name2") &&
-            entity.getFlowRuns().size() == 1));
-      }
-
-      // Query flowruns based on UID returned in query above.
-      List<String> listFlowRunUIDs = new ArrayList<String>();
-      for (String flowUID : listFlowUIDs) {
-        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-            "timeline/flow-uid/" + flowUID + "/runs");
-        resp = getResponse(client, uri);
-        Set<FlowRunEntity> frEntities =
-            resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
-        assertNotNull(frEntities);
-        for (FlowRunEntity entity : frEntities) {
-          String flowRunUID =
-              (String)entity.getInfo().get(TimelineReaderManager.UID_KEY);
-          listFlowRunUIDs.add(flowRunUID);
-          assertEquals(TimelineUIDConverter.FLOWRUN_UID.encodeUID(
-              new TimelineReaderContext("cluster1", entity.getUser(),
-              entity.getName(), entity.getRunId(), null, null, null)),
-              flowRunUID);
-        }
-      }
-      assertEquals(3, listFlowRunUIDs.size());
-
-      // Query single flowrun based on UIDs' returned in query to get flowruns.
-      for (String flowRunUID : listFlowRunUIDs) {
-        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-            "timeline/run-uid/" + flowRunUID);
-        resp = getResponse(client, uri);
-        FlowRunEntity entity = resp.getEntity(FlowRunEntity.class);
-        assertNotNull(entity);
-      }
-
-      // Query apps based on UIDs' returned in query to get flowruns.
-      List<String> listAppUIDs = new ArrayList<String>();
-      for (String flowRunUID : listFlowRunUIDs) {
-        TimelineReaderContext context =
-            TimelineUIDConverter.FLOWRUN_UID.decodeUID(flowRunUID);
-        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-            "timeline/run-uid/" + flowRunUID + "/apps");
-        resp = getResponse(client, uri);
-        Set<TimelineEntity> appEntities =
-            resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-        assertNotNull(appEntities);
-        for (TimelineEntity entity : appEntities) {
-          String appUID =
-              (String)entity.getInfo().get(TimelineReaderManager.UID_KEY);
-          listAppUIDs.add(appUID);
-          assertEquals(TimelineUIDConverter.APPLICATION_UID.encodeUID(
-              new TimelineReaderContext(context.getClusterId(),
-              context.getUserId(), context.getFlowName(),
-              context.getFlowRunId(), entity.getId(), null, null)), appUID);
-        }
-      }
-      assertEquals(4, listAppUIDs.size());
-
-      // Query single app based on UIDs' returned in query to get apps.
-      for (String appUID : listAppUIDs) {
-        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-            "timeline/app-uid/" + appUID);
-        resp = getResponse(client, uri);
-        TimelineEntity entity = resp.getEntity(TimelineEntity.class);
-        assertNotNull(entity);
-      }
-
-      // Query entities based on UIDs' returned in query to get apps and
-      // a specific entity type(in this case type1).
-      List<String> listEntityUIDs = new ArrayList<String>();
-      for (String appUID : listAppUIDs) {
-        TimelineReaderContext context =
-            TimelineUIDConverter.APPLICATION_UID.decodeUID(appUID);
-        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-            "timeline/app-uid/" + appUID + "/entities/type1");
-        resp = getResponse(client, uri);
-        Set<TimelineEntity> entities =
-            resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-        assertNotNull(entities);
-        for (TimelineEntity entity : entities) {
-          String entityUID =
-              (String)entity.getInfo().get(TimelineReaderManager.UID_KEY);
-          listEntityUIDs.add(entityUID);
-          assertEquals(TimelineUIDConverter.GENERIC_ENTITY_UID.encodeUID(
-              new TimelineReaderContext(context.getClusterId(),
-              context.getUserId(), context.getFlowName(),
-              context.getFlowRunId(), context.getAppId(), "type1",
-              entity.getId())), entityUID);
-        }
-      }
-      assertEquals(2, listEntityUIDs.size());
-
-      // Query single entity based on UIDs' returned in query to get entities.
-      for (String entityUID : listEntityUIDs) {
-        uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-            "timeline/entity-uid/" + entityUID);
-        resp = getResponse(client, uri);
-        TimelineEntity entity = resp.getEntity(TimelineEntity.class);
-        assertNotNull(entity);
-      }
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/flow-uid/dummy:flow/runs");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/run-uid/dummy:flowrun");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-
-      // Run Id is not a numerical value.
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/run-uid/some:dummy:flow:123v456");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/run-uid/dummy:flowrun/apps");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/app-uid/dummy:app");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/app-uid/dummy:app/entities/type1");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/entity-uid/dummy:entity");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testUIDQueryWithAndWithoutFlowContextInfo() throws Exception {
-    Client client = createClient();
-    try {
-      String appUIDWithFlowInfo =
-          "cluster1!user1!flow_name!1002345678919!application_1111111111_1111";
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/"+
-          "timeline/app-uid/" + appUIDWithFlowInfo);
-      ClientResponse resp = getResponse(client, uri);
-      TimelineEntity appEntity1 = resp.getEntity(TimelineEntity.class);
-      assertNotNull(appEntity1);
-      assertEquals(
-          TimelineEntityType.YARN_APPLICATION.toString(), appEntity1.getType());
-      assertEquals("application_1111111111_1111", appEntity1.getId());
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "app-uid/" + appUIDWithFlowInfo + "/entities/type1");
-      resp = getResponse(client, uri);
-      Set<TimelineEntity> entities1 =
-          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities1);
-      assertEquals(2, entities1.size());
-      for (TimelineEntity entity : entities1) {
-        assertNotNull(entity.getInfo());
-        assertEquals(1, entity.getInfo().size());
-        String uid =
-            (String) entity.getInfo().get(TimelineReaderManager.UID_KEY);
-        assertNotNull(uid);
-        assertTrue(uid.equals(appUIDWithFlowInfo + "!type1!entity1") ||
-            uid.equals(appUIDWithFlowInfo + "!type1!entity2"));
-      }
-
-      String appUIDWithoutFlowInfo = "cluster1!application_1111111111_1111";
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+
-          "app-uid/" + appUIDWithoutFlowInfo);
-      resp = getResponse(client, uri);;
-      TimelineEntity appEntity2 = resp.getEntity(TimelineEntity.class);
-      assertNotNull(appEntity2);
-      assertEquals(
-          TimelineEntityType.YARN_APPLICATION.toString(), appEntity2.getType());
-      assertEquals("application_1111111111_1111", appEntity2.getId());
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "app-uid/" + appUIDWithoutFlowInfo + "/entities/type1");
-      resp = getResponse(client, uri);
-      Set<TimelineEntity> entities2 =
-          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities2);
-      assertEquals(2, entities2.size());
-      for (TimelineEntity entity : entities2) {
-        assertNotNull(entity.getInfo());
-        assertEquals(1, entity.getInfo().size());
-        String uid =
-            (String) entity.getInfo().get(TimelineReaderManager.UID_KEY);
-        assertNotNull(uid);
-        assertTrue(uid.equals(appUIDWithoutFlowInfo + "!type1!entity1") ||
-            uid.equals(appUIDWithoutFlowInfo + "!type1!entity2"));
-      }
-
-      String entityUIDWithFlowInfo = appUIDWithFlowInfo + "!type1!entity1";
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+
-          "entity-uid/" + entityUIDWithFlowInfo);
-      resp = getResponse(client, uri);;
-      TimelineEntity singleEntity1 = resp.getEntity(TimelineEntity.class);
-      assertNotNull(singleEntity1);
-      assertEquals("type1", singleEntity1.getType());
-      assertEquals("entity1", singleEntity1.getId());
-
-      String entityUIDWithoutFlowInfo =
-          appUIDWithoutFlowInfo + "!type1!entity1";
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/"+
-          "entity-uid/" + entityUIDWithoutFlowInfo);
-      resp = getResponse(client, uri);;
-      TimelineEntity singleEntity2 = resp.getEntity(TimelineEntity.class);
-      assertNotNull(singleEntity2);
-      assertEquals("type1", singleEntity2.getType());
-      assertEquals("entity1", singleEntity2.getId());
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testUIDNotProperlyEscaped() throws Exception {
-    Client client = createClient();
-    try {
-      String appUID =
-          "cluster1!user*1!flow_name!1002345678919!application_1111111111_1111";
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/"+
-          "timeline/app-uid/" + appUID);
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetFlows() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/flows");
-      ClientResponse resp = getResponse(client, uri);
-      Set<FlowActivityEntity> entities =
-          resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-      for (FlowActivityEntity entity : entities) {
-        assertTrue((entity.getId().endsWith("@flow_name") &&
-            entity.getFlowRuns().size() == 2) ||
-            (entity.getId().endsWith("@flow_name2") &&
-            entity.getFlowRuns().size() == 1));
-      }
-
-      // 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/clusters/cluster1/flows?limit=1");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-
-      long firstFlowActivity =
-          TimelineStorageUtils.getTopOfTheDayTimestamp(1425016501000L);
-
-      DateFormat fmt = TimelineReaderWebServices.DATE_FORMAT.get();
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/flows?daterange="
-          + fmt.format(firstFlowActivity) + "-"
-          + fmt.format(dayTs));
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-      for (FlowActivityEntity entity : entities) {
-        assertTrue((entity.getId().endsWith("@flow_name") &&
-            entity.getFlowRuns().size() == 2) ||
-            (entity.getId().endsWith("@flow_name2") &&
-            entity.getFlowRuns().size() == 1));
-      }
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/flows?daterange=" +
-          fmt.format(dayTs + (4*86400000L)));
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(0, entities.size());
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/flows?daterange=-" +
-          fmt.format(dayTs));
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/flows?daterange=" +
-           fmt.format(firstFlowActivity) + "-");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<FlowActivityEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/flows?daterange=20150711:20150714");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/flows?daterange=20150714-20150711");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/flows?daterange=2015071129-20150712");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/flows?daterange=20150711-2015071243");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetApp() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111?" +
-          "userid=user1&fields=ALL&flowname=flow_name&flowrunid=1002345678919");
-      ClientResponse resp = getResponse(client, uri);
-      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
-      assertNotNull(entity);
-      assertEquals("application_1111111111_1111", entity.getId());
-      assertEquals(3, 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);
-      TimelineMetric m3 = newMetric(TimelineMetric.Type.TIME_SERIES,
-          "MAP1_SLOT_MILLIS", ts - 100000, 2L);
-      m3.addValue(ts - 80000, 40L);
-      for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue(verifyMetrics(metric, m1, m2, m3));
-      }
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-              "timeline/apps/application_1111111111_2222?userid=user1" +
-              "&fields=metrics&flowname=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 m4 = newMetric(TimelineMetric.Type.TIME_SERIES,
-         "MAP_SLOT_MILLIS", ts - 100000, 5L);
-      m4.addValue(ts - 80000, 101L);
-      for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue(verifyMetrics(metric, m4));
-      }
-    } finally {
-        client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetAppWithoutFlowInfo() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111?" +
-          "fields=ALL");
-      ClientResponse resp = getResponse(client, uri);
-      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
-      assertNotNull(entity);
-      assertEquals("application_1111111111_1111", entity.getId());
-      assertEquals(3, 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);
-      TimelineMetric m3 = newMetric(TimelineMetric.Type.TIME_SERIES,
-          "MAP1_SLOT_MILLIS", ts - 100000, 2L);
-      m3.addValue(ts - 80000, 40L);
-      for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue(verifyMetrics(metric, m1, m2, m3));
-      }
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetEntityWithoutFlowInfo() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/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/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/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();
-    }
-  }
-
-  /**
-   * Tests if specific configs and metrics are retrieve for getEntities call.
-   */
-  @Test
-  public void testGetEntitiesDataToRetrieve() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?confstoretrieve=cfg_");
-      ClientResponse resp = getResponse(client, uri);
-      Set<TimelineEntity> entities =
-          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-      int cfgCnt = 0;
-      for (TimelineEntity entity : entities) {
-        cfgCnt += entity.getConfigs().size();
-        for (String configKey : entity.getConfigs().keySet()) {
-          assertTrue(configKey.startsWith("cfg_"));
-        }
-      }
-      assertEquals(2, cfgCnt);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?confstoretrieve=cfg_,config_");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-      cfgCnt = 0;
-      for (TimelineEntity entity : entities) {
-        cfgCnt += entity.getConfigs().size();
-        for (String configKey : entity.getConfigs().keySet()) {
-          assertTrue(configKey.startsWith("cfg_") ||
-              configKey.startsWith("config_"));
-        }
-      }
-      assertEquals(5, cfgCnt);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?confstoretrieve=!(cfg_,config_)");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-      cfgCnt = 0;
-      for (TimelineEntity entity : entities) {
-        cfgCnt += entity.getConfigs().size();
-        for (String configKey : entity.getConfigs().keySet()) {
-          assertTrue(configKey.startsWith("configuration_"));
-        }
-      }
-      assertEquals(1, cfgCnt);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?metricstoretrieve=MAP_");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-      int metricCnt = 0;
-      for (TimelineEntity entity : entities) {
-        metricCnt += entity.getMetrics().size();
-        for (TimelineMetric metric : entity.getMetrics()) {
-          assertTrue(metric.getId().startsWith("MAP_"));
-        }
-      }
-      assertEquals(1, metricCnt);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?metricstoretrieve=MAP1_,HDFS_");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-      metricCnt = 0;
-      for (TimelineEntity entity : entities) {
-        metricCnt += entity.getMetrics().size();
-        for (TimelineMetric metric : entity.getMetrics()) {
-          assertTrue(metric.getId().startsWith("MAP1_") ||
-              metric.getId().startsWith("HDFS_"));
-        }
-      }
-      assertEquals(3, metricCnt);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?metricstoretrieve=!(MAP1_,HDFS_)");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-      metricCnt = 0;
-      for (TimelineEntity entity : entities) {
-        metricCnt += entity.getMetrics().size();
-        for (TimelineMetric metric : entity.getMetrics()) {
-          assertTrue(metric.getId().startsWith("MAP_") ||
-              metric.getId().startsWith("MAP11_"));
-        }
-      }
-      assertEquals(2, metricCnt);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?metricstoretrieve=!(MAP1_,HDFS_");
-      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetEntitiesConfigFilters() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?conffilters=config_param1%20eq%20value1%20OR%20" +
-          "config_param1%20eq%20value3");
-      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"));
-      }
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?conffilters=config_param1%20eq%20value1%20AND" +
-          "%20configuration_param2%20eq%20value2");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(0, entities.size());
-
-      // conffilters=(config_param1 eq value1 AND configuration_param2 eq
-      // value2) OR (config_param1 eq value3 AND cfg_param3 eq value1)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" +
-          "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" +
-          "%20value3%20AND%20cfg_param3%20eq%20value1)");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      int cfgCnt = 0;
-      for (TimelineEntity entity : entities) {
-        cfgCnt += entity.getConfigs().size();
-        assertTrue(entity.getId().equals("entity2"));
-      }
-      assertEquals(0, cfgCnt);
-
-      // conffilters=(config_param1 eq value1 AND configuration_param2 eq
-      // value2) OR (config_param1 eq value3 AND cfg_param3 eq value1)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" +
-          "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" +
-          "%20value3%20AND%20cfg_param3%20eq%20value1)&fields=CONFIGS");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      cfgCnt = 0;
-      for (TimelineEntity entity : entities) {
-        cfgCnt += entity.getConfigs().size();
-        assertTrue(entity.getId().equals("entity2"));
-      }
-      assertEquals(3, cfgCnt);
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" +
-          "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" +
-          "%20value3%20AND%20cfg_param3%20eq%20value1)&confstoretrieve=cfg_," +
-          "configuration_");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      cfgCnt = 0;
-      for (TimelineEntity entity : entities) {
-        cfgCnt += entity.getConfigs().size();
-        assertTrue(entity.getId().equals("entity2"));
-        for (String configKey : entity.getConfigs().keySet()) {
-          assertTrue(configKey.startsWith("cfg_") ||
-              configKey.startsWith("configuration_"));
-        }
-      }
-      assertEquals(2, cfgCnt);
-
-      // Test for behavior when compare op is ne(not equals) vs ene
-      // (exists and not equals). configuration_param2 does not exist for
-      // entity1. For ne, both entity1 and entity2 will be returned. For ene,
-      // only entity2 will be returned as we are checking for existence too.
-      // conffilters=configuration_param2 ne value3
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?conffilters=configuration_param2%20ne%20value3");
-      resp = getResponse(client, uri);
-      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"));
-      }
-      // conffilters=configuration_param2 ene value3
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?conffilters=configuration_param2%20ene%20value3");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      for (TimelineEntity entity : entities) {
-        assertTrue(entity.getId().equals("entity2"));
-      }
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetEntitiesInfoFilters() throws Exception {
-    Client client = createClient();
-    try {
-      // infofilters=info1 eq cluster1 OR info1 eq cluster2
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?infofilters=info1%20eq%20cluster1%20OR%20info1%20eq" +
-          "%20cluster2");
-      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"));
-      }
-
-      // infofilters=info1 eq cluster1 AND info4 eq 35000
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?infofilters=info1%20eq%20cluster1%20AND%20info4%20" +
-          "eq%2035000");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(0, entities.size());
-
-      // infofilters=info4 eq 35000 OR info4 eq 36000
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?infofilters=info4%20eq%2035000%20OR%20info4%20eq" +
-          "%2036000");
-      resp = getResponse(client, uri);
-      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"));
-      }
-
-      // infofilters=(info1 eq cluster1 AND info4 eq 35000) OR
-      // (info1 eq cluster2 AND info2 eq 2.0)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?infofilters=(info1%20eq%20cluster1%20AND%20info4%20" +
-          "eq%2035000)%20OR%20(info1%20eq%20cluster2%20AND%20info2%20eq%202.0)");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      int infoCnt = 0;
-      for (TimelineEntity entity : entities) {
-        infoCnt += entity.getInfo().size();
-        assertTrue(entity.getId().equals("entity2"));
-      }
-      // Includes UID in info field even if fields not specified as INFO.
-      assertEquals(1, infoCnt);
-
-      // infofilters=(info1 eq cluster1 AND info4 eq 35000) OR
-      // (info1 eq cluster2 AND info2 eq 2.0)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?infofilters=(info1%20eq%20cluster1%20AND%20info4%20" +
-          "eq%2035000)%20OR%20(info1%20eq%20cluster2%20AND%20info2%20eq%20" +
-          "2.0)&fields=INFO");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      infoCnt = 0;
-      for (TimelineEntity entity : entities) {
-        infoCnt += entity.getInfo().size();
-        assertTrue(entity.getId().equals("entity2"));
-      }
-      // Includes UID in info field.
-      assertEquals(4, infoCnt);
-
-      // Test for behavior when compare op is ne(not equals) vs ene
-      // (exists and not equals). info3 does not exist for entity2. For ne,
-      // both entity1 and entity2 will be returned. For ene, only entity2 will
-      // be returned as we are checking for existence too.
-      // infofilters=info3 ne 39000
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?infofilters=info3%20ne%2039000");
-      resp = getResponse(client, uri);
-      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"));
-      }
-      // infofilters=info3 ene 39000
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?infofilters=info3%20ene%2039000");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      for (TimelineEntity entity : entities) {
-        assertTrue(entity.getId().equals("entity1"));
-      }
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetEntitiesMetricFilters() throws Exception {
-    Client client = createClient();
-    try {
-      // metricfilters=HDFS_BYTES_READ lt 60 OR HDFS_BYTES_READ eq 157
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?metricfilters=HDFS_BYTES_READ%20lt%2060%20OR%20" +
-          "HDFS_BYTES_READ%20eq%20157");
-      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"));
-      }
-
-      // metricfilters=HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?metricfilters=HDFS_BYTES_READ%20lt%2060%20AND%20" +
-          "MAP_SLOT_MILLIS%20gt%2040");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(0, entities.size());
-
-      // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR
-      // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
-          "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
-          "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      int metricCnt = 0;
-      for (TimelineEntity entity : entities) {
-        metricCnt += entity.getMetrics().size();
-        assertTrue(entity.getId().equals("entity2"));
-      }
-      assertEquals(0, metricCnt);
-
-      // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR
-      // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
-          "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
-          "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)&fields=METRICS");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      metricCnt = 0;
-      for (TimelineEntity entity : entities) {
-        metricCnt += entity.getMetrics().size();
-        assertTrue(entity.getId().equals("entity2"));
-      }
-      assertEquals(3, metricCnt);
-
-      // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR
-      // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122)
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
-          "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
-          "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)&metricstoretrieve=" +
-          "!(HDFS)");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      metricCnt = 0;
-      for (TimelineEntity entity : entities) {
-        metricCnt += entity.getMetrics().size();
-        assertTrue(entity.getId().equals("entity2"));
-        for (TimelineMetric metric : entity.getMetrics()) {
-          assertTrue(metric.getId().startsWith("MAP1"));
-        }
-      }
-      assertEquals(2, metricCnt);
-
-      // Test for behavior when compare op is ne(not equals) vs ene
-      // (exists and not equals). MAP11_SLOT_MILLIS does not exist for
-      // entity1. For ne, both entity1 and entity2 will be returned. For ene,
-      // only entity2 will be returned as we are checking for existence too.
-      // metricfilters=MAP11_SLOT_MILLIS ne 100
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?metricfilters=MAP11_SLOT_MILLIS%20ne%20100");
-      resp = getResponse(client, uri);
-      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"));
-      }
-      // metricfilters=MAP11_SLOT_MILLIS ene 100
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?metricfilters=MAP11_SLOT_MILLIS%20ene%20100");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      for (TimelineEntity entity : entities) {
-        assertTrue(entity.getId().equals("entity2"));
-      }
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetEntitiesEventFilters() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?eventfilters=event1,event3");
-      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"));
-      }
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?eventfilters=!(event1,event3)");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(0, entities.size());
-
-      // eventfilters=!(event1,event3) OR event5,event6
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?eventfilters=!(event1,event3)%20OR%20event5,event6");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      for (TimelineEntity entity : entities) {
-        assertTrue(entity.getId().equals("entity2"));
-      }
-
-      //  eventfilters=(!(event1,event3) OR event5,event6) OR
-      // (event1,event2 AND (event3,event4))
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?eventfilters=(!(event1,event3)%20OR%20event5," +
-          "event6)%20OR%20(event1,event2%20AND%20(event3,event4))");
-      resp = getResponse(client, uri);
-      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 testGetEntitiesRelationFilters() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1?isrelatedto=type3:entity31,type2:entity21:entity22");
-      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"));
-      }
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
-          "?isrelatedto=!(type3:entity31,type2:entity21:entity22)");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(0, entities.size());
-
-      // isrelatedto=!(type3:entity31,type2:entity21:entity22)OR type5:entity51,
-      // type6:entity61:entity66
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
-          "?isrelatedto=!(type3:entity31,type2:entity21:entity22)%20OR%20" +
-          "type5:entity51,type6:entity61:entity66");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      for (TimelineEntity entity : entities) {
-        assertTrue(entity.getId().equals("entity2"));
-      }
-
-      // isrelatedto=(!(type3:entity31,type2:entity21:entity22)OR type5:
-      // entity51,type6:entity61:entity66) OR (type1:entity14,type2:entity21:
-      // entity22 AND (type3:entity32:entity35,type4:entity42))
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
-          "?isrelatedto=(!(type3:entity31,type2:entity21:entity22)%20OR%20" +
-          "type5:entity51,type6:entity61:entity66)%20OR%20(type1:entity14," +
-          "type2:entity21:entity22%20AND%20(type3:entity32:entity35,"+
-          "type4:entity42))");
-      resp = getResponse(client, uri);
-      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"));
-      }
-
-      // relatesto=!(type3:entity31,type2:entity21:entity22)OR type5:entity51,
-      // type6:entity61:entity66
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
-          "?relatesto=!%20(type3:entity31,type2:entity21:entity22%20)%20OR%20" +
-          "type5:entity51,type6:entity61:entity66");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(1, entities.size());
-      for (TimelineEntity entity : entities) {
-        assertTrue(entity.getId().equals("entity2"));
-      }
-
-      // relatesto=(!(type3:entity31,type2:entity21:entity22)OR type5:entity51,
-      // type6:entity61:entity66) OR (type1:entity14,type2:entity21:entity22 AND
-      // (type3:entity32:entity35 , type4:entity42))
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
-          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
-          "?relatesto=(!(%20type3:entity31,type2:entity21:entity22)%20OR%20" +
-          "type5:entity51,type6:entity61:entity66%20)%20OR%20(type1:entity14," +
-          "type2:entity21:entity22%20AND%20(type3:entity32:entity35%20,%20"+
-          "type4:entity42))");
-      resp = getResponse(client, uri);
-      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();
-    }
-  }
-
-  /**
-   * Tests if specific configs and metrics are retrieve for getEntity call.
-   */
-  @Test
-  public void testGetEntityDataToRetrieve() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1/entity2?confstoretrieve=cfg_,configuration_");
-      ClientResponse resp = getResponse(client, uri);
-      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
-      assertNotNull(entity);
-      assertEquals("entity2", entity.getId());
-      assertEquals("type1", entity.getType());
-      assertEquals(2, entity.getConfigs().size());
-      for (String configKey : entity.getConfigs().keySet()) {
-        assertTrue(configKey.startsWith("configuration_") ||
-            configKey.startsWith("cfg_"));
-      }
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1/entity2?confstoretrieve=!(cfg_,configuration_)");
-      resp = getResponse(client, uri);
-      entity = resp.getEntity(TimelineEntity.class);
-      assertNotNull(entity);
-      assertEquals("entity2", entity.getId());
-      assertEquals("type1", entity.getType());
-      assertEquals(1, entity.getConfigs().size());
-      for (String configKey : entity.getConfigs().keySet()) {
-        assertTrue(configKey.startsWith("config_"));
-      }
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1/entity2?metricstoretrieve=MAP1_,HDFS_");
-      resp = getResponse(client, uri);
-      entity = resp.getEntity(TimelineEntity.class);
-      assertNotNull(entity);
-      assertEquals("entity2", entity.getId());
-      assertEquals("type1", entity.getType());
-      assertEquals(2, entity.getMetrics().size());
-      for (TimelineMetric  metric : entity.getMetrics()) {
-        assertTrue(metric.getId().startsWith("MAP1_") ||
-            metric.getId().startsWith("HDFS_"));
-      }
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
-          "entities/type1/entity2?metricstoretrieve=!(MAP1_,HDFS_)");
-      resp = getResponse(client, uri);
-      entity = resp.getEntity(TimelineEntity.class);
-      assertNotNull(entity);
-      assertEquals("entity2", entity.getId());
-      assertEquals("type1", entity.getType());
-      assertEquals(1, entity.getMetrics().size());
-      for (TimelineMetric  metric : entity.getMetrics()) {
-        assertTrue(metric.getId().startsWith("MAP11_"));
-      }
-    } finally {
-      client.destroy();
-    }
-  }
-
-  @Test
-  public void testGetFlowRunApps() throws Exception {
-    Client client = createClient();
-    try {
-      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs/" +
-          "1002345678919/apps?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() == 3) ||
-            (entity.getId().equals("application_1111111111_2222") &&
-            entity.getMetrics().size() == 1));
-      }
-
-      // Query without specifying cluster ID.
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/users/user1/flows/flow_name/runs/1002345678919/apps");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(2, entities.size());
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/users/user1/flows/flow_name/runs/1002345678919/" +
-          "apps?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/clusters/cluster1/users/user1/flows/flow_name/apps?" +
-          "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() == 3) ||
-            (entity.getId().equals("application_1111111111_2222") &&
-            entity.getMetrics().size() == 1) ||
-            (entity.getId().equals("application_1111111111_2224") &&
-            entity.getMetrics().size() == 1));
-      }
-
-      // Query without specifying cluster ID.
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/users/user1/flows/flow_name/apps");
-      resp = getResponse(client, uri);
-      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
-      assertNotNull(entities);
-      assertEquals(3, entities.size());
-
-      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
-          "timeline/users/user1/flows/flow_name/apps?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/clusters/cluster1/users/user1/flows/flow_name/apps?" +
-          "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/clusters/cluster1/users/user1/flows/flow_name/apps?" +
-          "metricfilters=HDFS_BYTES_READ%20ge%200");
-      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/clusters/cluster1/users/user1/flows/flow_name/apps?" +
-          "conffilters=cfg1%20eq%20value1");
-      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/clusters/cluster1/users/user1/flows/flow_name/runs/" +
-          "1002345678929");
-      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/clusters/cluster2/flows");
-      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/clusters/cluster1/apps/application_1111111111_1378");
-      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/clusters/cluster2/users/user1/flows/flow_name/runs/" +
-          "1002345678919/apps");
-      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/clusters/cluster2/users/user1/flows/flow_name55/apps");
-      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;
-    }
-  }
-}


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


[02/50] [abbrv] hadoop git commit: YARN-3863. Support complex filters in TimelineReader (Varun Saxena via sjlee)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/366eb54e/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 4e07ecf..6b57ec4 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
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -52,10 +53,14 @@ import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrieve;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
+import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
 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.reader.filter.TimelineKeyValuesFilter;
 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;
@@ -75,9 +80,6 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-
 /**
  * Various tests to test writing entities to HBase and reading them back from
  * it.
@@ -113,30 +115,29 @@ public class TestHBaseTimelineStorage {
     String id = "application_1111111111_2222";
     entity.setId(id);
     entity.setType(TimelineEntityType.YARN_APPLICATION.toString());
-    Long cTime = 1425016501000L;
+    Long cTime = 1425016502000L;
     entity.setCreatedTime(cTime);
     // add the info map in Timeline Entity
     Map<String, Object> infoMap = new HashMap<String, Object>();
-    infoMap.put("infoMapKey1", "infoMapValue1");
-    infoMap.put("infoMapKey2", 10);
+    infoMap.put("infoMapKey1", "infoMapValue2");
+    infoMap.put("infoMapKey2", 20);
+    infoMap.put("infoMapKey3", 85.85);
     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);
+    isRelatedToSet.add("relatedto1");
     Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
-    isRelatedTo.put(key, isRelatedToSet);
+    isRelatedTo.put("task", 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);
+    relatesToSet.add("relatesto1");
+    relatesToSet.add("relatesto3");
     Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
-    relatesTo.put(key, relatesToSet);
+    relatesTo.put("container", relatesToSet);
+    Set<String> relatesToSet11 = new HashSet<String>();
+    relatesToSet11.add("relatesto4");
+    relatesTo.put("container1", relatesToSet11);
     entity.setRelatesToEntities(relatesTo);
     // add some config entries
     Map<String, String> conf = new HashMap<String, String>();
@@ -166,8 +167,8 @@ public class TestHBaseTimelineStorage {
     metrics.add(m12);
     entity.addMetrics(metrics);
     TimelineEvent event = new TimelineEvent();
-    event.setId("event1");
-    event.setTimestamp(ts - 2000);
+    event.setId("start_event");
+    event.setTimestamp(ts);
     entity.addEvent(event);
     te.addEntity(entity);
 
@@ -176,7 +177,7 @@ public class TestHBaseTimelineStorage {
     String id1 = "application_1111111111_3333";
     entity1.setId(id1);
     entity1.setType(TimelineEntityType.YARN_APPLICATION.toString());
-    entity1.setCreatedTime(cTime);
+    entity1.setCreatedTime(cTime + 20L);
 
     // add the info map in Timeline Entity
     Map<String, Object> infoMap1 = new HashMap<String, Object>();
@@ -185,23 +186,22 @@ public class TestHBaseTimelineStorage {
     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);
+    isRelatedToSet1.add("relatedto3");
+    isRelatedToSet1.add("relatedto5");
     Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
-    isRelatedTo1.put(key, isRelatedToSet1);
+    isRelatedTo1.put("task1", isRelatedToSet1);
+    Set<String> isRelatedToSet11 = new HashSet<String>();
+    isRelatedToSet11.add("relatedto4");
+    isRelatedTo1.put("task2", isRelatedToSet11);
     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);
+    relatesToSet1.add("relatesto1");
+    relatesToSet1.add("relatesto2");
     Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
-    relatesTo1.put(key1, relatesToSet1);
+    relatesTo1.put("container", relatesToSet1);
     entity1.setRelatesToEntities(relatesTo1);
 
     // add some config entries
@@ -226,6 +226,14 @@ public class TestHBaseTimelineStorage {
     m2.setValues(metricValues1);
     metrics1.add(m2);
     entity1.addMetrics(metrics1);
+    TimelineEvent event11 = new TimelineEvent();
+    event11.setId("end_event");
+    event11.setTimestamp(ts);
+    entity1.addEvent(event11);
+    TimelineEvent event12 = new TimelineEvent();
+    event12.setId("update_event");
+    event12.setTimestamp(ts - 10);
+    entity1.addEvent(event12);
     te1.addEntity(entity1);
 
     TimelineEntities te2 = new TimelineEntities();
@@ -233,7 +241,22 @@ public class TestHBaseTimelineStorage {
     String id2 = "application_1111111111_4444";
     entity2.setId(id2);
     entity2.setType(TimelineEntityType.YARN_APPLICATION.toString());
-    entity2.setCreatedTime(cTime);
+    entity2.setCreatedTime(cTime + 40L);
+    TimelineEvent event21 = new TimelineEvent();
+    event21.setId("update_event");
+    event21.setTimestamp(ts - 20);
+    entity2.addEvent(event21);
+    Set<String> isRelatedToSet2 = new HashSet<String>();
+    isRelatedToSet2.add("relatedto3");
+    Map<String, Set<String>> isRelatedTo2 = new HashMap<String, Set<String>>();
+    isRelatedTo2.put("task1", isRelatedToSet2);
+    entity2.setIsRelatedToEntities(isRelatedTo2);
+    Map<String, Set<String>> relatesTo3 = new HashMap<String, Set<String>>();
+    Set<String> relatesToSet14 = new HashSet<String>();
+    relatesToSet14.add("relatesto7");
+    relatesTo3.put("container2", relatesToSet14);
+    entity2.setRelatesToEntities(relatesTo3);
+
     te2.addEntity(entity2);
     HBaseTimelineWriterImpl hbi = null;
     try {
@@ -267,31 +290,30 @@ public class TestHBaseTimelineStorage {
     String type = "world";
     entity.setId(id);
     entity.setType(type);
-    Long cTime = 1425016501000L;
+    Long cTime = 1425016502000L;
     entity.setCreatedTime(cTime);
     // add the info map in Timeline Entity
     Map<String, Object> infoMap = new HashMap<String, Object>();
-    infoMap.put("infoMapKey1", "infoMapValue1");
-    infoMap.put("infoMapKey2", 10);
+    infoMap.put("infoMapKey1", "infoMapValue2");
+    infoMap.put("infoMapKey2", 20);
+    infoMap.put("infoMapKey3", 71.4);
     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);
+    isRelatedToSet.add("relatedto1");
     Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
-    isRelatedTo.put(key, isRelatedToSet);
+    isRelatedTo.put("task", 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);
+    relatesToSet.add("relatesto1");
+    relatesToSet.add("relatesto3");
     Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
-    relatesTo.put(key, relatesToSet);
+    relatesTo.put("container", relatesToSet);
+    Set<String> relatesToSet11 = new HashSet<String>();
+    relatesToSet11.add("relatesto4");
+    relatesTo.put("container1", relatesToSet11);
     entity.setRelatesToEntities(relatesTo);
 
     // add some config entries
@@ -312,7 +334,7 @@ public class TestHBaseTimelineStorage {
     metricValues.put(ts - 80000, 300000000);
     metricValues.put(ts - 60000, 400000000);
     metricValues.put(ts - 40000, 50000000000L);
-    metricValues.put(ts - 20000, 60000000000L);
+    metricValues.put(ts - 20000, 70000000000L);
     m1.setType(Type.TIME_SERIES);
     m1.setValues(metricValues);
     metrics.add(m1);
@@ -322,13 +344,17 @@ public class TestHBaseTimelineStorage {
     m12.addValue(ts, 50);
     metrics.add(m12);
     entity.addMetrics(metrics);
+    TimelineEvent event = new TimelineEvent();
+    event.setId("start_event");
+    event.setTimestamp(ts);
+    entity.addEvent(event);
     te.addEntity(entity);
 
     TimelineEntity entity1 = new TimelineEntity();
     String id1 = "hello1";
     entity1.setId(id1);
     entity1.setType(type);
-    entity1.setCreatedTime(cTime);
+    entity1.setCreatedTime(cTime + 20L);
 
     // add the info map in Timeline Entity
     Map<String, Object> infoMap1 = new HashMap<String, Object>();
@@ -336,24 +362,34 @@ public class TestHBaseTimelineStorage {
     infoMap1.put("infoMapKey2", 10);
     entity1.addInfo(infoMap1);
 
+    // add event.
+    TimelineEvent event11 = new TimelineEvent();
+    event11.setId("end_event");
+    event11.setTimestamp(ts);
+    entity1.addEvent(event11);
+    TimelineEvent event12 = new TimelineEvent();
+    event12.setId("update_event");
+    event12.setTimestamp(ts - 10);
+    entity1.addEvent(event12);
+
+
     // add the isRelatedToEntity info
-    String key1 = "task";
-    String value1 = "is_related_to_entity_id_here";
     Set<String> isRelatedToSet1 = new HashSet<String>();
-    isRelatedToSet1.add(value1);
+    isRelatedToSet1.add("relatedto3");
+    isRelatedToSet1.add("relatedto5");
     Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
-    isRelatedTo1.put(key, isRelatedToSet1);
+    isRelatedTo1.put("task1", isRelatedToSet1);
+    Set<String> isRelatedToSet11 = new HashSet<String>();
+    isRelatedToSet11.add("relatedto4");
+    isRelatedTo1.put("task2", isRelatedToSet11);
     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);
+    relatesToSet1.add("relatesto1");
+    relatesToSet1.add("relatesto2");
     Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
-    relatesTo1.put(key1, relatesToSet1);
+    relatesTo1.put("container", relatesToSet1);
     entity1.setRelatesToEntities(relatesTo1);
 
     // add some config entries
@@ -384,7 +420,21 @@ public class TestHBaseTimelineStorage {
     String id2 = "hello2";
     entity2.setId(id2);
     entity2.setType(type);
-    entity2.setCreatedTime(cTime);
+    entity2.setCreatedTime(cTime + 40L);
+    TimelineEvent event21 = new TimelineEvent();
+    event21.setId("update_event");
+    event21.setTimestamp(ts - 20);
+    entity2.addEvent(event21);
+    Set<String> isRelatedToSet2 = new HashSet<String>();
+    isRelatedToSet2.add("relatedto3");
+    Map<String, Set<String>> isRelatedTo2 = new HashMap<String, Set<String>>();
+    isRelatedTo2.put("task1", isRelatedToSet2);
+    entity2.setIsRelatedToEntities(isRelatedTo2);
+    Map<String, Set<String>> relatesTo3 = new HashMap<String, Set<String>>();
+    Set<String> relatesToSet14 = new HashSet<String>();
+    relatesToSet14.add("relatesto7");
+    relatesTo3.put("container2", relatesToSet14);
+    entity2.setRelatesToEntities(relatesTo3);
     te.addEntity(entity2);
     HBaseTimelineWriterImpl hbi = null;
     try {
@@ -1113,19 +1163,585 @@ public class TestHBaseTimelineStorage {
 
   @Test
   public void testReadEntities() throws Exception {
-    TimelineEntity e1 = reader.getEntity(
+    TimelineEntity entity = reader.getEntity(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111","world", "hello"),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-    assertNotNull(e1);
-    assertEquals(3, e1.getConfigs().size());
-    assertEquals(1, e1.getIsRelatedToEntities().size());
-    Set<TimelineEntity> es1 = reader.getEntities(
+    assertNotNull(entity);
+    assertEquals(3, entity.getConfigs().size());
+    assertEquals(1, entity.getIsRelatedToEntities().size());
+    Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111","world",
         null), new TimelineEntityFilters(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-    assertEquals(3, es1.size());
+    assertEquals(3, entities.size());
+    int cfgCnt = 0;
+    int metricCnt = 0;
+    int infoCnt = 0;
+    int eventCnt = 0;
+    int relatesToCnt = 0;
+    int isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      cfgCnt += (timelineEntity.getConfigs() == null) ? 0 :
+          timelineEntity.getConfigs().size();
+      metricCnt += (timelineEntity.getMetrics() == null) ? 0 :
+          timelineEntity.getMetrics().size();
+      infoCnt += (timelineEntity.getInfo() == null) ? 0 :
+          timelineEntity.getInfo().size();
+      eventCnt += (timelineEntity.getEvents() == null) ? 0 :
+          timelineEntity.getEvents().size();
+      relatesToCnt += (timelineEntity.getRelatesToEntities() == null) ? 0 :
+          timelineEntity.getRelatesToEntities().size();
+      isRelatedToCnt += (timelineEntity.getIsRelatedToEntities() == null) ? 0 :
+          timelineEntity.getIsRelatedToEntities().size();
+    }
+    assertEquals(5, cfgCnt);
+    assertEquals(3, metricCnt);
+    assertEquals(5, infoCnt);
+    assertEquals(4, eventCnt);
+    assertEquals(4, relatesToCnt);
+    assertEquals(4, isRelatedToCnt);
+  }
+
+  @Test
+  public void testFilterEntitiesByCreatedTime() throws Exception {
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, 1425016502000L, 1425016502040L, null,
+        null, null, null, null, null), new TimelineDataToRetrieve());
+    assertEquals(3, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("hello") && !entity.getId().equals("hello1") &&
+          !entity.getId().equals("hello2")) {
+        Assert.fail("Entities with ids' hello, hello1 and hello2 should be" +
+           " present");
+      }
+    }
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, 1425016502015L, null, null, null, null,
+        null, null, null), new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("hello1") &&
+          !entity.getId().equals("hello2")) {
+        Assert.fail("Entities with ids' hello1 and hello2 should be present");
+      }
+    }
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, 1425016502015L, null, null, null,
+        null, null, null), new TimelineDataToRetrieve());
+     assertEquals(1, entities.size());
+     for (TimelineEntity entity : entities) {
+       if (!entity.getId().equals("hello")) {
+         Assert.fail("Entity with id hello should be present");
+       }
+     }
+  }
+
+  @Test
+  public void testReadEntitiesRelationsAndEventFiltersDefaultView()
+      throws Exception {
+    TimelineFilterList eventFilter = new TimelineFilterList();
+    eventFilter.addFilter(new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL,
+        "end_event"));
+    TimelineFilterList relatesTo = new TimelineFilterList(Operator.OR);
+    relatesTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    relatesTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList isRelatedTo = new TimelineFilterList();
+    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, relatesTo, isRelatedTo,
+        null, null, null, eventFilter), new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    int eventCnt = 0;
+    int isRelatedToCnt = 0;
+    int relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity id should have been hello2");
+      }
+    }
+    assertEquals(0, eventCnt);
+    assertEquals(0, isRelatedToCnt);
+    assertEquals(0, relatesToCnt);
+  }
+
+  @Test
+  public void testReadEntitiesEventFilters() throws Exception {
+    TimelineFilterList ef = new TimelineFilterList();
+    ef.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    ef.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(1, entities.size());
+    int eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity id should have been hello2");
+      }
+    }
+    assertEquals(1, eventCnt);
+
+    TimelineFilterList ef1 = new TimelineFilterList();
+    ef1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    ef1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef1),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity id should have been hello2");
+      }
+    }
+    assertEquals(0, eventCnt);
+
+    TimelineFilterList ef2 = new TimelineFilterList();
+    ef2.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef2),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity ids' should have been hello and hello2");
+      }
+    }
+    assertEquals(0, eventCnt);
+
+    TimelineFilterList ef3 = new TimelineFilterList();
+    ef3.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    ef3.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "dummy_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef3),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "update_event"));
+    list1.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "dummy_event"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.EQUAL, "start_event"));
+    TimelineFilterList ef4 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef4),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello")) {
+        Assert.fail("Entity id should have been hello");
+      }
+    }
+    assertEquals(0, eventCnt);
+
+    TimelineFilterList ef5 = new TimelineFilterList();
+    ef5.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "update_event"));
+    ef5.addFilter(new TimelineExistsFilter(
+        TimelineCompareOp.NOT_EQUAL, "end_event"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        null, ef5),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    eventCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      if (!timelineEntity.getId().equals("hello")) {
+          Assert.fail("Entity id should have been hello");
+        }
+    }
+    assertEquals(0, eventCnt);
+  }
+
+  @Test
+  public void testReadEntitiesIsRelatedTo() throws Exception {
+    TimelineFilterList irt = new TimelineFilterList(Operator.OR);
+    irt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    irt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, irt, null, null, null,
+        null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(2, entities.size());
+    int isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity ids' should have been hello and hello1");
+      }
+    }
+    assertEquals(3, isRelatedToCnt);
+
+    TimelineFilterList irt1 = new TimelineFilterList();
+    irt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    irt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt1, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity id should have been hello2");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt2 = new TimelineFilterList(Operator.OR);
+    irt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    irt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt2, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity ids' should have been hello and hello1");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt3 = new TimelineFilterList();
+    irt3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt3, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity id should have been hello1");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt4 = new TimelineFilterList();
+    irt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    irt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_task",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt4, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList irt5 = new TimelineFilterList();
+    irt5.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto7"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt5, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_task",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    TimelineFilterList irt6 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, null, irt6, null, null,
+        null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity id should have been hello1");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+  }
+
+  @Test
+  public void testReadEntitiesRelatesTo() throws Exception {
+    TimelineFilterList rt = new TimelineFilterList(Operator.OR);
+    rt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    rt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, rt, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(2, entities.size());
+    int relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity ids' should have been hello and hello2");
+      }
+    }
+    assertEquals(3, relatesToCnt);
+
+    TimelineFilterList rt1 = new TimelineFilterList();
+    rt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    rt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt1, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity id should have been hello1");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt2 = new TimelineFilterList(Operator.OR);
+    rt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    rt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt2, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello") &&
+          !timelineEntity.getId().equals("hello2")) {
+        Assert.fail("Entity ids' should have been hello and hello2");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt3 = new TimelineFilterList();
+    rt3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1", "relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt3, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello")) {
+        Assert.fail("Entity id should have been hello");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt4 = new TimelineFilterList();
+    rt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    rt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_container",
+        new HashSet<Object>(Arrays.asList("relatesto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt4, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList rt5 = new TimelineFilterList();
+    rt5.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatedto1", "relatesto8"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt5, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_container",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList rt6 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt6, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello")) {
+        Assert.fail("Entity id should have been hello");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList list3 = new TimelineFilterList();
+    list3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    list3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList list4 = new TimelineFilterList();
+    list4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    list4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto2"))));
+    TimelineFilterList combinedList =
+        new TimelineFilterList(Operator.OR, list3, list4);
+    TimelineFilterList rt7 = new TimelineFilterList(Operator.AND, combinedList,
+        new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111", "world", null),
+        new TimelineEntityFilters(null, null, null, rt7, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("hello1")) {
+        Assert.fail("Entity id should have been hello1");
+      }
+    }
+    assertEquals(0, relatesToCnt);
   }
 
   @Test
@@ -1177,7 +1793,7 @@ public class TestHBaseTimelineStorage {
       infoCnt += entity.getInfo().size();
     }
     assertEquals(0, infoCnt);
-    assertEquals(2, isRelatedToCnt);
+    assertEquals(4, isRelatedToCnt);
     assertEquals(3, metricsCnt);
   }
 
@@ -1200,13 +1816,122 @@ public class TestHBaseTimelineStorage {
     int cfgCnt = 0;
     for (TimelineEntity entity : es1) {
       cfgCnt += entity.getConfigs().size();
+      for (String confKey : entity.getConfigs().keySet()) {
+        assertTrue("Config key returned should start with cfg_",
+            confKey.startsWith("cfg_"));
+      }
+    }
+    assertEquals(3, cfgCnt);
+  }
+
+  @Test
+  public void testReadEntitiesConfigFilters() throws Exception {
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param2", "value2"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value3"));
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_param2", "value2"));
+    TimelineFilterList confFilterList =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(2, entities.size());
+    int cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(5, cfgCnt);
+
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(2, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(5, cfgCnt);
+
+    TimelineFilterList confFilterList1 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList1, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(1, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
     }
     assertEquals(3, cfgCnt);
+
+    TimelineFilterList confFilterList2 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"),
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "config_param2", "value2"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList2, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList confFilterList3 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "dummy_config", "value1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList3, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList confFilterList4 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1"));
+    entities = reader.getEntities(
+            new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+            1002345678919L, "application_1231111111_1111","world", null),
+            new TimelineEntityFilters(null, null, null, null, null, null,
+            confFilterList4, null, null),
+            new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList confFilterList5 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1", false));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList5, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(3, entities.size());
   }
 
   @Test
   public void testReadEntitiesConfigFilterPrefix() throws Exception {
-    Map<String, String> confFilters = ImmutableMap.of("cfg_param1","value1");
+    TimelineFilterList confFilterList = new TimelineFilterList();
+    confFilterList.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
     TimelineFilterList list =
         new TimelineFilterList(Operator.OR,
             new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
@@ -1214,12 +1939,47 @@ public class TestHBaseTimelineStorage {
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilters, null, null),
+        confFilterList, null, null),
         new TimelineDataToRetrieve(list, null, null));
     assertEquals(1, entities.size());
     int cfgCnt = 0;
     for (TimelineEntity entity : entities) {
       cfgCnt += entity.getConfigs().size();
+      for (String confKey : entity.getConfigs().keySet()) {
+        assertTrue("Config key returned should start with cfg_",
+            confKey.startsWith("cfg_"));
+      }
+    }
+    assertEquals(2, cfgCnt);
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param2", "value2"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value3"));
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_param2", "value2"));
+    TimelineFilterList confFilterList1 =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    TimelineFilterList confsToRetrieve =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "config_"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList1, null, null),
+        new TimelineDataToRetrieve(confsToRetrieve, null, null));
+    assertEquals(2, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+      for (String confKey : entity.getConfigs().keySet()) {
+        assertTrue("Config key returned should start with config_",
+            confKey.startsWith("config_"));
+       }
     }
     assertEquals(2, cfgCnt);
   }
@@ -1243,47 +2003,356 @@ public class TestHBaseTimelineStorage {
     int metricCnt = 0;
     for (TimelineEntity entity : es1) {
       metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
     }
     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_"));
+  public void testReadEntitiesMetricFilters() throws Exception {
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 50000000900L));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 80000000000L));
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.EQUAL, "MAP1_BYTES", 50));
+    TimelineFilterList metricFilterList =
+        new TimelineFilterList(Operator.OR, list1, list2);
     Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1","user1", "some_flow_name",
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, "application_1231111111_1111","world", null),
         new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilters, null),
-        new TimelineDataToRetrieve(null, list, null));
-    assertEquals(1, entities.size());
+        metricFilterList, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+    assertEquals(2, entities.size());
     int metricCnt = 0;
     for (TimelineEntity entity : entities) {
       metricCnt += entity.getMetrics().size();
     }
-    assertEquals(1, metricCnt);
-  }
+    assertEquals(3, metricCnt);
 
-  @Test
-  public void testReadApps() throws Exception {
-    TimelineEntity e1 = reader.getEntity(
+    entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1111111111_2222",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(2, entities.size());
+    metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(3, metricCnt);
+
+    TimelineFilterList metricFilterList1 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.LESS_OR_EQUAL, "MAP_SLOT_MILLIS", 80000000000L),
+        new TimelineCompareFilter(
+        TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList1, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+    assertEquals(1, entities.size());
+    metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+    }
+    assertEquals(2, metricCnt);
+
+    TimelineFilterList metricFilterList2 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 40000000000L),
+        new TimelineCompareFilter(
+        TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList2, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList metricFilterList3 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.EQUAL, "dummy_metric", 5));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList3, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList metricFilterList4 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_metric", 5));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList4, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList metricFilterList5 = new TimelineFilterList(
+        new TimelineCompareFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_metric", 5, false));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList5, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
+    assertEquals(3, entities.size());
+  }
+
+  @Test
+  public void testReadEntitiesMetricFilterPrefix() throws Exception {
+    TimelineFilterList metricFilterList = new TimelineFilterList();
+    metricFilterList.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 0L));
+    TimelineFilterList list =
+        new TimelineFilterList(Operator.OR,
+            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList, null),
+        new TimelineDataToRetrieve(null, list, null));
+    assertEquals(1, entities.size());
+    int metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
+    }
+    assertEquals(1, metricCnt);
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 50000000900L));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 80000000000L));
+    list2.addFilter(new TimelineCompareFilter(
+        TimelineCompareOp.EQUAL, "MAP1_BYTES", 50));
+    TimelineFilterList metricFilterList1 =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    TimelineFilterList metricsToRetrieve = new TimelineFilterList(Operator.OR,
+        new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, null, null,
+        metricFilterList1, null),
+        new TimelineDataToRetrieve(
+        null, metricsToRetrieve, EnumSet.of(Field.METRICS)));
+    assertEquals(2, entities.size());
+    metricCnt = 0;
+    for (TimelineEntity entity : entities) {
+      metricCnt += entity.getMetrics().size();
+      for (TimelineMetric metric : entity.getMetrics()) {
+        assertTrue("Metric Id returned should start with MAP1_",
+            metric.getId().startsWith("MAP1_"));
+      }
+    }
+    assertEquals(2, metricCnt);
+  }
+
+  @Test
+  public void testReadEntitiesInfoFilters() throws Exception {
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "infoMapKey3", 71.4));
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "infoMapKey1", "infoMapValue2"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "infoMapKey1", "infoMapValue1"));
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "infoMapKey2", 10));
+    TimelineFilterList infoFilterList =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+    assertEquals(2, entities.size());
+    int infoCnt = 0;
+    for (TimelineEntity entity : entities) {
+      infoCnt += entity.getInfo().size();
+    }
+    assertEquals(5, infoCnt);
+
+    TimelineFilterList infoFilterList1 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "infoMapKey1", "infoMapValue1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList1,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+    assertEquals(1, entities.size());
+    infoCnt = 0;
+    for (TimelineEntity entity : entities) {
+      infoCnt += entity.getInfo().size();
+    }
+    assertEquals(3, infoCnt);
+
+    TimelineFilterList infoFilterList2 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "infoMapKey1", "infoMapValue2"),
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "infoMapKey3", 71.4));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList2,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList infoFilterList3 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "dummy_info", "some_value"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList3,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList infoFilterList4 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_info", "some_value"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList4,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+    assertEquals(0, entities.size());
+
+    TimelineFilterList infoFilterList5 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "dummy_info", "some_value", false));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1231111111_1111","world", null),
+        new TimelineEntityFilters(null, null, null, null, null, infoFilterList5,
+        null, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.INFO)));
+    assertEquals(3, entities.size());
+  }
+
+  @Test
+  public void testReadApps() throws Exception {
+    TimelineEntity entity = reader.getEntity(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, "application_1111111111_2222",
         TimelineEntityType.YARN_APPLICATION.toString(), null),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-    assertNotNull(e1);
-    assertEquals(3, e1.getConfigs().size());
-    assertEquals(1, e1.getIsRelatedToEntities().size());
-    Set<TimelineEntity> es1 = reader.getEntities(
+    assertNotNull(entity);
+    assertEquals(3, entity.getConfigs().size());
+    assertEquals(1, entity.getIsRelatedToEntities().size());
+    Set<TimelineEntity> entities = reader.getEntities(
         new TimelineReaderContext("cluster1", "user1", "some_flow_name",
         1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
         null),
         new TimelineEntityFilters(),
         new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-    assertEquals(3, es1.size());
+    assertEquals(3, entities.size());
+    int cfgCnt = 0;
+    int metricCnt = 0;
+    int infoCnt = 0;
+    int eventCnt = 0;
+    int relatesToCnt = 0;
+    int isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      cfgCnt += (timelineEntity.getConfigs() == null) ? 0 :
+          timelineEntity.getConfigs().size();
+      metricCnt += (timelineEntity.getMetrics() == null) ? 0 :
+          timelineEntity.getMetrics().size();
+      infoCnt += (timelineEntity.getInfo() == null) ? 0 :
+          timelineEntity.getInfo().size();
+      eventCnt += (timelineEntity.getEvents() == null) ? 0 :
+          timelineEntity.getEvents().size();
+      relatesToCnt += (timelineEntity.getRelatesToEntities() == null) ? 0 :
+          timelineEntity.getRelatesToEntities().size();
+      isRelatedToCnt += (timelineEntity.getIsRelatedToEntities() == null) ? 0 :
+          timelineEntity.getIsRelatedToEntities().size();
+    }
+    assertEquals(5, cfgCnt);
+    assertEquals(3, metricCnt);
+    assertEquals(5, infoCnt);
+    assertEquals(4, eventCnt);
+    assertEquals(4, relatesToCnt);
+    assertEquals(4, isRelatedToCnt);
+  }
+
+  @Test
+  public void testFilterAppsByCreatedTime() throws Exception {
+    Set<TimelineEntity> entities = reader.getEntities(
+       new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+       1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+       null),
+       new TimelineEntityFilters(null, 1425016502000L, 1425016502040L, null,
+       null, null, null, null, null),
+       new TimelineDataToRetrieve());
+    assertEquals(3, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("application_1111111111_2222") &&
+          !entity.getId().equals("application_1111111111_3333") &&
+          !entity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entities with ids' application_1111111111_2222, " +
+           "application_1111111111_3333 and application_1111111111_4444" +
+            " should be present");
+      }
+    }
+    entities =  reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, 1425016502015L, null, null, null, null,
+        null, null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("application_1111111111_3333") &&
+          !entity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Apps with ids' application_1111111111_3333 and" +
+            " application_1111111111_4444 should be present");
+      }
+    }
+    entities =  reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, 1425016502015L, null, null, null,
+        null, null, null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    for (TimelineEntity entity : entities) {
+      if (!entity.getId().equals("application_1111111111_2222")) {
+        Assert.fail("App with id application_1111111111_2222 should" +
+            " be present");
+      }
+    }
   }
 
   @Test
@@ -1339,11 +2408,649 @@ public class TestHBaseTimelineStorage {
       infoCnt += entity.getInfo().size();
     }
     assertEquals(0, infoCnt);
-    assertEquals(2, isRelatedToCnt);
+    assertEquals(4, isRelatedToCnt);
     assertEquals(3, metricsCnt);
   }
 
   @Test
+  public void testReadAppsIsRelatedTo() throws Exception {
+    TimelineFilterList irt = new TimelineFilterList(Operator.OR);
+    irt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    irt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt, null, null, null,
+        null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(2, entities.size());
+    int isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222") &&
+          !timelineEntity.getId().equals("application_1111111111_3333")) {
+        Assert.fail("Entity ids' should have been application_1111111111_2222"
+            + " and application_1111111111_3333");
+      }
+    }
+    assertEquals(3, isRelatedToCnt);
+
+    TimelineFilterList irt1 = new TimelineFilterList();
+    irt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    irt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt1, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entity id should have been application_1111111111_4444");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt2 = new TimelineFilterList(Operator.OR);
+    irt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    irt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt2, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222") &&
+          !timelineEntity.getId().equals("application_1111111111_3333")) {
+        Assert.fail("Entity ids' should have been application_1111111111_2222"
+            + " and application_1111111111_3333");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt3 = new TimelineFilterList();
+    irt3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt3, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_3333")) {
+        Assert.fail("Entity id should have been application_1111111111_3333");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+
+    TimelineFilterList irt4 = new TimelineFilterList();
+    irt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    irt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_task",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt4, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList irt5 = new TimelineFilterList();
+    irt5.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto7"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt5, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task",
+        new HashSet<Object>(Arrays.asList("relatedto1"))));
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_task",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task2",
+        new HashSet<Object>(Arrays.asList("relatedto4"))));
+    TimelineFilterList irt6 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, irt6, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    isRelatedToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_3333")) {
+        Assert.fail("Entity id should have been application_1111111111_3333");
+      }
+    }
+    assertEquals(0, isRelatedToCnt);
+  }
+
+
+  @Test
+  public void testReadAppsRelatesTo() throws Exception {
+    TimelineFilterList rt = new TimelineFilterList(Operator.OR);
+    rt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    rt.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(2, entities.size());
+    int relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222") &&
+          !timelineEntity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entity ids' should have been application_1111111111_2222"
+            + " and application_1111111111_4444");
+      }
+    }
+    assertEquals(3, relatesToCnt);
+
+    TimelineFilterList rt1 = new TimelineFilterList();
+    rt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    rt1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt1, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_3333")) {
+        Assert.fail("Entity id should have been application_1111111111_3333");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt2 = new TimelineFilterList(Operator.OR);
+    rt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    rt2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt2, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(2, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222") &&
+          !timelineEntity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entity ids' should have been application_1111111111_2222"
+            + " and application_1111111111_4444");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt3 = new TimelineFilterList();
+    rt3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1", "relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt3, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222")) {
+        Assert.fail("Entity id should have been application_1111111111_2222");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList rt4 = new TimelineFilterList();
+    rt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    rt4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_container",
+        new HashSet<Object>(Arrays.asList("relatesto5"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt4, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList rt5 = new TimelineFilterList();
+    rt5.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatedto1", "relatesto8"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt5, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(0, entities.size());
+
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    list1.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "dummy_container",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList rt6 = new TimelineFilterList(Operator.OR, list1, list2);
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt6, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_2222")) {
+        Assert.fail("Entity id should have been application_1111111111_2222");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+
+    TimelineFilterList list3 = new TimelineFilterList();
+    list3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    list3.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList list4 = new TimelineFilterList();
+    list4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto1"))));
+    list4.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto2"))));
+    TimelineFilterList combinedList =
+        new TimelineFilterList(Operator.OR, list3, list4);
+    TimelineFilterList rt7 = new TimelineFilterList(Operator.AND, combinedList,
+        new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "container",
+        new HashSet<Object>(Arrays.asList("relatesto3"))));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, rt7, null, null, null, null,
+        null),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_3333")) {
+        Assert.fail("Entity id should have been application_1111111111_3333");
+      }
+    }
+    assertEquals(0, relatesToCnt);
+  }
+
+  @Test
+  public void testReadAppsRelationsAndEventFiltersDefaultView()
+      throws Exception {
+    TimelineFilterList eventFilter = new TimelineFilterList();
+    eventFilter.addFilter(new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL,
+        "end_event"));
+    TimelineFilterList relatesTo = new TimelineFilterList(Operator.OR);
+    relatesTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container2",
+        new HashSet<Object>(Arrays.asList("relatesto7"))));
+    relatesTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "container1",
+        new HashSet<Object>(Arrays.asList("relatesto4"))));
+    TimelineFilterList isRelatedTo = new TimelineFilterList();
+    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto3"))));
+    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
+        TimelineCompareOp.NOT_EQUAL, "task1",
+        new HashSet<Object>(Arrays.asList("relatedto5"))));
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, relatesTo, isRelatedTo,
+        null, null, null, eventFilter),
+        new TimelineDataToRetrieve());
+    assertEquals(1, entities.size());
+    int eventCnt = 0;
+    int isRelatedToCnt = 0;
+    int relatesToCnt = 0;
+    for (TimelineEntity timelineEntity : entities) {
+      eventCnt += timelineEntity.getEvents().size();
+      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
+      relatesToCnt += timelineEntity.getRelatesToEntities().size();
+      if (!timelineEntity.getId().equals("application_1111111111_4444")) {
+        Assert.fail("Entity id should have been application_1111111111_4444");
+      }
+    }
+    assertEquals(0, eventCnt);
+    assertEquals(0, isRelatedToCnt);
+    assertEquals(0, relatesToCnt);
+  }
+
+  @Test
+  public void testReadAppsConfigFilters() throws Exception {
+    TimelineFilterList list1 = new TimelineFilterList();
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
+    list1.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param2", "value2"));
+    TimelineFilterList list2 = new TimelineFilterList();
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "cfg_param1", "value3"));
+    list2.addFilter(new TimelineKeyValueFilter(
+        TimelineCompareOp.EQUAL, "config_param2", "value2"));
+    TimelineFilterList confFilterList =
+        new TimelineFilterList(Operator.OR, list1, list2);
+    Set<TimelineEntity> entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(2, entities.size());
+    int cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(5, cfgCnt);
+
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
+    assertEquals(2, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntity entity : entities) {
+      cfgCnt += entity.getConfigs().size();
+    }
+    assertEquals(5, cfgCnt);
+
+    TimelineFilterList confFilterList1 = new TimelineFilterList(
+        new TimelineKeyValueFilter(
+        TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"));
+    entities = reader.getEntities(
+        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
+        1002345678919L, null, TimelineEntityType.YARN_APPLICATION.toString(),
+        null),
+        new TimelineEntityFilters(null, null, null, null, null, null,
+        confFilterList1, null, null),
+        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
+    assertEquals(1, entities.size());
+    cfgCnt = 0;
+    for (TimelineEntit

<TRUNCATED>

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


[43/50] [abbrv] hadoop git commit: YARN-5052 Update timeline service v2 documentation to capture information about filters (Varun Saxena via Vrushali C)

Posted by vr...@apache.org.
YARN-5052 Update timeline service v2 documentation to capture information about filters (Varun Saxena via Vrushali C)


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

Branch: refs/heads/YARN-2928
Commit: 6c22a6ac7dc0939717dbf0163b26453f50857400
Parents: 695e7ed
Author: Vrushali Channapattan <vr...@apache.org>
Authored: Fri Jun 10 13:42:24 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:14 2016 -0700

----------------------------------------------------------------------
 .../src/site/markdown/Compatibility.md          |   1 +
 hadoop-project/src/site/site.xml                |   2 +
 .../src/site/markdown/TimelineServiceV2.md      | 696 +++++++++++++++++--
 3 files changed, 633 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c22a6ac/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
index c275518..d7827b5 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Compatibility.md
@@ -128,6 +128,7 @@ REST API compatibility corresponds to both the request (URLs) and responses to e
 * [MR Application Master](../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredAppMasterRest.html)
 * [History Server](../../hadoop-mapreduce-client/hadoop-mapreduce-client-hs/HistoryServerRest.html)
 * [Timeline Server v1 REST API](../../hadoop-yarn/hadoop-yarn-site/TimelineServer.html)
+* [Timeline Service v2 REST API](../../hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html)
 
 #### Policy
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c22a6ac/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index a89a220..c74c55b 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -126,6 +126,7 @@
       <item name="Node Labels" href="hadoop-yarn/hadoop-yarn-site/NodeLabel.html"/>
       <item name="Web Application Proxy" href="hadoop-yarn/hadoop-yarn-site/WebApplicationProxy.html"/>
       <item name="Timeline Server" href="hadoop-yarn/hadoop-yarn-site/TimelineServer.html"/>
+      <item name="Timeline Service V.2" href="hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html"/>
       <item name="Writing YARN Applications" href="hadoop-yarn/hadoop-yarn-site/WritingYarnApplications.html"/>
       <item name="YARN Application Security" href="hadoop-yarn/hadoop-yarn-site/YarnApplicationSecurity.html"/>
       <item name="NodeManager" href="hadoop-yarn/hadoop-yarn-site/NodeManager.html"/>
@@ -141,6 +142,7 @@
       <item name="Resource Manager" href="hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html"/>
       <item name="Node Manager" href="hadoop-yarn/hadoop-yarn-site/NodeManagerRest.html"/>
       <item name="Timeline Server" href="hadoop-yarn/hadoop-yarn-site/TimelineServer.html#Timeline_Server_REST_API_v1"/>
+      <item name="Timeline Service V.2" href="hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html#Timeline_Service_REST_API_v2"/>
     </menu>
     
     <menu name="Hadoop Compatible File Systems" inherit="top">

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6c22a6ac/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
index 90db556..6e151c9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/TimelineServiceV2.md
@@ -24,6 +24,14 @@ The YARN Timeline Service v.2
     * [Enabling the Timeline Service v.2](#Enabling_Timeline_Service_v2)
 * [Publishing of application specific data](#Publishing_of_application_specific_data)
 * [Timeline Service v.2 REST API](#Timeline_Service_REST_API_v2)
+    * [Query Flows](#REST_API_LIST_FLOWS)
+    * [Query Flow Runs](#REST_API_LIST_FLOW_RUNS)
+    * [Query Flow Run](#REST_API_LIST_FLOW_RUN)
+    * [Query Apps for a Flow](#REST_API_LIST_FLOW_APPS)
+    * [Query Apps for a Flow Run](#REST_API_LIST_FLOWRUN_APPS)
+    * [Query App](#REST_API_LIST_APP)
+    * [Query Generic Entities](#REST_API_LIST_ENTITIES)
+    * [Query Generic Entity](#REST_API_LIST_ENTITY)
 
 #<a name="Overview"></a>Overview
 
@@ -78,11 +86,11 @@ The following diagram illustrates the design at a high level.
 ### <a name="Current_Status"></a>Current Status and Future Plans
 
 YARN Timeline Service v.2 is currently in alpha. It is very much work in progress, and many things
-can and will change rapidly. Users should enable Timeline Service v.2 only on a test or
+can and will change rapidly. Users must enable Timeline Service v.2 only on a test or
 experimental cluster to test the feature.
 
-A complete end-to-end flow of writes and reads should be functional, with Apache HBase as the
-backend. You should be able to start generating data. When enabled, all YARN-generic events are
+A complete end-to-end flow of writes and reads must be functional, with Apache HBase as the
+backend. You must be able to start generating data. When enabled, all YARN-generic events are
 published as well as YARN system metrics such as CPU and memory. Furthermore, some applications
 including Distributed Shell and MapReduce write per-framework data to YARN Timeline Service v.2.
 
@@ -97,7 +105,7 @@ resource manager also has its dedicated in-process collector. The reader is curr
 instance. Currently, it is not possible to write to Timeline Service outside the context of a YARN
 application (i.e. no off-cluster client).
 
-When YARN Timeline Service v.2 is disabled, one should expect no functional or performance impact
+When YARN Timeline Service v.2 is disabled, one must expect no functional or performance impact
 on any other existing functionality.
 
 The work to make it production-ready continues. Some key items include
@@ -125,9 +133,9 @@ New configuration parameters that are introduced with v.2 are marked bold.
 | Configuration Property | Description |
 |:---- |:---- |
 | `yarn.timeline-service.enabled` | Indicate to clients whether Timeline service is enabled or not. If enabled, the `TimelineClient` library used by applications will post entities and events to the Timeline server. Defaults to `false`. |
-| `yarn.timeline-service.version` | 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 (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, the results may vary. Defaults to `1.0f`. |
-| **`yarn.timeline-service.writer.class`** | The class for the backend storage writer. Defaults to a filesystem storage writer, therefore it should be overridden. |
-| **`yarn.timeline-service.reader.class`** | The class for the backend storage reader. Defaults to a filesystem storage reader, therefore it should be overridden. |
+| `yarn.timeline-service.version` | 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 must 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 must 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, the results may vary. Defaults to `1.0f`. |
+| **`yarn.timeline-service.writer.class`** | The class for the backend storage writer. Defaults to a filesystem storage writer, therefore it must be overridden. |
+| **`yarn.timeline-service.reader.class`** | The class for the backend storage reader. Defaults to a filesystem storage reader, therefore it must be overridden. |
 | **`yarn.system-metrics-publisher.enabled`** | The setting that controls whether yarn system metrics is published on the Timeline service or not by RM And NM. Defaults to `false`. |
 | **`yarn.rm.system-metrics-publisher.emit-container-events`** | The setting that controls whether yarn container metrics is published to the timeline server or not by RM. This configuration setting is for ATS V2. Defaults to `false`. |
 
@@ -275,7 +283,7 @@ on `TimelineClient` suitable for writing to the Timeline Service v.2 are clearly
 they use the v.2 types as arguments.
 
 Timeline Service v.2 `putEntities` methods come in 2 varieties: `putEntities` and
-`putEntitiesAsync`. The former is a blocking operation which should be used for writing more
+`putEntitiesAsync`. The former is a blocking operation which must be used for writing more
 critical data (e.g. lifecycle events). The latter is a non-blocking operation. Note that neither
 has a return value.
 
@@ -324,7 +332,7 @@ Service, while an off-cluster MapReduce job submitter cannot.
 
 You can create and publish your own entities, events, and metrics as with previous versions.
 
-Application frameworks should set the "flow context" whenever possible in order to take advantage
+Application frameworks must set the "flow context" whenever possible in order to take advantage
 of the flow support Timeline Service v.2 provides. The flow context consists of the following:
 
 * Flow name: a string that identifies the high-level flow (e.g. "distributed grep" or any
@@ -377,16 +385,40 @@ Returns a JSON object describing the service instance and version information.
       "hadoop-version-built-on":"2016-04-11T23:14Z"
     }
 
-### Request Examples
+The following shows the supported queries on the REST API.
 
-The following shows some of the supported queries on the REST API. For example, to get the most
-recent flow activities,
+## <a name="REST_API_LIST_FLOWS"></a>Query Flows
 
-HTTP request:
+With Query Flows API, you can retrieve a list of active flows that had runs most recently.
+If REST endpoint without cluster name is used, cluster specified by the configuration yarn.resourcemanager.cluster-id in yarn-site.xml is taken.
+If none of the flows match the predicates, an empty list will be returned.
+
+### HTTP request:
 
     GET /ws/v2/timeline/clusters/{cluster name}/flows/
 
-Response:
+    or
+
+    GET /ws/v2/timeline/flows/
+
+### Query Parameters Supported:
+
+1. `limit` - If specified, defines the number of flows to return. The maximum possible value for limit
+  is maximum value of Long. If it is not specified or has a value less than 0, then limit will be
+  considered as 100.
+1. `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.<br/>
+  _For example_ :<br/>
+  "daterange=20150711" returns flows active on 20150711.<br/>
+  "daterange=20150711-20150714" returns flows active between these 2 dates.<br/>
+  "daterange=20150711-" returns flows active on and after 20150711.<br/>
+  "daterange=-20150711" returns flows active on and before 20150711.<br/>
+
+### Example JSON Response:
 
     [
       {
@@ -439,15 +471,49 @@ Response:
       }
     ]
 
-It returns the flows that had runs (specific instances of the flows) most recently.
+### Response Codes
+
+1. If successful, a HTTP 200 (OK) response is returned.
+1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
+1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
+
+
+## <a name="REST_API_LIST_FLOW_RUNS"></a>Query Flow Runs
+
+With Query Flow Runs API, you can drill further down to get the runs (specific instances) of a given flow.
+This returns the most recent runs that belong to the given flow. If REST endpoint without cluster name is used,
+cluster specified by the configuration yarn.resourcemanager.cluster-id in yarn-site.xml is taken.
+If none of the flow runs match the predicates, an empty list will be returned.
 
-You can drill further down to get the runs (specific instances) of a given flow.
+### HTTP request:
 
-HTTP request:
+    GET /ws/v2/timeline/clusters/{cluster name}/users/{user name}/flows/{flow name}/runs/
+
+    or
 
     GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs/
 
-Response:
+### Query Parameters Supported:
+
+1. `limit` - If specified, defines the number of flows to return. The maximum possible value for limit
+  is maximum value of Long. If it is not specified or has a value less than 0, then limit will be
+  considered as 100.
+1. `createdTimeStart` - If specified, then only flow runs started after this timestamp are returned.
+1. `createdTimeEnd` -  If specified, then only flow runs started before this timestamp are returned.
+1. `metricsToRetrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response.
+  metricsToRetrieve can be an expression of the form :<br/>
+  (&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for the
+  simple expression. Alternatively, expressions can be of the form :<br/>
+  !(&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.<br/>
+  If metricsToRetrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param
+  or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `fields` - Specifies which fields to retrieve. For querying flow runs, only `ALL` or `METRICS` are valid fields.
+  Other fields will lead to HTTP 400 (Bad Request) response. If not specified, in response, id, type, createdtime and info fields
+  will be returned.
+
+### Example JSON Response:
 
     [
       {
@@ -484,56 +550,291 @@ Response:
       }
     ]
 
-This returns the most recent runs that belong to the given flow.
+### Response Codes
+
+1. If successful, a HTTP 200 (OK) response is returned.
+1. If any problem occurs in parsing request or if an invalid field is specified in fields query param, HTTP 400 (Bad Request) is returned.
+1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
+
+
+## <a name="REST_API_LIST_FLOW_RUN"></a>Query Flow Run
+
+With this API, you can query a specific flow run identified by cluster, user, flow name and run id.If REST endpoint without cluster name is
+used, cluster specified by the configuration yarn.resourcemanager.cluster-id in yarn-site.xml is taken. Metrics are returned by default while
+querying individual flow run.
 
-You can provide a `limit` query parameter to limit the number of entries that returned in a query.
-If you want to limit the number of flow runs in the above query, you can do the following:
+### HTTP request:
 
-HTTP request:
+    GET /ws/v2/timeline/clusters/{cluster name}/users/{user name}/flows/{flow name}/runs/{run id}
 
-    GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs?limit=1
+    or
 
-Response:
+    GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs/{run id}
+
+### Query Parameters Supported:
+
+1. `metricsToRetrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response.
+  metricsToRetrieve can be an expression of the form :<br/>
+  (&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for the
+  simple expression. Alternatively, expressions can be of the form :<br/>
+  !(&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+
+### Example JSON Response:
+
+    {
+      "metrics": [
+        {
+          "type": "SINGLE_VALUE",
+          "id": "org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter:BYTES_READ",
+          "aggregationOp": "NOP",
+          "values": {
+            "1465246377261": 118
+          }
+        },
+        {
+          "type": "SINGLE_VALUE",
+          "id": "org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter:BYTES_WRITTEN",
+          "aggregationOp": "NOP",
+          "values": {
+            "1465246377261": 97
+          }
+        }
+      ],
+      "events": [],
+      "id": "varun@QuasiMonteCarlo/1465246348599",
+      "type": "YARN_FLOW_RUN",
+      "createdtime": 1465246348599,
+      "isrelatedto": {},
+      "info": {
+        "UID":"yarn-cluster!varun!QuasiMonteCarlo!1465246348599",
+        "SYSTEM_INFO_FLOW_RUN_END_TIME":1465246378051,
+        "SYSTEM_INFO_FLOW_NAME":"QuasiMonteCarlo",
+        "SYSTEM_INFO_USER":"varun",
+        "SYSTEM_INFO_FLOW_RUN_ID":1465246348599
+      },
+      "relatesto": {}
+    }
+
+### Response Codes
+
+1. If successful, a HTTP 200(OK) response is returned.
+1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
+1. If flow run for the given flow run id cannot be found, HTTP 404 (Not Found) is returned.
+1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
+
+## <a name="REST_API_LIST_FLOW_APPS"></a>Query Apps for a flow
+
+With this API, you can query all the YARN applications that are part of a specific flow.If REST endpoint without cluster name
+is used, cluster specified by the configuration yarn.resourcemanager.cluster-id in yarn-site.xml is taken. If number of matching applications are
+more than the limit, the most recent apps up to the limit will be returned. If none of the apps match the predicates, an empty list will be returned.
+
+### HTTP request:
+
+    GET /ws/v2/timeline/clusters/{cluster name}/users/{user name}/flows/{flow name}/apps
+
+    or
+
+    GET /ws/v2/timeline/users/{user name}/flows/{flow name}/apps
+
+### Query Parameters Supported:
+
+1. `limit` - If specified, defines the number of applications to return. The maximum possible value for limit
+  is maximum value of Long. If it is not specified or has a value less than 0, then limit will be
+  considered as 100.
+1. `createdTimeStart` - If specified, then only applications created after this timestamp are returned.
+1. `createdTimeEnd` -  If specified, then only applications created before this timestamp are returned.
+1. `relatesTo` - If specified, matched applications must relate to or not relate to given entities associated with a entity type.
+  relatesto is represented as an expression of the form :<br/>
+  "(&lt;entitytype&gt;:&lt;entityid&gt;:&lt;entityid&gt;...,&lt;entitytype&gt;:&lt;entityid&gt;:&lt;entityid&gt;...) &lt;op&gt; !(&lt;entitytype&gt;:&lt;entityid&gt;:&lt;entityid&gt;...,&lt;entitytype&gt;:&lt;entityid&gt;:&lt;entityid&gt;...)".<br/>
+  If relatesTo expression has entity type - entity id(s) relations specified within enclosing brackets proceeding "!", this means apps with
+  these relations in its relatesTo field, will not be returned. For expressions or subexpressions without "!", all apps which have the specified
+  relations in its relatesTo field, will be returned. "op" is a logical operator and can be either AND or OR. entity type can be followed by any number
+  of entity id(s). And we can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.<br/>
+  _For example_ : relatesTo can be "(((type1:id1:id2:id3,type3:id9) AND !(type2:id7:id8)) OR (type1:id4))".<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `isRelatedTo` - If specified, matched applications must be related to or not related to given entities associated with a entity type. isRelatedTo is
+  represented in the same form as relatesTo.
+1. `infofilters` - If specified, matched applications must have exact matches to the given info key and must be either equal or not equal to
+  given value. The info key is a string but value can be any object. infofilters are represented as an expression of the form :<br/>
+  "(&lt;key&gt; &lt;compareop&gt; &lt;value&gt;) &lt;op&gt; (&lt;key&gt; &lt;compareop&gt; &lt;value&gt;)".<br/>
+  Here op can be either of AND or OR. And compareop can be either of "eq", "ne" or "ene".<br/>
+  "eq" means equals, "ne" means not equals and existence of key is not required for a match and "ene" means not equals but existence of key is
+  required. We can combine any number of ANDs' and ORs' to create complex expressions.  Brackets can be used to club expressions together.<br/>
+  _For example_ : infofilters can be "(((infokey1 eq value1) AND (infokey2 ne value1)) OR (infokey1 ene value3))".<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `conffilters` - If specified, matched applications must have exact matches to the given config name and must be either equal or not equal
+  to the given config value. Both the config name and value must be strings. conffilters are represented in the same form as infofilters.
+1. `metricfilters` - If specified, matched applications must have exact matches to the given metric and satisfy the specified relation with the
+  metric value. Metric id must be a string and and metric value must be an integral value.  metricfilters are represented as an expression of the form :<br/>
+  "(&lt;metricid&gt; &lt;compareop&gt; &lt;metricvalue&gt;) &lt;op&gt; (&lt;metricid&gt; &lt;compareop&gt; &lt;metricvalue&gt;)".<br/>
+  Here op can be either of AND or OR. And compareop can be either of "eq", "ne", "ene", "gt", "ge", "lt" and "le".<br/>
+  "eq" means equals, "ne" means not equals and existence of metric is not required for a match, "ene" means not equals but existence of metric is
+  required, "gt" means greater than, "ge" means greater than or equals, "lt" means less than and "le" means less than or equals. We can combine
+  any number of ANDs' and ORs' to create complex expressions.  Brackets can be used to club expressions together.<br/>
+  _For example_ : metricfilters can be "(((metric1 eq 50) AND (metric2 gt 40)) OR (metric1 lt 20))".<br/>
+  This in essence is an expression equivalent to "(metric1 == 50 AND metric2 &gt; 40) OR (metric1 &lt; 20)"<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `eventfilters` - If specified, matched applications must contain or not contain the given events depending on the expression. eventfilters is
+  represented as an expression of the form :<br/>
+  "(&lt;eventid&gt;,&lt;eventid&gt;) &lt;op&gt; !(&lt;eventid&gt;,&lt;eventid&gt;,&lt;eventid&gt;)".<br/>
+  Here, "!" means none of the comma-separated list of events within the enclosed brackets proceeding "!" must exist for a match to occur.
+  If "!" is not specified, the specified events within the enclosed brackets must exist. op is a logical operator and can be either AND or OR.
+  We can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.<br/>
+  _For example_ : eventfilters can be "(((event1,event2) AND !(event4)) OR (event3,event7,event5))".<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `metricsToRetrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response.
+  metricsToRetrieve can be an expression of the form :<br/>
+  (&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for
+  the simple expression. Alternatively, expressions can be of the form:<br/>
+  !(&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.<br/>
+  If metricsToRetrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param
+  or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `confsToRetrieve` - If specified, defines which configs to retrieve or which ones not to retrieve and send back in response.
+  confsToRetrieve can be an expression of the form :<br/>
+  (&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;...)<br/>
+  This specifies a comma separated list of config name prefixes. Only configs matching any of the prefixes will be retrieved. Brackets are optional for
+  the simple expression. Alternatively, expressions can be of the form:<br/>
+  !(&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;...)<br/>
+  This specifies a comma separated list of config name prefixes. Only configs not matching any of the prefixes will be retrieved.<br/>
+  If confsToRetrieve is specified, configs will be retrieved irrespective of whether `CONFIGS` is specified in fields query param
+  or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `fields` - Specifies which fields to retrieve. Possible values for fields can be `EVENTS`, `INFO`, `CONFIGS`, `METRICS`, `RELATES_TO`,
+ `IS_RELATED_TO` and `ALL`. All fields will be retrieved if `ALL` is specified. Multiple fields can be specified as a comma-separated list.
+  If fields is not specified, in response, app id, type (equivalent to YARN_APPLICATION), app createdtime and UID in info field will be returned.
+1. `metricsLimit` - If specified, defines the number of metrics to return. Considered only if fields contains METRICS/ALL
+  or metricsToRetrieve is specified. Ignored otherwise. The maximum possible value for metricsLimit can be maximum value of
+  Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricsLimit will be
+  considered as 1 i.e. latest single value of metric(s) will be returned.
+
+### Example JSON Response:
 
     [
       {
-        "metrics": [],
-        "events": [],
-        "id": "sjlee@ds-date/1460420587974",
-        "type": "YARN_FLOW_RUN",
-        "createdtime": 1460420587974,
+        "metrics": [ ],
+        "events": [ ],
+        "type": "YARN_APPLICATION",
+        "id": "application_1465246237936_0001",
+        "createdtime": 1465246348599,
+        "isrelatedto": { },
+        "configs": { },
         "info": {
-          "UID": "test-cluster!sjlee!ds-date!1460420587974",
-          "SYSTEM_INFO_FLOW_RUN_ID": 1460420587974,
-          "SYSTEM_INFO_FLOW_NAME": "ds-date",
-          "SYSTEM_INFO_FLOW_RUN_END_TIME": 1460420595198,
-          "SYSTEM_INFO_USER": "sjlee"
+          "UID": "yarn-cluster!application_1465246237936_0001"
         },
-        "isrelatedto": {},
-        "relatesto": {}
+        "relatesto": { }
+      },
+      {
+        "metrics": [ ],
+        "events": [ ],
+        "type": "YARN_APPLICATION",
+        "id": "application_1464983628730_0005",
+        "createdtime": 1465033881959,
+        "isrelatedto": { },
+        "configs": { },
+        "info": {
+          "UID": "yarn-cluster!application_1464983628730_0005"
+        },
+        "relatesto": { }
       }
     ]
 
-Most queries in the v.2 REST API support the following query parameters:
+### Response Codes
+
+1. If successful, a HTTP 200 (OK) response is returned.
+1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
+1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
 
-* `limit`
-* `createdtimestart`
-* `createdtimeend`
-* `relatesto`
-* `isrelatedto`
-* `infofilters`
-* `conffilters`
-* `metricfilters`
-* `eventfilters`
-* `fields`
+## <a name="REST_API_LIST_FLOWRUN_APPS"></a>Query Apps for a flow run
 
-Given a flow run, you can query all the YARN applications that are part of that flow run:
+With this API, you can query all the YARN applications that are part of a specific flow run.If REST endpoint without cluster name
+is used, cluster specified by the configuration yarn.resourcemanager.cluster-id in yarn-site.xml is taken. If number of matching applications are
+more than the limit, the most recent apps up to the limit will be returned. If none of the apps match the predicates, an empty list will be returned.
 
-HTTP request:
+### HTTP request:
+
+    GET /ws/v2/timeline/clusters/{cluster name}/users/{user name}/flows/{flow name}/runs/{run id}/apps
+
+    or
 
     GET /ws/v2/timeline/users/{user name}/flows/{flow name}/runs/{run id}/apps/
 
-Response:
+### Query Parameters Supported:
+
+1. `limit` - If specified, defines the number of applications to return. The maximum possible value for limit
+  is maximum value of Long. If it is not specified or has a value less than 0, then limit will be
+  considered as 100.
+1. `createdTimeStart` - If specified, then only applications created after this timestamp are returned.
+1. `createdTimeEnd` -  If specified, then only applications created before this timestamp are returned.
+1. `relatesTo` - If specified, matched applications must relate to or not relate to given entities associated with a entity type.
+  relatesto is represented as an expression of the form :<br/>
+  "(&lt;entitytype&gt;:&lt;entityid&gt;:&lt;entityid&gt;...,&lt;entitytype&gt;:&lt;entityid&gt;:&lt;entityid&gt;...) &lt;op&gt; !(&lt;entitytype&gt;:&lt;entityid&gt;:&lt;entityid&gt;...,&lt;entitytype&gt;:&lt;entityid&gt;:&lt;entityid&gt;...)".<br/>
+  If relatesTo expression has entity type - entity id(s) relations specified within enclosing brackets proceeding "!", this means apps with
+  these relations in its relatesTo field, will not be returned. For expressions or subexpressions without "!", all apps which have the specified
+  relations in its relatesTo field, will be returned. "op" is a logical operator and can be either AND or OR. entity type can be followed by any number
+  of entity id(s). And we can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.<br/>
+  _For example_ : relatesTo can be "(((type1:id1:id2:id3,type3:id9) AND !(type2:id7:id8)) OR (type1:id4))".<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `isRelatedTo` - If specified, matched applications must be related to or not related to given entities associated with a entity type. isRelatedTo is
+  represented in the same form as relatesTo.
+1. `infofilters` - If specified, matched applications must have exact matches to the given info key and must be either equal or not equal to
+  given value. The info key is a string but value can be any object. infofilters are represented as an expression of the form :<br/>
+  "(&lt;key&gt; &lt;compareop&gt; &lt;value&gt;) &lt;op&gt; (&lt;key&gt; &lt;compareop&gt; &lt;value&gt;)".<br/>
+  Here op can be either of AND or OR. And compareop can be either of "eq", "ne" or "ene".<br/>
+  "eq" means equals, "ne" means not equals and existence of key is not required for a match and "ene" means not equals but existence of key is
+  required. We can combine any number of ANDs' and ORs' to create complex expressions.  Brackets can be used to club expressions together.<br/>
+  _For example_ : infofilters can be "(((infokey1 eq value1) AND (infokey2 ne value1)) OR (infokey1 ene value3))".<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `conffilters` - If specified, matched applications must have exact matches to the given config name and must be either equal or not equal
+  to the given config value. Both the config name and value must be strings. conffilters are represented in the same form as infofilters.
+1. `metricfilters` - If specified, matched applications must have exact matches to the given metric and satisfy the specified relation with the
+  metric value. Metric id must be a string and and metric value must be an integral value.  metricfilters are represented as an expression of the form :<br/>
+  "(&lt;metricid&gt; &lt;compareop&gt; &lt;metricvalue&gt;) &lt;op&gt; (&lt;metricid&gt; &lt;compareop&gt; &lt;metricvalue&gt;)".<br/>
+  Here op can be either of AND or OR. And compareop can be either of "eq", "ne", "ene", "gt", "ge", "lt" and "le".<br/>
+  "eq" means equals, "ne" means not equals and existence of metric is not required for a match, "ene" means not equals but existence of metric is
+  required, "gt" means greater than, "ge" means greater than or equals, "lt" means less than and "le" means less than or equals. We can combine
+  any number of ANDs' and ORs' to create complex expressions.  Brackets can be used to club expressions together.<br/>
+  _For example_ : metricfilters can be "(((metric1 eq 50) AND (metric2 gt 40)) OR (metric1 lt 20))".<br/>
+  This in essence is an expression equivalent to "(metric1 == 50 AND metric2 &gt; 40) OR (metric1 &lt; 20)"<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `eventfilters` - If specified, matched applications must contain or not contain the given events depending on the expression. eventfilters is
+  represented as an expression of the form :<br/>
+  "(&lt;eventid&gt;,&lt;eventid&gt;) &lt;op&gt; !(&lt;eventid&gt;,&lt;eventid&gt;,&lt;eventid&gt;)".<br/>
+  Here, "!" means none of the comma-separated list of events within the enclosed brackets proceeding "!" must exist for a match to occur.
+  If "!" is not specified, the specified events within the enclosed brackets must exist. op is a logical operator and can be either AND or OR.
+  We can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.<br/>
+  _For example_ : eventfilters can be "(((event1,event2) AND !(event4)) OR (event3,event7,event5))".<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `metricsToRetrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response.
+  metricsToRetrieve can be an expression of the form :<br/>
+  (&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for the
+  simple expression. Alternatively, expressions can be of the form :<br/>
+  !(&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.<br/>
+  If metricsToRetrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param
+  or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `confsToRetrieve` - If specified, defines which configs to retrieve or which ones not to retrieve and send back in response.
+  confsToRetrieve can be an expression of the form :<br/>
+  (&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;...)<br/>
+  This specifies a comma separated list of config name prefixes. Only configs matching any of the prefixes will be retrieved. Brackets are optional for the
+  simple expression. Alternatively, expressions can be of the form :<br/>
+  !(&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;...)<br/>
+  This specifies a comma separated list of config name prefixes. Only configs not matching any of the prefixes will be retrieved.<br/>
+  If confsToRetrieve is specified, configs will be retrieved irrespective of whether `CONFIGS` is specified in fields query param
+  or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `fields` - Specifies which fields to retrieve. Possible values for fields can be `EVENTS`, `INFO`, `CONFIGS`, `METRICS`, `RELATES_TO`,
+ `IS_RELATED_TO` and `ALL`. All fields will be retrieved if `ALL` is specified. Multiple fields can be specified as a comma-separated list.
+  If fields is not specified, in response, app id, type (equivalent to YARN_APPLICATION), app createdtime and UID in info field will be returned.
+1. `metricsLimit` - If specified, defines the number of metrics to return. Considered only if fields contains METRICS/ALL
+  or metricsToRetrieve is specified. Ignored otherwise. The maximum possible value for metricsLimit can be maximum value of
+  Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricsLimit will be
+  considered as 1 i.e. latest single value of metric(s) will be returned.
+
+### Example JSON Response:
 
     [
       {
@@ -541,9 +842,9 @@ Response:
         "events": [],
         "id": "application_1460419579913_0002",
         "type": "YARN_APPLICATION",
-        "createdtime": 0,
+        "createdtime": 1460419580171,
         "info": {
-          "UID": "test-cluster!sjlee!ds-date!1460420587974!application_1460419579913_0002"
+          "UID": "test-cluster!sjlee!ds-date!1460419580171!application_1460419579913_0002"
         },
         "configs": {},
         "isrelatedto": {},
@@ -551,26 +852,289 @@ Response:
       }
     ]
 
-You can also provide per-framework entity type to query for them. For example,
-
-HTTP request:
+### Response Codes
+
+1. If successful, a HTTP 200 (OK) response is returned.
+1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
+1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
+
+
+## <a name="REST_API_LIST_APP"></a>Query app
+
+With this API, you can query a single YARN application identified by cluster and application ID. If REST endpoint without cluster name
+is used, cluster specified by the configuration yarn.resourcemanager.cluster-id in yarn-site.xml is taken. Flow context information i.e.
+user, flow name and run id are not mandatory but if specified in query param can preclude the need for an additional operation to fetch
+flow context information based on cluster and app id.
+
+### HTTP request:
+
+    GET /ws/v2/timeline/clusters/{cluster name}/apps/{app id}
+
+    or
+
+    GET /ws/v2/timeline/apps/{app id}
+
+### Query Parameters Supported:
+
+1. `userid` -  If specified, only applications belonging to this user will be returned. This query param must be specified along with flowname and flowrunid query params, otherwise it will be ignored.
+  If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query.
+1. `flowname` - Only applications belonging to this flowname will be returned. This query param must be specified along with userid and flowrunid query params, otherwise it will be ignored.
+  If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query.
+1. `flowrunid` - Only applications belonging to this flow run id will be returned. This query param must be specified along with userid and flowname query params, otherwise it will be ignored.
+  If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query.
+1. `metricsToRetrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response.
+  metricsToRetrieve can be an expression of the form :<br/>
+  (&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for the
+  simple expression. Alternatively, expressions can be of the form :<br/>
+  !(&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.<br/>
+  If metricsToRetrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param
+  or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `confsToRetrieve` - If specified, defines which configs to retrieve or which ones not to retrieve and send back in response.
+  confsToRetrieve can be an expression of the form :<br/>
+  (&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;...)<br/>
+  This specifies a comma separated list of config name prefixes. Only configs matching any of the prefixes will be retrieved. Brackets are optional for the
+  simple expression. Alternatively, expressions can be of the form :<br/>
+  !(&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;...)<br/>
+  This specifies a comma separated list of config name prefixes. Only configs not matching any of the prefixes will be retrieved.<br/>
+  If confsToRetrieve is specified, configs will be retrieved irrespective of whether `CONFIGS` is specified in fields query param
+  or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `fields` - Specifies which fields to retrieve. Possible values for fields can be `EVENTS`, `INFO`, `CONFIGS`, `METRICS`, `RELATES_TO`,
+ `IS_RELATED_TO` and `ALL`. All fields will be retrieved if `ALL` is specified. Multiple fields can be specified as a comma-separated list.
+  If fields is not specified, in response, app id, type (equivalent to YARN_APPLICATION), app createdtime and UID in info field will be returned.
+1. `metricsLimit` - If specified, defines the number of metrics to return. Considered only if fields contains METRICS/ALL
+  or metricsToRetrieve is specified. Ignored otherwise. The maximum possible value for metricsLimit can be maximum value of
+  Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricsLimit will be
+  considered as 1 i.e. latest single value of metric(s) will be returned.
+
+### Example JSON Response:
 
-    GET /ws/v2/timeline/clusters/{cluster name}/apps/{app id}/entities/DS_APP_ATTEMPT
+    {
+      "metrics": [],
+      "events": [],
+      "id": "application_1460419579913_0002",
+      "type": "YARN_APPLICATION",
+      "createdtime": 1460419580171,
+      "info": {
+        "UID": "test-cluster!sjlee!ds-date!1460419580171!application_1460419579913_0002"
+      },
+      "configs": {},
+      "isrelatedto": {},
+      "relatesto": {}
+    }
 
-Response:
+### Response Codes
+
+1. If successful, a HTTP 200(OK) response is returned.
+1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
+1. If flow context information cannot be retrieved or application for the given app id cannot be found, HTTP 404 (Not Found) is returned.
+1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
+
+## <a name="REST_API_LIST_ENTITIES"></a>Query generic entities
+
+With this API, you can query generic entities identified by cluster ID, application ID and per-framework entity type. If REST endpoint without cluster name
+is used, cluster specified by the configuration yarn.resourcemanager.cluster-id in yarn-site.xml is taken. Flow context information i.e.
+user, flow name and run id are not mandatory but if specified in query param can preclude the need for an additional operation to fetch
+flow context information based on cluster and app id.  If number of matching entities are more than the limit, the most recent entities up to the limit
+will be returned. This endpoint can be used to query containers, application attempts or any other generic entity which clients put into the backend.
+For instance, we can query containers by specifying entity type as `YARN_CONTAINER` and application attempts by specifying entity type as `YARN_APPLICATION_ATTEMPT`.
+If none of the entities match the predicates, an empty list will be returned.
+
+### HTTP request:
+
+    GET /ws/v2/timeline/clusters/{cluster name}/apps/{app id}/entities/{entity type}
+
+    or
+
+    GET /ws/v2/timeline/apps/{app id}/entities/{entity type}
+
+### Query Parameters Supported:
+
+1. `userid` -  If specified, only entities belonging to this user will be returned. This query param must be specified along with flowname and flowrunid query params, otherwise it will be ignored.
+  If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query.
+1. `flowname` - If specified, only entities belonging to this flowname will be returned. This query param must be specified along with userid and flowrunid query params, otherwise it will be ignored.
+  If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query.
+1. `flowrunid` - If specified, only entities belonging to this flow run id will be returned. This query param must be specified along with userid and flowname query params, otherwise it will be ignored.
+  If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query.
+1. `limit` - If specified, defines the number of entities to return. The maximum possible value for limit is maximum value of Long. If it is not specified
+  or has a value less than 0, then limit will be considered as 100.
+1. `createdTimeStart` - If specified, then only entities created after this timestamp are returned.
+1. `createdTimeEnd` -  If specified, then only entities created before this timestamp are returned.
+1. `relatesTo` - If specified, matched entities must relate to or not relate to given entities associated with a entity type.
+  relatesto is represented as an expression of the form :<br/>
+  "(&lt;entitytype&gt;:&lt;entityid&gt;:&lt;entityid&gt;...,&lt;entitytype&gt;:&lt;entityid&gt;:&lt;entityid&gt;...) &lt;op&gt; !(&lt;entitytype&gt;:&lt;entityid&gt;:&lt;entityid&gt;...,&lt;entitytype&gt;:&lt;entityid&gt;:&lt;entityid&gt;...)".<br/>
+  If relatesTo expression has entity type - entity id(s) relations specified within enclosing brackets proceeding "!", this means entities with
+  these relations in its relatesTo field, will not be returned. For expressions or subexpressions without "!", all entities which have the specified
+  relations in its relatesTo field, will be returned. "op" is a logical operator and can be either AND or OR. entity type can be followed by any number
+  of entity id(s). And we can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.<br/>
+  _For example_ : relatesTo can be "(((type1:id1:id2:id3,type3:id9) AND !(type2:id7:id8)) OR (type1:id4))".<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `isRelatedTo` - If specified, matched entities must be related to or not related to given entities associated with a entity type. isRelatedTo is
+  represented in the same form as relatesTo.
+1. `infofilters` - If specified, matched entities must have exact matches to the given info key and must be either equal or not equal to
+  given value. The info key is a string but value can be any object. infofilters are represented as an expression of the form :<br/>
+  "(&lt;key&gt; &lt;compareop&gt; &lt;value&gt;) &lt;op&gt; (&lt;key&gt; &lt;compareop&gt; &lt;value&gt;)".<br/>
+  Here op can be either of AND or OR. And compareop can be either of "eq", "ne" or "ene".<br/>
+  "eq" means equals, "ne" means not equals and existence of key is not required for a match and "ene" means not equals but existence of key is
+  required. We can combine any number of ANDs' and ORs' to create complex expressions.  Brackets can be used to club expressions together.<br/>
+  _For example_ : infofilters can be "(((infokey1 eq value1) AND (infokey2 ne value1)) OR (infokey1 ene value3))".<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `conffilters` - If specified, matched entities must have exact matches to the given config name and must be either equal or not equal
+  to the given config value. Both the config name and value must be strings. conffilters are represented in the same form as infofilters.
+1. `metricfilters` - If specified, matched entities must have exact matches to the given metric and satisfy the specified relation with the
+  metric value. Metric id must be a string and and metric value must be an integral value.  metricfilters are represented as an expression of the form :<br/>
+  "(&lt;metricid&gt; &lt;compareop&gt; &lt;metricvalue&gt;) &lt;op&gt; (&lt;metricid&gt; &lt;compareop&gt; &lt;metricvalue&gt;)"<br/>
+  Here op can be either of AND or OR. And compareop can be either of "eq", "ne", "ene", "gt", "ge", "lt" and "le".<br/>
+  "eq" means equals, "ne" means not equals and existence of metric is not required for a match, "ene" means not equals but existence of metric is
+  required, "gt" means greater than, "ge" means greater than or equals, "lt" means less than and "le" means less than or equals. We can combine
+  any number of ANDs' and ORs' to create complex expressions.  Brackets can be used to club expressions together.<br/>
+  _For example_ : metricfilters can be "(((metric1 eq 50) AND (metric2 gt 40)) OR (metric1 lt 20))".<br/>
+  This in essence is an expression equivalent to "(metric1 == 50 AND metric2 &gt; 40) OR (metric1 &lt; 20)"<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `eventfilters` - If specified, matched entities must contain or not contain the given events depending on the expression. eventfilters is
+  represented as an expression of the form :<br/>
+  "(&lt;eventid&gt;,&lt;eventid&gt;) &lt;op&gt; !(&lt;eventid&gt;,&lt;eventid&gt;,&lt;eventid&gt;)".<br/>
+  Here, "!" means none of the comma-separated list of events within the enclosed brackets proceeding "!" must exist for a match to occur.
+  If "!" is not specified, the specified events within the enclosed brackets must exist. op is a logical operator and can be either AND or OR.
+  We can combine any number of ANDs' and ORs' to create complex expressions. Brackets can be used to club expressions together.<br/>
+  _For example_ : eventfilters can be "(((event1,event2) AND !(event4)) OR (event3,event7,event5))".<br/>
+  Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `metricsToRetrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response.
+  metricsToRetrieve can be an expression of the form :<br/>
+  (&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional for
+  the simple expression. Alternatively, expressions can be of the form:<br/>
+  !(&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.<br/>
+  If metricsToRetrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param
+  or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `confsToRetrieve` - If specified, defines which configs to retrieve or which ones not to retrieve and send back in response.
+  confsToRetrieve can be an expression of the form :<br/>
+  (&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;...)<br/>
+  This specifies a comma separated list of config name prefixes. Only configs matching any of the prefixes will be retrieved. Brackets are optional for
+  the simple expression. Alternatively, expressions can be of the form:<br/>
+  !(&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;...)<br/>
+  This specifies a comma separated list of config name prefixes. Only configs not matching any of the prefixes will be retrieved.<br/>
+  If confsToRetrieve is specified, configs will be retrieved irrespective of whether `CONFIGS` is specified in fields query param
+  or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `fields` - Specifies which fields to retrieve. Possible values for fields can be `EVENTS`, `INFO`, `CONFIGS`, `METRICS`, `RELATES_TO`,
+ `IS_RELATED_TO` and `ALL`. All fields will be retrieved if `ALL` is specified. Multiple fields can be specified as a comma-separated list.
+  If fields is not specified, in response, entity id, entity type, createdtime and UID in info field will be returned.
+1. `metricsLimit` - If specified, defines the number of metrics to return. Considered only if fields contains METRICS/ALL
+  or metricsToRetrieve is specified. Ignored otherwise. The maximum possible value for metricsLimit can be maximum value of
+  Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricsLimit will be
+  considered as 1 i.e. latest single value of metric(s) will be returned.
+
+### Example JSON Response:
 
     [
       {
-        "metrics": [],
-        "events": [],
-        "id": "appattempt_1460419579913_0002_000001",
-        "type": "DS_APP_ATTEMPT",
-        "createdtime": 0,
+        "metrics": [ ],
+        "events": [ ],
+        "type": "YARN_APPLICATION_ATTEMPT",
+        "id": "appattempt_1465246237936_0001_000001",
+        "createdtime": 1465246358873,
+        "isrelatedto": { },
+        "configs": { },
         "info": {
-          "UID": "test-cluster!application_1460419579913_0002!DS_APP_ATTEMPT!appattempt_1460419579913_0002_000001"
+          "UID": "yarn-cluster!application_1465246237936_0001!YARN_APPLICATION_ATTEMPT!appattempt_1465246237936_0001_000001"
         },
-        "configs": {},
-        "isrelatedto": {},
-        "relatesto": {}
+        "relatesto": { }
+      },
+      {
+        "metrics": [ ],
+        "events": [ ],
+        "type": "YARN_APPLICATION_ATTEMPT",
+        "id": "appattempt_1465246237936_0001_000002",
+        "createdtime": 1465246359045,
+        "isrelatedto": { },
+        "configs": { },
+        "info": {
+          "UID": "yarn-cluster!application_1465246237936_0001!YARN_APPLICATION_ATTEMPT!appattempt_1465246237936_0001_000002"
+        },
+        "relatesto": { }
       }
     ]
+
+### Response Codes
+
+1. If successful, a HTTP 200(OK) response is returned.
+1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
+1. If flow context information cannot be retrieved, HTTP 404 (Not Found) is returned.
+1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
+
+## <a name="REST_API_LIST_ENTITY"></a>Query generic entity
+
+With this API, you can query a specific generic entity identified by cluster ID, application ID, per-framework entity type and entity ID. If REST endpoint without
+cluster name is used, cluster specified by the configuration yarn.resourcemanager.cluster-id in yarn-site.xml is taken. Flow context information i.e.
+user, flow name and run id are not mandatory but if specified in query param can preclude the need for an additional operation to fetch
+flow context information based on cluster and app id. This endpoint can be used to query a single container, application attempt or any other generic entity which
+clients put into the backend. For instance, we can query a specific YARN container by specifying entity type as `YARN_CONTAINER` and giving entity ID as container ID.
+Similarly, application attempt can be queried by specifying entity type as `YARN_APPLICATION_ATTEMPT` and entity ID being the application attempt ID.
+
+### HTTP request:
+
+    GET /ws/v2/timeline/clusters/{cluster name}/apps/{app id}/entities/{entity type}/{entity id}
+
+    or
+
+    GET /ws/v2/timeline/apps/{app id}/entities/{entity type}/{entity id}
+
+### Query Parameters Supported:
+
+1. `userid` -  If specified, entity must belong to this user. This query param must be specified along with flowname and flowrunid query params, otherwise it will be ignored.
+  If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query.
+1. `flowname` - If specified, entity must belong to this flow name. This query param must be specified along with userid and flowrunid query params, otherwise it will be ignored.
+  If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query.
+1. `flowrunid` - If specified, entity must belong to this flow run id. This query param must be specified along with userid and flowname query params, otherwise it will be ignored.
+  If userid, flowname and flowrunid are not specified, we would have to fetch flow context information based on cluster and appid while executing the query.
+1. `metricsToRetrieve` - If specified, defines which metrics to retrieve or which ones not to retrieve and send back in response.
+  metricsToRetrieve can be an expression of the form :<br/>
+  (&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics matching any of the prefixes will be retrieved. Brackets are optional
+  for the simple expression. Alternatively, expressions can be of the form:<br/>
+  !(&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;,&lt;metricprefix&gt;...)<br/>
+  This specifies a comma separated list of metric id prefixes. Only metrics not matching any of the prefixes will be retrieved.<br/>
+  If metricsToRetrieve is specified, metrics will be retrieved irrespective of whether `METRICS` is specified in fields query param
+  or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `confsToRetrieve` - If specified, defines which configs to retrieve or which ones not to retrieve and send back in response.
+  confsToRetrieve can be an expression of the form :<br/>
+  (&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;...)<br/>
+  This specifies a comma separated list of config name prefixes. Only configs matching any of the prefixes will be retrieved. Brackets are optional for
+  the simple expression. Alternatively, expressions can be of the form :<br/>
+  !(&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;,&lt;config\_name\_prefix&gt;...)<br/>
+  This specifies a comma separated list of config name prefixes. Only configs not matching any of the prefixes will be retrieved.<br/>
+  If confsToRetrieve is specified, configs will be retrieved irrespective of whether `CONFIGS` is specified in fields query param
+  or not. Please note that URL unsafe characters such as spaces will have to be suitably encoded.
+1. `fields` - Specifies which fields to retrieve. Possible values for fields can be `EVENTS`, `INFO`, `CONFIGS`, `METRICS`, `RELATES_TO`,
+ `IS_RELATED_TO` and `ALL`. All fields will be retrieved if `ALL` is specified. Multiple fields can be specified as a comma-separated list.
+  If fields is not specified, in response, entity id, entity type, createdtime and UID in info field will be returned.
+1. `metricsLimit` - If specified, defines the number of metrics to return. Considered only if fields contains METRICS/ALL
+  or metricsToRetrieve is specified. Ignored otherwise. The maximum possible value for metricsLimit can be maximum value of
+  Integer. If it is not specified or has a value less than 1, and metrics have to be retrieved, then metricsLimit will be
+  considered as 1 i.e. latest single value of metric(s) will be returned.
+
+### Example JSON Response:
+
+    {
+      "metrics": [ ],
+      "events": [ ],
+      "type": "YARN_APPLICATION_ATTEMPT",
+      "id": "appattempt_1465246237936_0001_000001",
+      "createdtime": 1465246358873,
+      "isrelatedto": { },
+      "configs": { },
+      "info": {
+        "UID": "yarn-cluster!application_1465246237936_0001!YARN_APPLICATION_ATTEMPT!appattempt_1465246237936_0001_000001"
+      },
+      "relatesto": { }
+    }
+
+### Response Codes
+
+1. If successful, a HTTP 200 (OK) response is returned.
+1. If any problem occurs in parsing request, HTTP 400 (Bad Request) is returned.
+1. If flow context information cannot be retrieved or entity for the given entity id cannot be found, HTTP 404 (Not Found) is returned.
+1. For non-recoverable errors while retrieving data, HTTP 500 (Internal Server Error) is returned.
+


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


[30/50] [abbrv] hadoop git commit: YARN-5097. NPE in Separator.joinEncoded() (Vrushali C via sjlee)

Posted by vr...@apache.org.
YARN-5097. NPE in Separator.joinEncoded() (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/9f6a75d7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9f6a75d7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9f6a75d7

Branch: refs/heads/YARN-2928
Commit: 9f6a75d7dfa7006505a09913f27cb076fba37c0d
Parents: bab078d
Author: Sangjin Lee <sj...@apache.org>
Authored: Wed May 25 15:49:08 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:11 2016 -0700

----------------------------------------------------------------------
 .../storage/TestHBaseTimelineStorage.java       | 57 ++++++++++++++++++++
 .../storage/HBaseTimelineWriterImpl.java        |  9 ++++
 2 files changed, 66 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f6a75d7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/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-hbase-tests/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-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
index aebd936..68135a0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
@@ -482,6 +482,63 @@ public class TestHBaseTimelineStorage {
     }
   }
 
+
+  @Test
+  public void testWriteNullApplicationToHBase() throws Exception {
+    TimelineEntities te = new TimelineEntities();
+    ApplicationEntity entity = new ApplicationEntity();
+    String appId = "application_1000178881110_2002";
+    entity.setId(appId);
+    long cTime = 1425016501000L;
+    entity.setCreatedTime(cTime);
+
+    // 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);
+
+    te.addEntity(entity);
+    HBaseTimelineWriterImpl hbi = null;
+    try {
+      Configuration c1 = util.getConfiguration();
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      hbi.start();
+      String cluster = "cluster_check_null_application";
+      String user = "user1check_null_application";
+      //set the flow name to null
+      String flow = null;
+      String flowVersion = "AB7822C10F1111";
+      long runid = 1002345678919L;
+      hbi.write(cluster, user, flow, flowVersion, runid, appId, te);
+      hbi.stop();
+
+      // retrieve the row
+      Scan scan = new Scan();
+      scan.setStartRow(Bytes.toBytes(cluster));
+      Connection conn = ConnectionFactory.createConnection(c1);
+      ResultScanner resultScanner = new ApplicationTable()
+          .getResultScanner(c1, conn, scan);
+
+      assertTrue(resultScanner != null);
+      // try to iterate over results
+      int count = 0;
+      for (Result rr = resultScanner.next(); rr != null;
+          rr = resultScanner.next()) {
+         count++;
+      }
+      // there should be no rows written
+      // no exceptions thrown during write
+      assertEquals(0, count);
+    } finally {
+      if (hbi != null) {
+        hbi.stop();
+        hbi.close();
+      }
+    }
+  }
+
   @Test
   public void testWriteApplicationToHBase() throws Exception {
     TimelineEntities te = new TimelineEntities();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9f6a75d7/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 172f982..fe4671f 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
@@ -118,6 +118,15 @@ public class HBaseTimelineWriterImpl extends AbstractService implements
       TimelineEntities data) throws IOException {
 
     TimelineWriteResponse putStatus = new TimelineWriteResponse();
+    // defensive coding to avoid NPE during row key construction
+    if ((flowName == null) || (appId == null) || (clusterId == null)
+        || (userId == null)) {
+      LOG.warn("Found null for one of: flowName=" + flowName + " appId=" + appId
+          + " userId=" + userId + " clusterId=" + clusterId
+          + " . Not proceeding with writing to hbase");
+      return putStatus;
+    }
+
     for (TimelineEntity te : data.getEntities()) {
 
       // a set can have at most 1 null


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


[24/50] [abbrv] hadoop git commit: YARN-5045. hbase unit tests fail due to dependency issues. (Sangjin Lee via varunsaxena)

Posted by vr...@apache.org.
YARN-5045. hbase unit tests fail due to dependency issues. (Sangjin Lee via varunsaxena)


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

Branch: refs/heads/YARN-2928
Commit: ccdec4a1df4d218df91ef8facf1936ac871be4ec
Parents: 91a9099
Author: Varun Saxena <va...@apache.org>
Authored: Sat May 7 13:48:06 2016 +0530
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:08 2016 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |   30 +
 .../pom.xml                                     |  390 ++
 ...stTimelineReaderWebServicesHBaseStorage.java | 2008 ++++++++++
 .../storage/TestHBaseTimelineStorage.java       | 3461 ++++++++++++++++++
 ...TestPhoenixOfflineAggregationWriterImpl.java |  161 +
 .../storage/flow/TestFlowDataGenerator.java     |  383 ++
 .../flow/TestHBaseStorageFlowActivity.java      |  469 +++
 .../storage/flow/TestHBaseStorageFlowRun.java   |  851 +++++
 .../flow/TestHBaseStorageFlowRunCompaction.java |  671 ++++
 .../src/test/resources/log4j.properties         |   19 +
 .../hadoop-yarn-server-timelineservice/pom.xml  |   69 +-
 ...stTimelineReaderWebServicesHBaseStorage.java | 2008 ----------
 .../storage/TestHBaseTimelineStorage.java       | 3461 ------------------
 ...TestPhoenixOfflineAggregationWriterImpl.java |  161 -
 .../storage/flow/TestFlowDataGenerator.java     |  383 --
 .../flow/TestHBaseStorageFlowActivity.java      |  469 ---
 .../storage/flow/TestHBaseStorageFlowRun.java   |  851 -----
 .../flow/TestHBaseStorageFlowRunCompaction.java |  671 ----
 .../hadoop-yarn/hadoop-yarn-server/pom.xml      |    1 +
 19 files changed, 8484 insertions(+), 8033 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 20928e7..04464e4 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -51,6 +51,7 @@
     <kafka.version>0.8.2.1</kafka.version>
     <hbase.version>1.0.1</hbase.version>
     <phoenix.version>4.5.0-SNAPSHOT</phoenix.version>
+    <hbase-compatible-hadoop.version>2.5.1</hbase-compatible-hadoop.version>
 
     <hadoop.assemblies.version>${project.version}</hadoop.assemblies.version>
     <commons-daemon.version>1.0.13</commons-daemon.version>
@@ -474,6 +475,11 @@
         <version>1.4.1</version>
       </dependency>
       <dependency>
+        <groupId>org.apache.commons</groupId>
+        <artifactId>commons-csv</artifactId>
+        <version>1.0</version>
+      </dependency>
+      <dependency>
         <groupId>xmlenc</groupId>
         <artifactId>xmlenc</artifactId>
         <version>0.52</version>
@@ -1034,6 +1040,18 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-common</artifactId>
+        <version>${hbase.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-common</artifactId>
+        <version>${hbase.version}</version>
+        <scope>test</scope>
+        <classifier>tests</classifier>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
         <artifactId>hbase-client</artifactId>
         <version>${hbase.version}</version>
         <exclusions>
@@ -1045,6 +1063,18 @@
         </exclusions>
       </dependency>
       <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-server</artifactId>
+        <version>${hbase.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-server</artifactId>
+        <version>${hbase.version}</version>
+        <scope>test</scope>
+        <classifier>tests</classifier>
+      </dependency>
+      <dependency>
         <groupId>org.apache.phoenix</groupId>
         <artifactId>phoenix-core</artifactId>
         <version>${phoenix.version}</version>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
new file mode 100644
index 0000000..b5dc3c0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml
@@ -0,0 +1,390 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+                      http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>hadoop-yarn-server</artifactId>
+    <groupId>org.apache.hadoop</groupId>
+    <version>3.0.0-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-yarn-server-timelineservice-hbase-tests</artifactId>
+  <version>3.0.0-SNAPSHOT</version>
+  <name>Apache Hadoop YARN Timeline Service HBase tests</name>
+
+  <properties>
+    <!-- Needed for generating FindBugs warnings using parent pom -->
+    <yarn.basedir>${project.parent.parent.basedir}</yarn.basedir>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- Since hbase 1.0.1 requires an older version of hadoop we enforce an
+         older version of hadoop just for the hbase unit tests. We also need to
+         exclude hadoop-common and hadoop-hdfs on each dependency that has
+         transitive depedencies on them.
+    -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-timelineservice</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.phoenix</groupId>
+          <artifactId>phoenix-core</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>${hbase-compatible-hadoop.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-auth</artifactId>
+      <version>${hbase-compatible-hadoop.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- 'mvn dependency:analyze' fails to detect use of this direct
+         dependency -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-common</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-client</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-client</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-hdfs</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-hdfs-client</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-hdfs</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-hdfs-client</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- 'mvn dependency:analyze' fails to detect use of this direct
+         dependency -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-it</artifactId>
+      <scope>test</scope>
+      <classifier>tests</classifier>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-hdfs</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.phoenix</groupId>
+      <artifactId>phoenix-core</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-hdfs</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.phoenix</groupId>
+      <artifactId>phoenix-core</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-hdfs</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- for runtime dependencies -->
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>${hbase-compatible-hadoop.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <version>${hbase-compatible-hadoop.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <version>${hbase-compatible-hadoop.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-testing-util</artifactId>
+      <scope>test</scope>
+      <optional>true</optional>
+      <exclusions>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <phase>test-compile</phase>
+          </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>


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


[35/50] [abbrv] hadoop git commit: YARN-5109. timestamps are stored unencoded causing parse errors (Varun Saxena via sjlee)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyConverter.java
----------------------------------------------------------------------
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/FlowRunRowKeyConverter.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/FlowRunRowKeyConverter.java
new file mode 100644
index 0000000..642f065
--- /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/FlowRunRowKeyConverter.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.storage.flow;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+
+/**
+ * Encodes and decodes row key for flow run table.
+ * The row key is of the form : clusterId!userId!flowName!flowrunId.
+ * flowrunId is a long and rest are strings.
+ */
+public final class FlowRunRowKeyConverter implements
+    KeyConverter<FlowRunRowKey> {
+  private static final FlowRunRowKeyConverter INSTANCE =
+      new FlowRunRowKeyConverter();
+
+  public static FlowRunRowKeyConverter getInstance() {
+    return INSTANCE;
+  }
+
+  private FlowRunRowKeyConverter() {
+  }
+
+  // Flow run row key is of the form
+  // clusterId!userId!flowName!flowrunId with each segment separated by !.
+  // The sizes below indicate sizes of each one of these segments in sequence.
+  // clusterId, userId and flowName are strings. flowrunId is a long hence 8
+  // bytes in size. Strings are variable in size (i.e. end whenever separator is
+  // encountered). This is used while decoding and helps in determining where to
+  // split.
+  private static final int[] SEGMENT_SIZES = {
+      Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+      Bytes.SIZEOF_LONG };
+
+  /*
+   * (non-Javadoc)
+   *
+   * Encodes FlowRunRowKey object into a byte array with each component/field in
+   * FlowRunRowKey separated by Separator#QUALIFIERS. This leads to an
+   * flow run row key of the form clusterId!userId!flowName!flowrunId
+   * If flowRunId in passed FlowRunRowKey object is null (and the fields
+   * preceding it i.e. clusterId, userId and flowName are not null), this
+   * returns a row key prefix of the form clusterId!userName!flowName!
+   * flowRunId is inverted while encoding as it helps maintain a descending
+   * order for flow keys in flow run table.
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #encode(java.lang.Object)
+   */
+  @Override
+  public byte[] encode(FlowRunRowKey rowKey) {
+    byte[] first = Separator.QUALIFIERS.join(
+        Separator.encode(rowKey.getClusterId(), Separator.SPACE, Separator.TAB,
+            Separator.QUALIFIERS),
+        Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
+            Separator.QUALIFIERS),
+        Separator.encode(rowKey.getFlowName(), Separator.SPACE, Separator.TAB,
+            Separator.QUALIFIERS));
+    if (rowKey.getFlowRunId() == null) {
+      return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES);
+    } else {
+      // Note that flowRunId is a long, so we can't encode them all at the same
+      // time.
+      byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(
+          rowKey.getFlowRunId()));
+      return Separator.QUALIFIERS.join(first, second);
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * Decodes an flow run row key of the form
+   * clusterId!userId!flowName!flowrunId represented in byte format and converts
+   * it into an FlowRunRowKey object. flowRunId is inverted while decoding as
+   * it was inverted while encoding.
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
+   * #decode(byte[])
+   */
+  @Override
+  public FlowRunRowKey decode(byte[] rowKey) {
+    byte[][] rowKeyComponents =
+        Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
+    if (rowKeyComponents.length != 4) {
+      throw new IllegalArgumentException("the row key is not valid for " +
+          "a flow run");
+    }
+    String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    String userId = Separator.decode(Bytes.toString(rowKeyComponents[1]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    String flowName = Separator.decode(Bytes.toString(rowKeyComponents[2]),
+        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+    Long flowRunId =
+        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
+    return new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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 398d7b4..648c77b 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
@@ -44,9 +44,10 @@ import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 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.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimestampGenerator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -193,7 +194,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 = TimelineStorageUtils.EMPTY_BYTES;
+    byte[] currentColumnQualifier = Separator.EMPTY_BYTES;
     AggregationOperation currentAggOp = null;
     SortedSet<Cell> currentColumnCells = new TreeSet<>(KeyValue.COMPARATOR);
     Set<String> alreadySeenAggDim = new HashSet<>();
@@ -314,7 +315,7 @@ class FlowScanner implements RegionScanner, Closeable {
             + " cell qualifier="
             + Bytes.toString(CellUtil.cloneQualifier(cell))
             + " cell value= "
-            + (Number) converter.decodeValue(CellUtil.cloneValue(cell))
+            + converter.decodeValue(CellUtil.cloneValue(cell))
             + " timestamp=" + cell.getTimestamp());
       }
 
@@ -480,7 +481,7 @@ class FlowScanner implements RegionScanner, Closeable {
             LOG.trace("MAJOR COMPACTION loop sum= " + sum
                 + " discarding now: " + " qualifier="
                 + Bytes.toString(CellUtil.cloneQualifier(cell)) + " value="
-                + (Number) converter.decodeValue(CellUtil.cloneValue(cell))
+                + converter.decodeValue(CellUtil.cloneValue(cell))
                 + " timestamp=" + cell.getTimestamp() + " " + this.action);
           }
         } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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
index d8ca038..faecd14 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/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
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineDataToRetrie
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongKeyConverter;
 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;
@@ -125,7 +126,7 @@ class FlowActivityEntityReader extends TimelineEntityReader {
   protected TimelineEntity parseEntity(Result result) throws IOException {
     FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(result.getRow());
 
-    long time = rowKey.getDayTimestamp();
+    Long time = rowKey.getDayTimestamp();
     String user = rowKey.getUserId();
     String flowName = rowKey.getFlowName();
 
@@ -135,10 +136,11 @@ class FlowActivityEntityReader extends TimelineEntityReader {
     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());
+    Map<Long, Object> runIdsMap =
+        FlowActivityColumnPrefix.RUN_ID.readResults(result,
+            LongKeyConverter.getInstance());
+    for (Map.Entry<Long, Object> e : runIdsMap.entrySet()) {
+      Long runId = e.getKey();
       String version = (String)e.getValue();
       FlowRunEntity flowRun = new FlowRunEntity();
       flowRun.setUser(user);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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
index 4299de9..be27643 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/reader/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
@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilter
 import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
 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.StringKeyConverter;
 
 /**
  * The base class for reading and deserializing timeline entities from the
@@ -329,7 +330,8 @@ public abstract class TimelineEntityReader {
   protected void readMetrics(TimelineEntity entity, Result result,
       ColumnPrefix<?> columnPrefix) throws IOException {
     NavigableMap<String, NavigableMap<Long, Number>> metricsResult =
-        columnPrefix.readResultsWithTimestamps(result);
+        columnPrefix.readResultsWithTimestamps(
+            result, StringKeyConverter.getInstance());
     for (Map.Entry<String, NavigableMap<Long, Number>> metricResult:
         metricsResult.entrySet()) {
       TimelineMetric metric = new TimelineMetric();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestKeyConverters.java
new file mode 100644
index 0000000..74e4b5d
--- /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/TestKeyConverters.java
@@ -0,0 +1,293 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.timelineservice.storage.common;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKeyConverter;
+import org.junit.Test;
+
+public class TestKeyConverters {
+  private final static String QUALIFIER_SEP = Separator.QUALIFIERS.getValue();
+  private final static byte[] QUALIFIER_SEP_BYTES =
+      Bytes.toBytes(QUALIFIER_SEP);
+  private final static String CLUSTER = "cl" + QUALIFIER_SEP + "uster";
+  private final static String USER = QUALIFIER_SEP + "user";
+  private final static String FLOW_NAME =
+      "dummy_" + QUALIFIER_SEP + "flow" + QUALIFIER_SEP;
+  private final static Long FLOW_RUN_ID;
+  private final static String APPLICATION_ID;
+  static {
+    long runid = Long.MAX_VALUE - 900L;
+    byte[] longMaxByteArr = Bytes.toBytes(Long.MAX_VALUE);
+    byte[] byteArr = Bytes.toBytes(runid);
+    int sepByteLen = QUALIFIER_SEP_BYTES.length;
+    if (sepByteLen <= byteArr.length) {
+      for (int i = 0; i < sepByteLen; i++) {
+        byteArr[i] = (byte)(longMaxByteArr[i] - QUALIFIER_SEP_BYTES[i]);
+      }
+    }
+    FLOW_RUN_ID = Bytes.toLong(byteArr);
+    long clusterTs = System.currentTimeMillis();
+    byteArr = Bytes.toBytes(clusterTs);
+    if (sepByteLen <= byteArr.length) {
+      for (int i = 0; i < sepByteLen; i++) {
+        byteArr[byteArr.length - sepByteLen + i] =
+            (byte)(longMaxByteArr[byteArr.length - sepByteLen + i] -
+                QUALIFIER_SEP_BYTES[i]);
+      }
+    }
+    clusterTs = Bytes.toLong(byteArr);
+    int seqId = 222;
+    APPLICATION_ID = ApplicationId.newInstance(clusterTs, seqId).toString();
+  }
+
+  private static void verifyRowPrefixBytes(byte[] byteRowKeyPrefix) {
+    int sepLen = QUALIFIER_SEP_BYTES.length;
+    for (int i = 0; i < sepLen; i++) {
+      assertTrue("Row key prefix not encoded properly.",
+        byteRowKeyPrefix[byteRowKeyPrefix.length - sepLen  + i] ==
+            QUALIFIER_SEP_BYTES[i]);
+    }
+  }
+
+  @Test
+  public void testFlowActivityRowKeyConverter() {
+    Long ts = TimelineStorageUtils.getTopOfTheDayTimestamp(1459900830000L);
+    byte[] byteRowKey = FlowActivityRowKeyConverter.getInstance().encode(
+        new FlowActivityRowKey(CLUSTER, ts, USER, FLOW_NAME));
+    FlowActivityRowKey rowKey =
+        FlowActivityRowKeyConverter.getInstance().decode(byteRowKey);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(ts, rowKey.getDayTimestamp());
+    assertEquals(USER, rowKey.getUserId());
+    assertEquals(FLOW_NAME, rowKey.getFlowName());
+
+    byte[] byteRowKeyPrefix = FlowActivityRowKeyConverter.getInstance().encode(
+        new FlowActivityRowKey(CLUSTER, null, null, null));
+    byte[][] splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
+        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
+    assertEquals(2, splits.length);
+    assertEquals(0, splits[1].length);
+    assertEquals(CLUSTER,
+        Separator.QUALIFIERS.decode(Bytes.toString(splits[0])));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+
+    byteRowKeyPrefix = FlowActivityRowKeyConverter.getInstance().encode(
+        new FlowActivityRowKey(CLUSTER, ts, null, null));
+    splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
+        Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE });
+    assertEquals(3, splits.length);
+    assertEquals(0, splits[2].length);
+    assertEquals(CLUSTER,
+        Separator.QUALIFIERS.decode(Bytes.toString(splits[0])));
+    assertEquals(ts, (Long) TimelineStorageUtils.invertLong(
+        Bytes.toLong(splits[1])));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+  }
+
+  @Test
+  public void testFlowRunRowKeyConverter() {
+    byte[] byteRowKey = FlowRunRowKeyConverter.getInstance().encode(
+        new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID));
+    FlowRunRowKey rowKey =
+        FlowRunRowKeyConverter.getInstance().decode(byteRowKey);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(USER, rowKey.getUserId());
+    assertEquals(FLOW_NAME, rowKey.getFlowName());
+    assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
+
+    byte[] byteRowKeyPrefix = FlowRunRowKeyConverter.getInstance().encode(
+        new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, null));
+    byte[][] splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
+        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
+    assertEquals(4, splits.length);
+    assertEquals(0, splits[3].length);
+    assertEquals(FLOW_NAME,
+        Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+  }
+
+  @Test
+  public void testApplicationRowKeyConverter() {
+    byte[] byteRowKey = ApplicationRowKeyConverter.getInstance().encode(
+        new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID,
+            APPLICATION_ID));
+    ApplicationRowKey rowKey =
+        ApplicationRowKeyConverter.getInstance().decode(byteRowKey);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(USER, rowKey.getUserId());
+    assertEquals(FLOW_NAME, rowKey.getFlowName());
+    assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
+    assertEquals(APPLICATION_ID, rowKey.getAppId());
+
+    byte[] byteRowKeyPrefix = ApplicationRowKeyConverter.getInstance().encode(
+        new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, null));
+    byte[][] splits =
+        Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
+            Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+            Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+            Separator.VARIABLE_SIZE });
+    assertEquals(5, splits.length);
+    assertEquals(0, splits[4].length);
+    assertEquals(FLOW_NAME,
+        Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
+    assertEquals(FLOW_RUN_ID, (Long)TimelineStorageUtils.invertLong(
+        Bytes.toLong(splits[3])));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+
+    byteRowKeyPrefix = ApplicationRowKeyConverter.getInstance().encode(
+        new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, null, null));
+    splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
+        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
+    assertEquals(4, splits.length);
+    assertEquals(0, splits[3].length);
+    assertEquals(FLOW_NAME,
+        Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+  }
+
+  @Test
+  public void testEntityRowKeyConverter() {
+    String entityId = "!ent!ity!!id!";
+    String entityType = "entity!Type";
+    byte[] byteRowKey = EntityRowKeyConverter.getInstance().encode(
+        new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID,
+            entityType, entityId));
+    EntityRowKey rowKey =
+        EntityRowKeyConverter.getInstance().decode(byteRowKey);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(USER, rowKey.getUserId());
+    assertEquals(FLOW_NAME, rowKey.getFlowName());
+    assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
+    assertEquals(APPLICATION_ID, rowKey.getAppId());
+    assertEquals(entityType, rowKey.getEntityType());
+    assertEquals(entityId, rowKey.getEntityId());
+
+    byte[] byteRowKeyPrefix = EntityRowKeyConverter.getInstance().encode(
+        new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID,
+            entityType, null));
+    byte[][] splits =
+        Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
+            Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+            Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+            AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE,
+            Separator.VARIABLE_SIZE });
+    assertEquals(7, splits.length);
+    assertEquals(0, splits[6].length);
+    assertEquals(APPLICATION_ID,
+        AppIdKeyConverter.getInstance().decode(splits[4]));
+    assertEquals(entityType, Separator.QUALIFIERS.decode(
+        Bytes.toString(splits[5])));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+
+    byteRowKeyPrefix = EntityRowKeyConverter.getInstance().encode(
+        new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID,
+        null, null));
+    splits = Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
+        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+        Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+        AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE });
+    assertEquals(6, splits.length);
+    assertEquals(0, splits[5].length);
+    assertEquals(APPLICATION_ID,
+        AppIdKeyConverter.getInstance().decode(splits[4]));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+  }
+
+  @Test
+  public void testAppToFlowRowKeyConverter() {
+    byte[] byteRowKey = AppToFlowRowKeyConverter.getInstance().encode(
+        new AppToFlowRowKey(CLUSTER, APPLICATION_ID));
+    AppToFlowRowKey rowKey =
+        AppToFlowRowKeyConverter.getInstance().decode(byteRowKey);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(APPLICATION_ID, rowKey.getAppId());
+  }
+
+  @Test
+  public void testAppIdKeyConverter() {
+    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 = AppIdKeyConverter.getInstance().encode(appIdStr1);
+    byte[] appIdBytes2 = AppIdKeyConverter.getInstance().encode(appIdStr2);
+    byte[] appIdBytes3 = AppIdKeyConverter.getInstance().encode(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 = AppIdKeyConverter.getInstance().decode(appIdBytes1);
+    String decodedAppId2 = AppIdKeyConverter.getInstance().decode(appIdBytes2);
+    String decodedAppId3 = AppIdKeyConverter.getInstance().decode(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));
+  }
+
+  @Test
+  public void testEventColumnNameConverter() {
+    String eventId = "=foo_=eve=nt=";
+    byte[] valSepBytes = Bytes.toBytes(Separator.VALUES.getValue());
+    byte[] maxByteArr =
+        Bytes.createMaxByteArray(Bytes.SIZEOF_LONG - valSepBytes.length);
+    byte[] ts = Bytes.add(valSepBytes, maxByteArr);
+    Long eventTs = Bytes.toLong(ts);
+    byte[] byteEventColName = EventColumnNameConverter.getInstance().encode(
+        new EventColumnName(eventId, eventTs, null));
+    EventColumnName eventColName =
+        EventColumnNameConverter.getInstance().decode(byteEventColName);
+    assertEquals(eventId, eventColName.getId());
+    assertEquals(eventTs, eventColName.getTimestamp());
+    assertNull(eventColName.getInfoKey());
+
+    String infoKey = "f=oo_event_in=fo=_key";
+    byteEventColName = EventColumnNameConverter.getInstance().encode(
+        new EventColumnName(eventId, eventTs, infoKey));
+    eventColName =
+        EventColumnNameConverter.getInstance().decode(byteEventColName);
+    assertEquals(eventId, eventColName.getId());
+    assertEquals(eventTs, eventColName.getTimestamp());
+    assertEquals(infoKey, eventColName.getInfoKey());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.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/TestSeparator.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/TestSeparator.java
index 8b25a83..0cda97c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.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/TestSeparator.java
@@ -19,11 +19,13 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.common;
 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 java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 
 import com.google.common.collect.Iterables;
@@ -32,7 +34,7 @@ public class TestSeparator {
 
   private static String villain = "Dr. Heinz Doofenshmirtz";
   private static String special =
-      ".   *   |   ?   +   (   )   [   ]   {   }   ^   $  \\ \"";
+      ".   *   |   ?   +   \t   (   )   [   ]   {   }   ^   $  \\ \"";
 
   /**
    *
@@ -47,6 +49,7 @@ public class TestSeparator {
       testEncodeDecode(separator, "?");
       testEncodeDecode(separator, "&");
       testEncodeDecode(separator, "+");
+      testEncodeDecode(separator, "\t");
       testEncodeDecode(separator, "Dr.");
       testEncodeDecode(separator, "Heinz");
       testEncodeDecode(separator, "Doofenshmirtz");
@@ -79,6 +82,83 @@ public class TestSeparator {
 
   }
 
+  @Test
+  public void testSplits() {
+    byte[] maxLongBytes = Bytes.toBytes(Long.MAX_VALUE);
+    byte[] maxIntBytes = Bytes.toBytes(Integer.MAX_VALUE);
+    for (Separator separator : Separator.values()) {
+      String str1 = "cl" + separator.getValue() + "us";
+      String str2 = separator.getValue() + "rst";
+      byte[] sepByteArr = Bytes.toBytes(separator.getValue());
+      byte[] longVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxLongBytes,
+          sepByteArr.length, Bytes.SIZEOF_LONG - sepByteArr.length));
+      byte[] intVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxIntBytes,
+          sepByteArr.length, Bytes.SIZEOF_INT - sepByteArr.length));
+      byte[] arr = separator.join(
+          Bytes.toBytes(separator.encode(str1)),longVal1Arr,
+          Bytes.toBytes(separator.encode(str2)), intVal1Arr);
+      int[] sizes = { Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+          Separator.VARIABLE_SIZE, Bytes.SIZEOF_INT };
+      byte[][] splits = separator.split(arr, sizes);
+      assertEquals(4, splits.length);
+      assertEquals(str1, separator.decode(Bytes.toString(splits[0])));
+      assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[1]));
+      assertEquals(str2, separator.decode(Bytes.toString(splits[2])));
+      assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[3]));
+
+      longVal1Arr = Bytes.add(Bytes.copy(maxLongBytes, 0, Bytes.SIZEOF_LONG -
+          sepByteArr.length), sepByteArr);
+      intVal1Arr = Bytes.add(Bytes.copy(maxIntBytes, 0, Bytes.SIZEOF_INT -
+          sepByteArr.length), sepByteArr);
+      arr = separator.join(Bytes.toBytes(separator.encode(str1)),longVal1Arr,
+          Bytes.toBytes(separator.encode(str2)), intVal1Arr);
+      splits = separator.split(arr, sizes);
+      assertEquals(4, splits.length);
+      assertEquals(str1, separator.decode(Bytes.toString(splits[0])));
+      assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[1]));
+      assertEquals(str2, separator.decode(Bytes.toString(splits[2])));
+      assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[3]));
+
+      longVal1Arr = Bytes.add(sepByteArr, Bytes.copy(maxLongBytes,
+          sepByteArr.length, 4 - sepByteArr.length), sepByteArr);
+      longVal1Arr = Bytes.add(longVal1Arr, Bytes.copy(maxLongBytes, 4, 3 -
+              sepByteArr.length), sepByteArr);
+      arr = separator.join(Bytes.toBytes(separator.encode(str1)),longVal1Arr,
+          Bytes.toBytes(separator.encode(str2)), intVal1Arr);
+      splits = separator.split(arr, sizes);
+      assertEquals(4, splits.length);
+      assertEquals(str1, separator.decode(Bytes.toString(splits[0])));
+      assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[1]));
+      assertEquals(str2, separator.decode(Bytes.toString(splits[2])));
+      assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[3]));
+
+      arr = separator.join(Bytes.toBytes(separator.encode(str1)),
+          Bytes.toBytes(separator.encode(str2)), intVal1Arr, longVal1Arr);
+      int[] sizes1 = { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+          Bytes.SIZEOF_INT, Bytes.SIZEOF_LONG };
+      splits = separator.split(arr, sizes1);
+      assertEquals(4, splits.length);
+      assertEquals(str1, separator.decode(Bytes.toString(splits[0])));
+      assertEquals(str2, separator.decode(Bytes.toString(splits[1])));
+      assertEquals(Bytes.toInt(intVal1Arr), Bytes.toInt(splits[2]));
+      assertEquals(Bytes.toLong(longVal1Arr), Bytes.toLong(splits[3]));
+
+      try {
+        int[] sizes2 = { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+            Bytes.SIZEOF_INT, 7 };
+        splits = separator.split(arr, sizes2);
+        fail("Exception should have been thrown.");
+      } catch (IllegalArgumentException e) {}
+
+      try {
+        int[] sizes2 = { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, 2,
+            Bytes.SIZEOF_LONG };
+        splits = separator.split(arr, sizes2);
+        fail("Exception should have been thrown.");
+      } catch (IllegalArgumentException e) {}
+    }
+  }
+
   /**
    * Simple test to encode and decode using the same separators and confirm that
    * we end up with the same as what we started with.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4ad33a07/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
deleted file mode 100644
index 046eda1..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/TestTimelineStorageUtils.java
+++ /dev/null
@@ -1,56 +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 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));
-  }
-}


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


[40/50] [abbrv] hadoop git commit: YARN-5167. Escape occurences of encodedValues. (Sangjin Lee via Varun Saxena)

Posted by vr...@apache.org.
YARN-5167. Escape occurences of encodedValues. (Sangjin Lee 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/2cab3fc0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2cab3fc0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2cab3fc0

Branch: refs/heads/YARN-2928
Commit: 2cab3fc03b80ae2e784acbd38a97d2e8b4cb221a
Parents: 4df6d9b
Author: Varun Saxena <va...@apache.org>
Authored: Mon Jun 6 09:39:59 2016 +0530
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:13 2016 -0700

----------------------------------------------------------------------
 .../storage/common/Separator.java               | 123 ++++++++++++++-----
 .../storage/common/TestSeparator.java           |   8 +-
 2 files changed, 101 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2cab3fc0/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 8a178db..5090b4d 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
@@ -20,12 +20,13 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Used to separate row qualifiers, column qualifiers and compount fields.
+ * Used to separate row qualifiers, column qualifiers and compound fields.
  */
 public enum Separator {
 
@@ -53,25 +54,39 @@ public enum Separator {
    */
   TAB("\t", "%3$");
 
+  // a reserved character that starts each of the encoded values and is encoded
+  // first in order to escape naturally occurring instances of encoded values
+  // although it can be expressed as an enum instance, we define them as private
+  // variables to hide it from callers
+  private static final String PERCENT = "%";
+  private static final String PERCENT_ENCODED = "%9$";
+
+  private static final Pattern PERCENT_PATTERN =
+      Pattern.compile(PERCENT, Pattern.LITERAL);
+  private static final String PERCENT_REPLACEMENT =
+      Matcher.quoteReplacement(PERCENT);
+
+  private static final Pattern PERCENT_ENCODED_PATTERN =
+      Pattern.compile(PERCENT_ENCODED, Pattern.LITERAL);
+  private static final String PERCENT_ENCODED_REPLACEMENT =
+      Matcher.quoteReplacement(PERCENT_ENCODED);
+
   /**
    * The string value of this separator.
    */
   private final String value;
 
   /**
-   * The URLEncoded version of this separator.
-   */
-  private final String encodedValue;
-
-  /**
    * The bye representation of value.
    */
   private final byte[] bytes;
 
-  /**
-   * The value quoted so that it can be used as a safe regex.
-   */
-  private final String quotedValue;
+  // pre-compiled patterns and quoted replacements for optimization
+  private final Pattern valuePattern;
+  private final String valueReplacement;
+
+  private final Pattern encodedValuePattern;
+  private final String encodedValueReplacement;
 
   /**
    * Indicator for variable size of an individual segment in a split. The
@@ -97,7 +112,6 @@ public enum Separator {
    */
   private Separator(String value, String encodedValue) {
     this.value = value;
-    this.encodedValue = encodedValue;
 
     // validation
     if (value == null || value.length() == 0 || encodedValue == null
@@ -107,7 +121,11 @@ public enum Separator {
     }
 
     this.bytes = Bytes.toBytes(value);
-    this.quotedValue = Pattern.quote(value);
+    this.valuePattern = Pattern.compile(value, Pattern.LITERAL);
+    this.valueReplacement = Matcher.quoteReplacement(value);
+
+    this.encodedValuePattern = Pattern.compile(encodedValue, Pattern.LITERAL);
+    this.encodedValueReplacement = Matcher.quoteReplacement(encodedValue);
   }
 
   /**
@@ -119,6 +137,13 @@ public enum Separator {
 
   /**
    * Used to make token safe to be used with this separator without collisions.
+   * It <em>must</em> be paired with {@link #decode(String)} for it to be
+   * decoded correctly.
+   * <p>
+   * If you need to encode a given string for multiple separators,
+   * {@link #encode(String, Separator...)} should be used over successive
+   * invocations of this method. It will result in a more compact version of the
+   * encoded value.
    *
    * @param token Token to be encoded.
    * @return the token with any occurrences of this separator URLEncoded.
@@ -128,11 +153,29 @@ public enum Separator {
       // Nothing to replace
       return token;
     }
-    return token.replace(value, encodedValue);
+    // first encode the percent to escape naturally occurring encoded values
+    String escaped = encodePercent(token);
+    return encodeSingle(escaped, this);
+  }
+
+  private static String replace(String token, Pattern pattern,
+      String replacement) {
+    return pattern.matcher(token).replaceAll(replacement);
+  }
+
+  private static String encodeSingle(String token, Separator separator) {
+    return replace(token, separator.valuePattern,
+        separator.encodedValueReplacement);
+  }
+
+  private static String encodePercent(String token) {
+    return replace(token, PERCENT_PATTERN, PERCENT_ENCODED_REPLACEMENT);
   }
 
   /**
-   * Decode the token encoded using {@link #encode}.
+   * Decode the token encoded using {@link #encode(String)}. It <em>must</em> be
+   * used for the result encoded with {@link #encode(String)} to be able to
+   * recover the original.
    *
    * @param token Token to be decoded.
    * @return the token with any occurrences of the encoded separator replaced by
@@ -143,13 +186,30 @@ public enum Separator {
       // Nothing to replace
       return token;
     }
-    return token.replace(encodedValue, value);
+    String escaped = decodeSingle(token, this);
+    // decode percent to de-escape
+    return decodePercent(escaped);
+  }
+
+  private static String decodeSingle(String token, Separator separator) {
+    return replace(token, separator.encodedValuePattern,
+        separator.valueReplacement);
+  }
+
+  private static String decodePercent(String token) {
+    return replace(token, PERCENT_ENCODED_PATTERN, PERCENT_REPLACEMENT);
   }
 
   /**
-   * Encode the given separators in the token with their encoding equivalent.
-   * This means that when encoding is already present in the token itself, this
-   * is not a reversible process. See also {@link #decode(String, Separator...)}
+   * Encode the given separators in the token with their encoding equivalents.
+   * It <em>must</em> be paired with {@link #decode(byte[], Separator...)} or
+   * {@link #decode(String, Separator...)} with the same separators for it to be
+   * decoded correctly.
+   * <p>
+   * If you need to encode a given string for multiple separators, this form of
+   * encoding should be used over successive invocations of
+   * {@link #encode(String)}. It will result in a more compact version of the
+   * encoded value.
    *
    * @param token containing possible separators that need to be encoded.
    * @param separators to be encoded in the token with their URLEncoding
@@ -158,22 +218,25 @@ public enum Separator {
    *         separators encoded.
    */
   public static byte[] encode(String token, Separator... separators) {
-    if (token == null) {
+    if (token == null || token.length() == 0) {
       return EMPTY_BYTES;
     }
     String result = token;
+    // first encode the percent to escape naturally occurring encoded values
+    result = encodePercent(token);
     for (Separator separator : separators) {
       if (separator != null) {
-        result = separator.encode(result);
+        result = encodeSingle(result, separator);
       }
     }
     return Bytes.toBytes(result);
   }
 
   /**
-   * Decode the given separators in the token with their decoding equivalent.
-   * This means that when encoding is already present in the token itself, this
-   * is not a reversible process.
+   * Decode the given separators in the token with their decoding equivalents.
+   * It <em>must</em> be used for the result encoded with
+   * {@link #encode(String, Separator...)} with the same separators to be able
+   * to recover the original.
    *
    * @param token containing possible separators that need to be encoded.
    * @param separators to be encoded in the token with their URLEncoding
@@ -189,9 +252,10 @@ public enum Separator {
   }
 
   /**
-   * Decode the given separators in the token with their decoding equivalent.
-   * This means that when encoding is already present in the token itself, this
-   * is not a reversible process.
+   * Decode the given separators in the token with their decoding equivalents.
+   * It <em>must</em> be used for the result encoded with
+   * {@link #encode(String, Separator...)} with the same separators to be able
+   * to recover the original.
    *
    * @param token containing possible separators that need to be encoded.
    * @param separators to be encoded in the token with their URLEncoding
@@ -206,10 +270,11 @@ public enum Separator {
     String result = token;
     for (Separator separator : separators) {
       if (separator != null) {
-        result = separator.decode(result);
+        result = decodeSingle(result, separator);
       }
     }
-    return result;
+    // decode percent to de-escape
+    return decodePercent(result);
   }
 
   /**
@@ -309,7 +374,7 @@ public enum Separator {
   public Collection<String> splitEncoded(String compoundValue) {
     List<String> result = new ArrayList<String>();
     if (compoundValue != null) {
-      for (String val : compoundValue.split(quotedValue)) {
+      for (String val : valuePattern.split(compoundValue)) {
         result.add(decode(val));
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2cab3fc0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.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/TestSeparator.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/TestSeparator.java
index 0cda97c..27750f3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestSeparator.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/TestSeparator.java
@@ -34,7 +34,7 @@ public class TestSeparator {
 
   private static String villain = "Dr. Heinz Doofenshmirtz";
   private static String special =
-      ".   *   |   ?   +   \t   (   )   [   ]   {   }   ^   $  \\ \"";
+      ".   *   |   ?   +   \t   (   )   [   ]   {   }   ^   $  \\ \"  %";
 
   /**
    *
@@ -81,6 +81,12 @@ public class TestSeparator {
         Separator.VALUES, Separator.SPACE);
 
   }
+  @Test
+  public void testEncodedValues() {
+    testEncodeDecode("Double-escape %2$ and %9$ or %%2$ or %%3$, nor  %%%2$" +
+        "= no problem!",
+        Separator.QUALIFIERS, Separator.VALUES, Separator.SPACE, Separator.TAB);
+   }
 
   @Test
   public void testSplits() {


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


[15/50] [abbrv] hadoop git commit: MAPREDUCE-6688. Store job configurations in Timeline Service v2 (Varun Saxena via sjlee)

Posted by vr...@apache.org.
MAPREDUCE-6688. Store job configurations in Timeline Service v2 (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/91a9099c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/91a9099c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/91a9099c

Branch: refs/heads/YARN-2928
Commit: 91a9099cdaa2f08f03da766b24ead29dc6907891
Parents: 8c00fef
Author: Sangjin Lee <sj...@apache.org>
Authored: Tue May 3 09:19:36 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:08 2016 -0700

----------------------------------------------------------------------
 .../jobhistory/JobHistoryEventHandler.java      | 57 +++++++++++-
 .../mapreduce/v2/app/job/impl/JobImpl.java      |  2 +-
 .../mapreduce/jobhistory/JobSubmittedEvent.java | 38 +++++++-
 .../mapreduce/util/JobHistoryEventUtils.java    |  3 +
 .../mapred/TestMRTimelineEventHandling.java     | 92 +++++++++++++++++---
 .../org/apache/hadoop/mapred/UtilsForTests.java |  8 ++
 6 files changed, 181 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a9099c/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 528b450..887533d 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
@@ -1074,7 +1074,16 @@ public class JobHistoryEventHandler extends AbstractService
     entity.setId(jobId.toString());
     return entity;
   }
-  
+
+  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+      createJobEntity(JobId jobId) {
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
+        new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
+    entity.setId(jobId.toString());
+    entity.setType(MAPREDUCE_JOB_ENTITY_TYPE);
+    return entity;
+  }
+
   // create ApplicationEntity with job finished Metrics from HistoryEvent
   private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
       createAppEntityWithJobMetrics(HistoryEvent event, JobId jobId) {
@@ -1133,6 +1142,46 @@ public class JobHistoryEventHandler extends AbstractService
     return entity;
   }
 
+  private void publishConfigsOnJobSubmittedEvent(JobSubmittedEvent event,
+      JobId jobId) {
+    if (event.getJobConf() == null) {
+      return;
+    }
+    // Publish job configurations both as job and app entity.
+    // Configs are split into multiple entities if they exceed 100kb in size.
+    org.apache.hadoop.yarn.api.records.timelineservice.
+        TimelineEntity jobEntityForConfigs = createJobEntity(jobId);
+    ApplicationEntity appEntityForConfigs = new ApplicationEntity();
+    String appId = jobId.getAppId().toString();
+    appEntityForConfigs.setId(appId);
+    try {
+      int configSize = 0;
+      for (Map.Entry<String, String> entry : event.getJobConf()) {
+        int size = entry.getKey().length() + entry.getValue().length();
+        configSize += size;
+        if (configSize > JobHistoryEventUtils.ATS_CONFIG_PUBLISH_SIZE_BYTES) {
+          if (jobEntityForConfigs.getConfigs().size() > 0) {
+            timelineClient.putEntities(jobEntityForConfigs);
+            timelineClient.putEntities(appEntityForConfigs);
+            jobEntityForConfigs = createJobEntity(jobId);
+            appEntityForConfigs = new ApplicationEntity();
+            appEntityForConfigs.setId(appId);
+          }
+          configSize = size;
+        }
+        jobEntityForConfigs.addConfig(entry.getKey(), entry.getValue());
+        appEntityForConfigs.addConfig(entry.getKey(), entry.getValue());
+      }
+      if (configSize > 0) {
+        timelineClient.putEntities(jobEntityForConfigs);
+        timelineClient.putEntities(appEntityForConfigs);
+      }
+    } catch (IOException | YarnException e) {
+      LOG.error("Exception while publishing configs on JOB_SUBMITTED Event " +
+          " for the job : " + jobId, e);
+    }
+  }
+
   private void processEventForNewTimelineService(HistoryEvent event,
       JobId jobId, long timestamp) {
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity tEntity =
@@ -1252,8 +1301,12 @@ public class JobHistoryEventHandler extends AbstractService
     } catch (IOException | YarnException e) {
       LOG.error("Failed to process Event " + event.getEventType()
           + " for the job : " + jobId, e);
+      return;
+    }
+    if (event.getEventType() == EventType.JOB_SUBMITTED) {
+      // Publish configs after main job submitted event has been posted.
+      publishConfigsOnJobSubmittedEvent((JobSubmittedEvent)event, jobId);
     }
-
   }
 
   private void setSummarySlotSeconds(JobSummary summary, Counters allCounters) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a9099c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
index b7036a5..5127a43 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
@@ -1445,7 +1445,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
             job.conf.get(MRJobConfig.WORKFLOW_NAME, ""),
             job.conf.get(MRJobConfig.WORKFLOW_NODE_NAME, ""),
             getWorkflowAdjacencies(job.conf),
-            job.conf.get(MRJobConfig.WORKFLOW_TAGS, ""));
+            job.conf.get(MRJobConfig.WORKFLOW_TAGS, ""), job.conf);
         job.eventHandler.handle(new JobHistoryEvent(job.jobId, jse));
         //TODO JH Verify jobACLs, UserName via UGI?
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a9099c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
index 07edb58..7d05571 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import org.apache.avro.util.Utf8;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.security.authorize.AccessControlList;
@@ -41,6 +42,7 @@ import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
 @InterfaceStability.Unstable
 public class JobSubmittedEvent implements HistoryEvent {
   private JobSubmitted datum = new JobSubmitted();
+  private JobConf jobConf = null;
 
   /**
    * Create an event to record job submission
@@ -83,6 +85,31 @@ public class JobSubmittedEvent implements HistoryEvent {
         workflowAdjacencies, "");
   }
 
+  /**
+   * Create an event to record job submission.
+   * @param id The job Id of the job
+   * @param jobName Name of the job
+   * @param userName Name of the user who submitted the job
+   * @param submitTime Time of submission
+   * @param jobConfPath Path of the Job Configuration file
+   * @param jobACLs The configured acls for the job.
+   * @param jobQueueName The job-queue to which this job was submitted to
+   * @param workflowId The Id of the workflow
+   * @param workflowName The name of the workflow
+   * @param workflowNodeName The node name of the workflow
+   * @param workflowAdjacencies The adjacencies of the workflow
+   * @param workflowTags Comma-separated tags for the workflow
+   */
+  public JobSubmittedEvent(JobID id, String jobName, String userName,
+      long submitTime, String jobConfPath,
+      Map<JobACL, AccessControlList> jobACLs, String jobQueueName,
+      String workflowId, String workflowName, String workflowNodeName,
+      String workflowAdjacencies, String workflowTags) {
+    this(id, jobName, userName, submitTime, jobConfPath, jobACLs,
+        jobQueueName, workflowId, workflowName, workflowNodeName,
+        workflowAdjacencies, workflowTags, null);
+  }
+
     /**
      * Create an event to record job submission
      * @param id The job Id of the job
@@ -97,12 +124,13 @@ public class JobSubmittedEvent implements HistoryEvent {
      * @param workflowNodeName The node name of the workflow
      * @param workflowAdjacencies The adjacencies of the workflow
      * @param workflowTags Comma-separated tags for the workflow
+     * @param conf Job configuration
      */
     public JobSubmittedEvent(JobID id, String jobName, String userName,
         long submitTime, String jobConfPath,
         Map<JobACL, AccessControlList> jobACLs, String jobQueueName,
         String workflowId, String workflowName, String workflowNodeName,
-        String workflowAdjacencies, String workflowTags) {
+        String workflowAdjacencies, String workflowTags, JobConf conf) {
     datum.setJobid(new Utf8(id.toString()));
     datum.setJobName(new Utf8(jobName));
     datum.setUserName(new Utf8(userName));
@@ -132,6 +160,7 @@ public class JobSubmittedEvent implements HistoryEvent {
     if (workflowTags != null) {
       datum.setWorkflowTags(new Utf8(workflowTags));
     }
+    jobConf = conf;
   }
 
   JobSubmittedEvent() {}
@@ -208,7 +237,11 @@ public class JobSubmittedEvent implements HistoryEvent {
   }
   /** Get the event type */
   public EventType getEventType() { return EventType.JOB_SUBMITTED; }
-  
+
+  public JobConf getJobConf() {
+    return jobConf;
+  }
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();
@@ -234,5 +267,4 @@ public class JobSubmittedEvent implements HistoryEvent {
   public Set<TimelineMetric> getTimelineMetrics() {
     return null;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a9099c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java
index 225d517..35d066c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/JobHistoryEventUtils.java
@@ -36,6 +36,9 @@ public final class JobHistoryEventUtils {
   private JobHistoryEventUtils() {
   }
 
+  // Number of bytes of config which can be published in one shot to ATSv2.
+  public static final int ATS_CONFIG_PUBLISH_SIZE_BYTES = 10 * 1024;
+
   public static JsonNode countersToJSON(Counters counters) {
     ObjectMapper mapper = new ObjectMapper();
     ArrayNode nodes = mapper.createArrayNode();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a9099c/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 5915a43..fde9e64 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
@@ -26,7 +26,9 @@ import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
 import java.util.EnumSet;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
@@ -55,6 +57,8 @@ import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.collect.Sets;
+
 public class TestMRTimelineEventHandling {
 
   private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_collector";
@@ -191,8 +195,17 @@ public class TestMRTimelineEventHandling {
       Path inDir = new Path("input");
       Path outDir = new Path("output");
       LOG.info("Run 1st job which should be successful.");
+      JobConf successConf = new JobConf(conf);
+      successConf.set("dummy_conf1",
+          UtilsForTests.createConfigValue(51 * 1024));
+      successConf.set("dummy_conf2",
+          UtilsForTests.createConfigValue(51 * 1024));
+      successConf.set("huge_dummy_conf1",
+          UtilsForTests.createConfigValue(101 * 1024));
+      successConf.set("huge_dummy_conf2",
+          UtilsForTests.createConfigValue(101 * 1024));
       RunningJob job =
-          UtilsForTests.runJobSucceed(new JobConf(conf), inDir, outDir);
+          UtilsForTests.runJobSucceed(successConf, inDir, outDir);
       Assert.assertEquals(JobStatus.SUCCEEDED,
           job.getJobStatus().getState().getValue());
 
@@ -270,7 +283,11 @@ public class TestMRTimelineEventHandling {
     Assert.assertTrue("jobEventFilePath: " + jobEventFilePath +
         " does not exist.",
         jobEventFile.exists());
-    verifyMetricsWhenEvent(jobEventFile, EventType.JOB_FINISHED.name());
+    verifyEntity(jobEventFile, EventType.JOB_FINISHED.name(),
+        true, false, null);
+    Set<String> cfgsToCheck = Sets.newHashSet("dummy_conf1", "dummy_conf2",
+        "huge_dummy_conf1", "huge_dummy_conf2");
+    verifyEntity(jobEventFile, null, false, true, cfgsToCheck);
 
     // for this test, we expect MR job metrics are published in YARN_APPLICATION
     String outputAppDir = basePath + "/YARN_APPLICATION/";
@@ -290,7 +307,8 @@ public class TestMRTimelineEventHandling {
         "appEventFilePath: " + appEventFilePath +
         " does not exist.",
         appEventFile.exists());
-    verifyMetricsWhenEvent(appEventFile, null);
+    verifyEntity(appEventFile, null, true, false, null);
+    verifyEntity(appEventFile, null, false, true, cfgsToCheck);
 
     // check for task event file
     String outputDirTask = basePath + "/MAPREDUCE_TASK/";
@@ -307,7 +325,8 @@ public class TestMRTimelineEventHandling {
     Assert.assertTrue("taskEventFileName: " + taskEventFilePath +
         " does not exist.",
         taskEventFile.exists());
-    verifyMetricsWhenEvent(taskEventFile, EventType.TASK_FINISHED.name());
+    verifyEntity(taskEventFile, EventType.TASK_FINISHED.name(),
+        true, false, null);
     
     // check for task attempt event file
     String outputDirTaskAttempt = basePath + "/MAPREDUCE_TASK_ATTEMPT/";
@@ -324,17 +343,30 @@ public class TestMRTimelineEventHandling {
     File taskAttemptEventFile = new File(taskAttemptEventFilePath);
     Assert.assertTrue("taskAttemptEventFileName: " + taskAttemptEventFilePath +
         " does not exist.", taskAttemptEventFile.exists());
-    verifyMetricsWhenEvent(taskAttemptEventFile,
-        EventType.MAP_ATTEMPT_FINISHED.name());
+    verifyEntity(taskAttemptEventFile, EventType.MAP_ATTEMPT_FINISHED.name(),
+        true, false, null);
   }
 
-  private void verifyMetricsWhenEvent(File entityFile, String eventId)
+  /**
+   * Verifies entity by reading the entity file written via FS impl.
+   * @param entityFile File to be read.
+   * @param eventId Event to be checked.
+   * @param chkMetrics If event is not null, this flag determines if metrics
+   *     exist when the event is encountered. If event is null, we merely check
+   *     if metrics exist in the entity file.
+   * @param chkCfg If event is not null, this flag determines if configs
+   *     exist when the event is encountered. If event is null, we merely check
+   *     if configs exist in the entity file.
+   * @param cfgsToVerify a set of configs which should exist in the entity file.
+   * @throws IOException
+   */
+  private void verifyEntity(File entityFile, String eventId,
+      boolean chkMetrics, boolean chkCfg, Set<String> cfgsToVerify)
       throws IOException {
     BufferedReader reader = null;
     String strLine;
     try {
       reader = new BufferedReader(new FileReader(entityFile));
-      boolean jobMetricsFoundForAppEntity = false;
       while ((strLine = reader.readLine()) != null) {
         if (strLine.trim().length() > 0) {
           org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
@@ -344,23 +376,57 @@ public class TestMRTimelineEventHandling {
           if (eventId == null) {
             // Job metrics are published without any events for
             // ApplicationEntity. There is also possibility that some other
-            // ApplicationEntity is published without events, hence loop all
-            if (entity.getEvents().size() == 0) {
-              jobMetricsFoundForAppEntity = entity.getMetrics().size() > 0;
-              if (jobMetricsFoundForAppEntity) {
+            // ApplicationEntity is published without events, hence loop till
+            // its found. Same applies to configs.
+            if (chkMetrics && entity.getMetrics().size() > 0) {
+              return;
+            }
+            if (chkCfg && entity.getConfigs().size() > 0) {
+              if (cfgsToVerify == null) {
                 return;
+              } else {
+                // Have configs to verify. Keep on removing configs from the set
+                // of configs to verify as they are found. When the all the
+                // entities have been looped through, we will check if the set
+                // is empty or not(indicating if all configs have been found or
+                // not).
+                for (Iterator<String> itr =
+                    cfgsToVerify.iterator(); itr.hasNext();) {
+                  String config = itr.next();
+                  if (entity.getConfigs().containsKey(config)) {
+                    itr.remove();
+                  }
+                }
+                // All the required configs have been verified, so return.
+                if (cfgsToVerify.isEmpty()) {
+                  return;
+                }
               }
             }
           } else {
             for (TimelineEvent event : entity.getEvents()) {
               if (event.getId().equals(eventId)) {
-                assertTrue(entity.getMetrics().size() > 0);
+                if (chkMetrics) {
+                  assertTrue(entity.getMetrics().size() > 0);
+                }
+                if (chkCfg) {
+                  assertTrue(entity.getConfigs().size() > 0);
+                  if (cfgsToVerify != null) {
+                    for (String cfg : cfgsToVerify) {
+                      assertTrue(entity.getConfigs().containsKey(cfg));
+                    }
+                  }
+                }
                 return;
               }
             }
           }
         }
       }
+      if (cfgsToVerify != null) {
+        assertTrue(cfgsToVerify.isEmpty());
+        return;
+      }
       fail("Expected event : " + eventId + " not found in the file "
           + entityFile);
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/91a9099c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
index cb494db..2fb6828 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/UtilsForTests.java
@@ -156,6 +156,14 @@ public class UtilsForTests {
     return buf.toString();
   }
 
+  public static String createConfigValue(int msgSize) {
+    StringBuilder sb = new StringBuilder(msgSize);
+    for (int i=0; i<msgSize; i++) {
+      sb.append('a');
+    }
+    return sb.toString();
+  }
+
   public static String safeGetCanonicalPath(File f) {
     try {
       String s = f.getCanonicalPath();


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


[48/50] [abbrv] hadoop git commit: YARN-5243. fix several rebase and other miscellaneous issues before merge. (Sangjin Lee via Varun Saxena)

Posted by vr...@apache.org.
YARN-5243. fix several rebase and other miscellaneous issues before merge. (Sangjin Lee 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/a8779bec
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a8779bec
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a8779bec

Branch: refs/heads/YARN-2928
Commit: a8779becb67d025d93e4509c380ef3022ed40793
Parents: 6c22a6a
Author: Varun Saxena <va...@apache.org>
Authored: Wed Jun 15 22:10:56 2016 +0530
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:16 2016 -0700

----------------------------------------------------------------------
 .../jobhistory/JobHistoryEventHandler.java      | 42 ++++++++++----------
 .../v2/app/rm/RMContainerAllocator.java         |  3 +-
 .../hadoop/mapreduce/jobhistory/TestEvents.java |  2 +-
 .../mapreduce/jobhistory/AMStartedEvent.java    |  2 +-
 .../mapreduce/jobhistory/HistoryEvent.java      |  2 +-
 .../mapreduce/jobhistory/JobFinishedEvent.java  |  2 +-
 .../jobhistory/JobInfoChangeEvent.java          |  2 +-
 .../mapreduce/jobhistory/JobInitedEvent.java    |  2 +-
 .../jobhistory/JobPriorityChangeEvent.java      |  2 +-
 .../jobhistory/JobQueueChangeEvent.java         |  2 +-
 .../jobhistory/JobStatusChangedEvent.java       |  2 +-
 .../mapreduce/jobhistory/JobSubmittedEvent.java |  2 +-
 .../JobUnsuccessfulCompletionEvent.java         |  2 +-
 .../jobhistory/MapAttemptFinishedEvent.java     |  2 +-
 .../jobhistory/NormalizedResourceEvent.java     |  2 +-
 .../jobhistory/ReduceAttemptFinishedEvent.java  |  2 +-
 .../jobhistory/TaskAttemptFinishedEvent.java    |  2 +-
 .../jobhistory/TaskAttemptStartedEvent.java     |  2 +-
 .../TaskAttemptUnsuccessfulCompletionEvent.java |  2 +-
 .../mapreduce/jobhistory/TaskFailedEvent.java   |  2 +-
 .../mapreduce/jobhistory/TaskFinishedEvent.java |  2 +-
 .../mapreduce/jobhistory/TaskStartedEvent.java  |  2 +-
 .../mapreduce/jobhistory/TaskUpdatedEvent.java  |  2 +-
 .../mapred/TestMRTimelineEventHandling.java     | 28 ++++++-------
 .../apache/hadoop/test/MapredTestDriver.java    |  2 +-
 .../dev-support/findbugs-exclude.xml            |  5 ---
 .../distributedshell/ApplicationMaster.java     |  2 +-
 .../distributedshell/TestDistributedShell.java  |  2 +-
 .../client/api/impl/TimelineClientImpl.java     |  4 +-
 .../src/main/resources/yarn-default.xml         |  2 +-
 .../yarn/util/TestTimelineServiceHelper.java    | 20 +++++-----
 .../impl/pb/NodeHeartbeatRequestPBImpl.java     |  4 +-
 .../hadoop/yarn/server/nodemanager/Context.java |  2 +-
 .../yarn/server/nodemanager/NodeManager.java    | 13 ++++--
 .../application/ApplicationImpl.java            |  2 +-
 .../monitor/ContainersMonitorImpl.java          |  6 +--
 .../nodemanager/webapp/TestNMWebServer.java     |  2 -
 .../hadoop-yarn-server-resourcemanager/pom.xml  |  2 +-
 .../resourcemanager/RMActiveServiceContext.java | 32 +--------------
 .../yarn/server/resourcemanager/RMContext.java  |  4 +-
 .../server/resourcemanager/RMContextImpl.java   |  2 +-
 .../server/resourcemanager/ResourceManager.java | 34 +++++-----------
 .../server/resourcemanager/rmapp/RMApp.java     | 14 +++++--
 .../server/resourcemanager/rmapp/RMAppImpl.java |  8 ++++
 44 files changed, 123 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 887533d..a09c17b 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
@@ -137,7 +137,7 @@ public class JobHistoryEventHandler extends AbstractService
   protected volatile boolean forceJobCompletion = false;
 
   protected TimelineClient timelineClient;
-  
+
   private boolean timelineServiceV2Enabled = false;
 
   private static String MAPREDUCE_JOB_ENTITY_TYPE = "MAPREDUCE_JOB";
@@ -263,16 +263,16 @@ public class JobHistoryEventHandler extends AbstractService
             MRJobConfig.MR_AM_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD,
             MRJobConfig.DEFAULT_MR_AM_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD);
 
-    // TODO replace MR specific configurations on timeline service with getting 
-    // configuration from RM through registerApplicationMaster() in 
-    // ApplicationMasterProtocol with return value for timeline service 
+    // TODO replace MR specific configurations on timeline service with getting
+    // configuration from RM through registerApplicationMaster() in
+    // ApplicationMasterProtocol with return value for timeline service
     // 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)) {
       LOG.info("Emitting job history data to the timeline service is enabled");
       if (YarnConfiguration.timelineServiceEnabled(conf)) {
 
-        timelineClient = 
+        timelineClient =
             ((MRAppMaster.RunningAppContext)context).getTimelineClient();
         timelineClient.init(conf);
         timelineServiceV2Enabled =
@@ -1062,11 +1062,11 @@ public class JobHistoryEventHandler extends AbstractService
           + "Server", ex);
     }
   }
-  
-  // create JobEntity from HistoryEvent with adding other info, like: 
+
+  // create JobEntity from HistoryEvent with adding other info, like:
   // jobId, timestamp and entityType.
-  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity 
-      createJobEntity(HistoryEvent event, long timestamp, JobId jobId, 
+  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+      createJobEntity(HistoryEvent event, long timestamp, JobId jobId,
       String entityType, boolean setCreatedTime) {
 
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
@@ -1093,16 +1093,16 @@ public class JobHistoryEventHandler extends AbstractService
     return entity;
   }
 
-  // create BaseEntity from HistoryEvent with adding other info, like: 
+  // create BaseEntity from HistoryEvent with adding other info, like:
   // timestamp and entityType.
-  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity 
+  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
       createBaseEntity(HistoryEvent event, long timestamp, String entityType,
       boolean setCreatedTime) {
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent tEvent =
         event.toTimelineEvent();
     tEvent.setTimestamp(timestamp);
-    
-    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity = 
+
+    org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
         new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity();
     entity.addEvent(tEvent);
     entity.setType(entityType);
@@ -1115,10 +1115,10 @@ public class JobHistoryEventHandler extends AbstractService
     }
     return entity;
   }
-  
-  // create TaskEntity from HistoryEvent with adding other info, like: 
+
+  // create TaskEntity from HistoryEvent with adding other info, like:
   // taskId, jobId, timestamp, entityType and relatedJobEntity.
-  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity 
+  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
       createTaskEntity(HistoryEvent event, long timestamp, String taskId,
       String entityType, String relatedJobEntity, JobId jobId,
       boolean setCreatedTime) {
@@ -1128,12 +1128,12 @@ public class JobHistoryEventHandler extends AbstractService
     entity.addIsRelatedToEntity(relatedJobEntity, jobId.toString());
     return entity;
   }
-  
-  // create TaskAttemptEntity from HistoryEvent with adding other info, like: 
+
+  // create TaskAttemptEntity from HistoryEvent with adding other info, like:
   // timestamp, taskAttemptId, entityType, relatedTaskEntity and taskId.
-  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity 
-      createTaskAttemptEntity(HistoryEvent event, long timestamp, 
-      String taskAttemptId, String entityType, String relatedTaskEntity, 
+  private org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity
+      createTaskAttemptEntity(HistoryEvent event, long timestamp,
+      String taskAttemptId, String entityType, String relatedTaskEntity,
       String taskId, boolean setCreatedTime) {
     org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity =
         createBaseEntity(event, timestamp, entityType, setCreatedTime);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
index e1fc84f..ea2046b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
@@ -822,8 +822,9 @@ public class RMContainerAllocator extends RMContainerRequestor
 
     handleUpdatedNodes(response);
     handleJobPriorityChange(response);
+    // handle receiving the timeline collector address for this app
     String collectorAddr = response.getCollectorAddr();
-    MRAppMaster.RunningAppContext appContext = 
+    MRAppMaster.RunningAppContext appContext =
         (MRAppMaster.RunningAppContext)this.getContext();
     if (collectorAddr != null && !collectorAddr.isEmpty()
         && appContext.getTimelineClient() != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
index f5f4a3c..ac510b3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
@@ -418,7 +418,7 @@ public class TestEvents {
     public Set<TimelineMetric> getTimelineMetrics() {
       return null;
     }
-    
+
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
index 82db9f3..a1447c5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java
@@ -170,7 +170,7 @@ public class AMStartedEvent implements HistoryEvent {
   public EventType getEventType() {
     return EventType.AM_STARTED;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java
index cdafe46..1d59ebe 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryEvent.java
@@ -41,7 +41,7 @@ public interface HistoryEvent {
 
   /** Set the Avro datum wrapped by this. */
   void setDatum(Object datum);
-  
+
   /** Map HistoryEvent to TimelineEvent */
   TimelineEvent toTimelineEvent();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
index 2efbc57..76d6897 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobFinishedEvent.java
@@ -139,7 +139,7 @@ public class JobFinishedEvent  implements HistoryEvent {
   public Counters getReduceCounters() {
     return reduceCounters;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
index eb9ad0d..f5941aa 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInfoChangeEvent.java
@@ -66,7 +66,7 @@ public class JobInfoChangeEvent implements HistoryEvent {
   public EventType getEventType() {
     return EventType.JOB_INFO_CHANGED;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
index d91208c..784267f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobInitedEvent.java
@@ -77,7 +77,7 @@ public class JobInitedEvent implements HistoryEvent {
   }
   /** Get whether the job's map and reduce stages were combined */
   public boolean getUberized() { return datum.getUberized(); }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
index ac4943e..1616dd1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobPriorityChangeEvent.java
@@ -66,7 +66,7 @@ public class JobPriorityChangeEvent implements HistoryEvent {
   public EventType getEventType() {
     return EventType.JOB_PRIORITY_CHANGED;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java
index b12006b..66f3781 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobQueueChangeEvent.java
@@ -64,7 +64,7 @@ public class JobQueueChangeEvent implements HistoryEvent {
     }
     return null;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
index 0b33b4f..0963b45 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobStatusChangedEvent.java
@@ -62,7 +62,7 @@ public class JobStatusChangedEvent implements HistoryEvent {
   public EventType getEventType() {
     return EventType.JOB_STATUS_CHANGED;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
index 7d05571..e394f5b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobSubmittedEvent.java
@@ -259,7 +259,7 @@ public class JobSubmittedEvent implements HistoryEvent {
     tEvent.addInfo("WORKFLOW_ADJACENCIES",
         getWorkflowAdjacencies());
     tEvent.addInfo("WORKFLOW_TAGS", getWorkflowTags());
-    
+
     return tEvent;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
index 5d3c45f..ce6fa32 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobUnsuccessfulCompletionEvent.java
@@ -123,7 +123,7 @@ public class JobUnsuccessfulCompletionEvent implements HistoryEvent {
     final CharSequence diagnostics = datum.getDiagnostics();
     return diagnostics == null ? NODIAGS : diagnostics.toString();
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
index 1d28335..3121c4e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/MapAttemptFinishedEvent.java
@@ -247,5 +247,5 @@ public class MapAttemptFinishedEvent implements HistoryEvent {
         .countersToTimelineMetric(getCounters(), finishTime);
     return metrics;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java
index d90ff93..7adae23 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/NormalizedResourceEvent.java
@@ -76,7 +76,7 @@ public class NormalizedResourceEvent implements HistoryEvent {
   public void setDatum(Object datum) {
     throw new UnsupportedOperationException("Not a seriable object");
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
index 5ddd839..9c0f09b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/ReduceAttemptFinishedEvent.java
@@ -228,7 +228,7 @@ public class ReduceAttemptFinishedEvent implements HistoryEvent {
   public int[] getPhysMemKbytes() {
     return physMemKbytes;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
index 30d2b09..a931ca2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptFinishedEvent.java
@@ -141,7 +141,7 @@ public class TaskAttemptFinishedEvent  implements HistoryEvent {
            ? EventType.MAP_ATTEMPT_FINISHED
            : EventType.REDUCE_ATTEMPT_FINISHED;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
index 99213fd..d09d5ca 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java
@@ -136,7 +136,7 @@ public class TaskAttemptStartedEvent implements HistoryEvent {
     }
     return null;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
index 8826c96..1732d91 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
@@ -251,7 +251,7 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
   public int[] getPhysMemKbytes() {
     return physMemKbytes;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
index 9a29b81..d14350d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
@@ -142,7 +142,7 @@ public class TaskFailedEvent implements HistoryEvent {
   public EventType getEventType() {
     return EventType.TASK_FAILED;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
index fd003cc..0bc4383 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFinishedEvent.java
@@ -130,7 +130,7 @@ public class TaskFinishedEvent implements HistoryEvent {
     tEvent.addInfo("FINISH_TIME", getFinishTime());
     tEvent.addInfo("STATUS", TaskStatus.State.SUCCEEDED.toString());
     tEvent.addInfo("SUCCESSFUL_TASK_ATTEMPT_ID",
-        getSuccessfulTaskAttemptId() == null ? "" : 
+        getSuccessfulTaskAttemptId() == null ? "" :
             getSuccessfulTaskAttemptId().toString());
     return tEvent;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
index 6792ce7..9d2fc0e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskStartedEvent.java
@@ -76,7 +76,7 @@ public class TaskStartedEvent implements HistoryEvent {
   public EventType getEventType() {
     return EventType.TASK_STARTED;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java
index 0722207..010129d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskUpdatedEvent.java
@@ -62,7 +62,7 @@ public class TaskUpdatedEvent implements HistoryEvent {
   public EventType getEventType() {
     return EventType.TASK_UPDATED;
   }
-  
+
   @Override
   public TimelineEvent toTimelineEvent() {
     TimelineEvent tEvent = new TimelineEvent();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 27ce48c..6b9f27e 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
@@ -65,7 +65,7 @@ public class TestMRTimelineEventHandling {
   private static final String TIMELINE_AUX_SERVICE_NAME = "timeline_collector";
   private static final Log LOG =
     LogFactory.getLog(TestMRTimelineEventHandling.class);
-  
+
   @Test
   public void testTimelineServiceStartInMiniCluster() throws Exception {
     Configuration conf = new YarnConfiguration();
@@ -168,7 +168,7 @@ public class TestMRTimelineEventHandling {
       }
     }
   }
-  
+
   @Test
   public void testMRNewTimelineServiceEventHandling() throws Exception {
     LOG.info("testMRNewTimelineServiceEventHandling start.");
@@ -184,7 +184,7 @@ public class TestMRTimelineEventHandling {
     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);
 
     MiniMRYarnCluster cluster = null;
@@ -215,9 +215,9 @@ public class TestMRTimelineEventHandling {
       YarnClient yarnClient = YarnClient.createYarnClient();
       yarnClient.init(new Configuration(cluster.getConfig()));
       yarnClient.start();
-      EnumSet<YarnApplicationState> appStates = 
+      EnumSet<YarnApplicationState> appStates =
           EnumSet.allOf(YarnApplicationState.class);
-      
+
       ApplicationId firstAppId = null;
       List<ApplicationReport> apps = yarnClient.getApplications(appStates);
       Assert.assertEquals(apps.size(), 1);
@@ -230,7 +230,7 @@ public class TestMRTimelineEventHandling {
       job = UtilsForTests.runJobFail(new JobConf(conf), inDir, outDir);
       Assert.assertEquals(JobStatus.FAILED,
           job.getJobStatus().getState().getValue());
-      
+
       apps = yarnClient.getApplications(appStates);
       Assert.assertEquals(apps.size(), 2);
 
@@ -250,10 +250,10 @@ public class TestMRTimelineEventHandling {
       if(testRootFolder.isDirectory()) {
         FileUtils.deleteDirectory(testRootFolder);
       }
-      
+
     }
   }
-  
+
   private void checkNewTimelineEvent(ApplicationId appId,
       ApplicationReport appReport) throws IOException {
     String tmpRoot =
@@ -261,7 +261,7 @@ public class TestMRTimelineEventHandling {
             + "/entities/";
 
     File tmpRootFolder = new File(tmpRoot);
-    
+
     Assert.assertTrue(tmpRootFolder.isDirectory());
     String basePath = tmpRoot + YarnConfiguration.DEFAULT_RM_CLUSTER_ID +
         "/" + UserGroupInformation.getCurrentUser().getShortUserName() +
@@ -319,7 +319,7 @@ public class TestMRTimelineEventHandling {
     Assert.assertTrue("Task output directory: " + outputDirTask +
         " does not exist.",
         taskFolder.isDirectory());
-    
+
     String taskEventFileName = appId.toString().replaceAll("application", "task")
         + "_m_000000" + FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
 
@@ -330,15 +330,15 @@ public class TestMRTimelineEventHandling {
         taskEventFile.exists());
     verifyEntity(taskEventFile, EventType.TASK_FINISHED.name(),
         true, false, null);
-    
+
     // check for task attempt event file
     String outputDirTaskAttempt = basePath + "/MAPREDUCE_TASK_ATTEMPT/";
     File taskAttemptFolder = new File(outputDirTaskAttempt);
-    Assert.assertTrue("TaskAttempt output directory: " + outputDirTaskAttempt + 
+    Assert.assertTrue("TaskAttempt output directory: " + outputDirTaskAttempt +
         " does not exist.", taskAttemptFolder.isDirectory());
-    
+
     String taskAttemptEventFileName = appId.toString().replaceAll(
-        "application", "attempt") + "_m_000000_0" + 
+        "application", "attempt") + "_m_000000_0" +
         FileSystemTimelineWriterImpl.TIMELINE_SERVICE_STORAGE_EXTENSION;
 
     String taskAttemptEventFilePath = outputDirTaskAttempt +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/test/MapredTestDriver.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/test/MapredTestDriver.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/test/MapredTestDriver.java
index 4c42bd3..76198b8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/test/MapredTestDriver.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/test/MapredTestDriver.java
@@ -93,7 +93,7 @@ public class MapredTestDriver {
       pgd.addClass("timelineperformance", TimelineServicePerformance.class,
                    "A job that launches mappers to test timline service " +
                    "performance.");
-      pgd.addClass("nnbench", NNBench.class, 
+      pgd.addClass("nnbench", NNBench.class,
           "A benchmark that stresses the namenode w/ MR.");
       pgd.addClass("nnbenchWithoutMR", NNBenchWithoutMR.class,
           "A benchmark that stresses the namenode w/o MR.");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 08c6ba2..7c19c5e 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -552,9 +552,4 @@
     <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/a8779bec/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 fb2ae6f..394d544 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
@@ -631,7 +631,7 @@ public class ApplicationMaster {
             DSEvent.DS_APP_ATTEMPT_START);
       } else {
         publishApplicationAttemptEvent(timelineClient, appAttemptID.toString(),
-          DSEvent.DS_APP_ATTEMPT_END, domainId, appSubmitterUgi);
+            DSEvent.DS_APP_ATTEMPT_START, domainId, appSubmitterUgi);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 74da919..a01d21b 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
@@ -149,7 +149,7 @@ public class TestDistributedShell {
     // Enable ContainersMonitorImpl
     conf.set(YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR,
         LinuxResourceCalculatorPlugin.class.getName());
-    conf.set(YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE, 
+    conf.set(YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE,
         ProcfsBasedProcessTree.class.getName());
     conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, true);
     conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 ef8838e..478efc4 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
@@ -495,8 +495,8 @@ public class TimelineClientImpl extends TimelineClient {
     }
     if (resp == null ||
         resp.getClientResponseStatus() != ClientResponse.Status.OK) {
-      String msg = "Response from the timeline server is " + 
-          ((resp == null) ? "null": 
+      String msg = "Response from the timeline server is " +
+          ((resp == null) ? "null":
           "not successful," + " HTTP error code: " + resp.getStatus()
           + ", Server response:\n" + resp.getEntity(String.class));
       LOG.error(msg);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 a54cc75..5b4fe10 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
@@ -784,7 +784,7 @@
 
   <property>
     <description>The setting that controls whether yarn system metrics is
-    published to the Timeline server (version one) or not, by RM. 
+    published to the Timeline server (version one) or not, by RM.
     This configuration is now deprecated in favor of
     yarn.system-metrics-publisher.enabled.</description>
     <name>yarn.resourcemanager.system-metrics-publisher.enabled</name>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java
index f852df0..34b9497 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestTimelineServiceHelper.java
@@ -27,49 +27,49 @@ import org.junit.Assert;
 import org.junit.Test;
 
 public class TestTimelineServiceHelper {
-  
+
   @Test
   public void testMapCastToHashMap() {
 
     // Test null map be casted to null
     Map<String, String> nullMap = null;
     Assert.assertNull(TimelineServiceHelper.mapCastToHashMap(nullMap));
-    
+
     // Test empty hashmap be casted to a empty hashmap
     Map<String, String> emptyHashMap = new HashMap<String, String>();
     Assert.assertEquals(TimelineServiceHelper.mapCastToHashMap(emptyHashMap).size(), 0);
-    
+
     // Test empty non-hashmap be casted to a empty hashmap
     Map<String, String> emptyTreeMap = new TreeMap<String, String>();
     Assert.assertEquals(TimelineServiceHelper.mapCastToHashMap(emptyTreeMap).size(), 0);
-    
+
     // Test non-empty hashmap be casted to hashmap correctly
     Map<String, String> firstHashMap = new HashMap<String, String>();
     String key = "KEY";
     String value = "VALUE";
     firstHashMap.put(key, value);
     Assert.assertEquals(TimelineServiceHelper.mapCastToHashMap(firstHashMap), firstHashMap);
-    
+
     // Test non-empty non-hashmap is casted correctly.
     Map<String, String> firstTreeMap = new TreeMap<String, String>();
     firstTreeMap.put(key, value);
-    HashMap<String, String> alternateHashMap = 
+    HashMap<String, String> alternateHashMap =
         TimelineServiceHelper.mapCastToHashMap(firstTreeMap);
     Assert.assertEquals(firstTreeMap.size(), alternateHashMap.size());
     Assert.assertEquals(alternateHashMap.get(key), value);
-    
+
     // Test complicated hashmap be casted correctly
     Map<String, Set<String>> complicatedHashMap = new HashMap<String, Set<String>>();
     Set<String> hashSet = new HashSet<String>();
     hashSet.add(value);
     complicatedHashMap.put(key, hashSet);
-    Assert.assertEquals(TimelineServiceHelper.mapCastToHashMap(complicatedHashMap), 
+    Assert.assertEquals(TimelineServiceHelper.mapCastToHashMap(complicatedHashMap),
         complicatedHashMap);
-    
+
     // Test complicated non-hashmap get casted correctly
     Map<String, Set<String>> complicatedTreeMap = new TreeMap<String, Set<String>>();
     complicatedTreeMap.put(key, hashSet);
-    Assert.assertEquals(TimelineServiceHelper.mapCastToHashMap(complicatedTreeMap).get(key), 
+    Assert.assertEquals(TimelineServiceHelper.mapCastToHashMap(complicatedTreeMap).get(key),
         hashSet);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 066abfc..912a7a3 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
@@ -59,7 +59,7 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
   private List<LogAggregationReport> logAggregationReportsForApps = null;
 
   Map<ApplicationId, String> registeredCollectors = null;
-  
+
   public NodeHeartbeatRequestPBImpl() {
     builder = NodeHeartbeatRequestProto.newBuilder();
   }
@@ -157,7 +157,7 @@ public class NodeHeartbeatRequestPBImpl extends NodeHeartbeatRequest {
       LogAggregationReport value) {
     return ((LogAggregationReportPBImpl) value).getProto();
   }
-  
+
   private void addRegisteredCollectorsToProto() {
     maybeInitBuilder();
     builder.clearRegisteredCollectors();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 e53c0b0..131eaa3 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
@@ -102,7 +102,7 @@ public interface Context {
   NMStateStoreService getNMStateStore();
 
   boolean getDecommissioned();
-  
+
   Configuration getConf();
 
   void setDecommissioned(boolean isDecommissioned);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 939306d..05905b8 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
@@ -464,9 +464,9 @@ public class NodeManager extends CompositeService
   public static class NMContext implements Context {
 
     private NodeId nodeId = null;
-    
+
     private Configuration conf = null;
-    
+
     protected final ConcurrentMap<ApplicationId, Application> applications =
         new ConcurrentHashMap<ApplicationId, Application>();
 
@@ -544,7 +544,7 @@ public class NodeManager extends CompositeService
     public ConcurrentMap<ApplicationId, Application> getApplications() {
       return this.applications;
     }
-    
+
     @Override
     public Configuration getConf() {
       return this.conf;
@@ -677,7 +677,12 @@ public class NodeManager extends CompositeService
 
     public void addRegisteredCollectors(
         Map<ApplicationId, String> newRegisteredCollectors) {
-      this.registeredCollectors.putAll(newRegisteredCollectors);
+      if (registeredCollectors != null) {
+        this.registeredCollectors.putAll(newRegisteredCollectors);
+      } else {
+        LOG.warn("collectors are added when the registered collectors are " +
+            "initialized");
+      }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 22779bb..5a5d30c 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
@@ -184,7 +184,7 @@ public class ApplicationImpl implements Application {
   public ApplicationId getAppId() {
     return appId;
   }
-  
+
   @Override
   public ApplicationState getApplicationState() {
     this.readLock.lock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 6df7211..4167ece 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
@@ -243,7 +243,6 @@ public class ContainersMonitorImpl extends AbstractService implements
         ;
       }
     }
-    
     super.serviceStop();
   }
 
@@ -422,7 +421,6 @@ public class ContainersMonitorImpl extends AbstractService implements
             .entrySet()) {
           ContainerId containerId = entry.getKey();
           ProcessTreeInfo ptInfo = entry.getValue();
-          
           try {
             String pId = ptInfo.getPID();
 
@@ -478,7 +476,7 @@ public class ContainersMonitorImpl extends AbstractService implements
 
             float cpuUsageTotalCoresPercentage = cpuUsagePercentPerCore /
                 resourceCalculatorPlugin.getNumProcessors();
-            
+
             // Multiply by 1000 to avoid losing data when converting to int
             int milliVcoresUsed = (int) (cpuUsageTotalCoresPercentage * 1000
                 * maxVCoresAllottedForContainers /nodeCpuPercentageForYARN);
@@ -514,7 +512,7 @@ public class ContainersMonitorImpl extends AbstractService implements
                   containerMetricsUnregisterDelayMs).recordCpuUsage
                   ((int)cpuUsagePercentPerCore, milliVcoresUsed);
             }
-            
+
             boolean isMemoryOverLimit = false;
             String msg = "";
             int containerExitStatus = ContainerExitStatus.INVALID;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 9923f40..41037f7 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
@@ -111,7 +111,6 @@ public class TestNMWebServer {
         return true;
       }
     };
-    
     conf.set(YarnConfiguration.NM_LOCAL_DIRS, testRootDir.getAbsolutePath());
     conf.set(YarnConfiguration.NM_LOG_DIRS, testLogDir.getAbsolutePath());
     NodeHealthCheckerService healthChecker = createNodeHealthCheckerService(conf);
@@ -175,7 +174,6 @@ public class TestNMWebServer {
         return true;
       }
     };
-    
     conf.set(YarnConfiguration.NM_LOCAL_DIRS, testRootDir.getAbsolutePath());
     conf.set(YarnConfiguration.NM_LOG_DIRS, testLogDir.getAbsolutePath());
     NodeHealthCheckerService healthChecker = createNodeHealthCheckerService(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
index 7b64ba0..834dc3d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml
@@ -177,7 +177,7 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.apache.hadoop</groupId>	
+      <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-server-timelineservice</artifactId>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
index 12daa95..caa0ff13 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
@@ -30,10 +30,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
-import org.apache.hadoop.yarn.server.resourcemanager.ahs.RMApplicationHistoryWriter;
-import org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher;
-import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMDelegatedNodeLabelsUpdater;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.placement.PlacementManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.NullRMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
@@ -95,8 +93,6 @@ public class RMActiveServiceContext {
   private NodesListManager nodesListManager;
   private ResourceTrackerService resourceTrackerService;
   private ApplicationMasterService applicationMasterService;
-  private RMApplicationHistoryWriter rmApplicationHistoryWriter;
-  private SystemMetricsPublisher systemMetricsPublisher;
   private RMTimelineCollectorManager timelineCollectorManager;
 
   private RMNodeLabelsManager nodeLabelManager;
@@ -375,12 +371,6 @@ public class RMActiveServiceContext {
 
   @Private
   @Unstable
-  public RMApplicationHistoryWriter getRMApplicationHistoryWriter() {
-    return rmApplicationHistoryWriter;
-  }
-
-  @Private
-  @Unstable
   public RMTimelineCollectorManager getRMTimelineCollectorManager() {
     return timelineCollectorManager;
   }
@@ -394,26 +384,6 @@ public class RMActiveServiceContext {
 
   @Private
   @Unstable
-  public void setSystemMetricsPublisher(
-      SystemMetricsPublisher metricsPublisher) {
-    this.systemMetricsPublisher = metricsPublisher;
-  }
-
-  @Private
-  @Unstable
-  public SystemMetricsPublisher getSystemMetricsPublisher() {
-    return systemMetricsPublisher;
-  }
-
-  @Private
-  @Unstable
-  public void setRMApplicationHistoryWriter(
-      RMApplicationHistoryWriter rmApplicationHistoryWriter) {
-    this.rmApplicationHistoryWriter = rmApplicationHistoryWriter;
-  }
-
-  @Private
-  @Unstable
   public long getEpoch() {
     return this.epoch;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.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/RMContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
index 3b8a5b4..2ba445c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContext.java
@@ -113,10 +113,10 @@ public interface RMContext {
   void setSystemMetricsPublisher(SystemMetricsPublisher systemMetricsPublisher);
 
   SystemMetricsPublisher getSystemMetricsPublisher();
-  
+
   void setRMTimelineCollectorManager(
       RMTimelineCollectorManager timelineCollectorManager);
-  
+
   RMTimelineCollectorManager getRMTimelineCollectorManager();
 
   ConfigurationProvider getConfigurationProvider();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index 12a5577..e1e3298 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -380,7 +380,7 @@ public class RMContextImpl implements RMContext {
   public RMTimelineCollectorManager getRMTimelineCollectorManager() {
     return activeServiceContext.getRMTimelineCollectorManager();
   }
-  
+
   @Override
   public void setSystemMetricsPublisher(
       SystemMetricsPublisher metricsPublisher) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 9fa10d2..0c1df33 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
@@ -22,11 +22,11 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
+import java.nio.charset.Charset;
 import java.security.PrivilegedExceptionAction;
+import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -123,15 +123,6 @@ import org.apache.hadoop.yarn.webapp.WebApps.Builder;
 import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.PrintStream;
-import java.net.InetSocketAddress;
-import java.nio.charset.Charset;
-import java.security.PrivilegedExceptionAction;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.List;
 import com.google.common.annotations.VisibleForTesting;
 
 /**
@@ -312,6 +303,15 @@ public class ResourceManager extends CompositeService implements Recoverable {
     addService(rmApplicationHistoryWriter);
     rmContext.setRMApplicationHistoryWriter(rmApplicationHistoryWriter);
 
+    // initialize the RM timeline collector first so that the system metrics
+    // publisher can bind to it
+    if (YarnConfiguration.timelineServiceV2Enabled(this.conf)) {
+      RMTimelineCollectorManager timelineCollectorManager =
+          createRMTimelineCollectorManager();
+      addService(timelineCollectorManager);
+      rmContext.setRMTimelineCollectorManager(timelineCollectorManager);
+    }
+
     SystemMetricsPublisher systemMetricsPublisher =
         createSystemMetricsPublisher();
     addIfService(systemMetricsPublisher);
@@ -602,18 +602,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
         rmContext.setDelegationTokenRenewer(delegationTokenRenewer);
       }
 
-      RMApplicationHistoryWriter rmApplicationHistoryWriter =
-          createRMApplicationHistoryWriter();
-      addService(rmApplicationHistoryWriter);
-      rmContext.setRMApplicationHistoryWriter(rmApplicationHistoryWriter);
-
-      if (YarnConfiguration.timelineServiceV2Enabled(configuration)) {
-        RMTimelineCollectorManager timelineCollectorManager =
-            createRMTimelineCollectorManager();
-        addService(timelineCollectorManager);
-        rmContext.setRMTimelineCollectorManager(timelineCollectorManager);
-      }
-
       // Register event handler for NodesListManager
       nodesListManager = new NodesListManager(rmContext);
       rmDispatcher.register(NodesListManagerEventType.class, nodesListManager);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.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/RMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
index 8a795cd..98cbd92 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
@@ -177,19 +177,25 @@ public interface RMApp extends EventHandler<RMAppEvent> {
   String getTrackingUrl();
 
   /**
-   * The collector address for the application.
-   * @return the address for the application's collector.
+   * The collector address for the application. It should be used only if the
+   * timeline service v.2 is enabled.
+   *
+   * @return the address for the application's collector, or null if the
+   * timeline service v.2 is not enabled.
    */
   String getCollectorAddr();
 
   /**
-   * Set collector address for the application
+   * Set collector address for the application. It should be used only if the
+   * timeline service v.2 is enabled.
+   *
    * @param collectorAddr the address of collector
    */
   void setCollectorAddr(String collectorAddr);
 
   /**
-   * Remove collector address when application is finished or killed.
+   * Remove collector address when application is finished or killed. It should
+   * be used only if the timeline service v.2 is enabled.
    */
   void removeCollectorAddr();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a8779bec/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 6e448f7..45ff79c 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
@@ -526,6 +526,10 @@ public class RMAppImpl implements RMApp, Recoverable {
     }
   }
 
+  /**
+   * Starts the application level timeline collector for this app. This should
+   * be used only if the timeline service v.2 is enabled.
+   */
   public void startTimelineCollector() {
     AppLevelTimelineCollector collector =
         new AppLevelTimelineCollector(applicationId);
@@ -533,6 +537,10 @@ public class RMAppImpl implements RMApp, Recoverable {
         applicationId, collector);
   }
 
+  /**
+   * Stops the application level timeline collector for this app. This should be
+   * used only if the timeline service v.2 is enabled.
+   */
   public void stopTimelineCollector() {
     rmContext.getRMTimelineCollectorManager().remove(applicationId);
   }


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


[18/50] [abbrv] hadoop git commit: YARN-5045. hbase unit tests fail due to dependency issues. (Sangjin Lee via varunsaxena)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/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
deleted file mode 100644
index 8ab54bc..0000000
--- 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
+++ /dev/null
@@ -1,3461 +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.Arrays;
-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.TimelineMetricOperation;
-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.reader.TimelineDataToRetrieve;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineEntityFilters;
-import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderContext;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
-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.reader.filter.TimelineKeyValuesFilter;
-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.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 org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-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;
-  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 = 1425016502000L;
-    entity.setCreatedTime(cTime);
-    // add the info map in Timeline Entity
-    Map<String, Object> infoMap = new HashMap<String, Object>();
-    infoMap.put("infoMapKey1", "infoMapValue2");
-    infoMap.put("infoMapKey2", 20);
-    infoMap.put("infoMapKey3", 85.85);
-    entity.addInfo(infoMap);
-    // add the isRelatedToEntity info
-    Set<String> isRelatedToSet = new HashSet<String>();
-    isRelatedToSet.add("relatedto1");
-    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
-    isRelatedTo.put("task", isRelatedToSet);
-    entity.setIsRelatedToEntities(isRelatedTo);
-    // add the relatesTo info
-    Set<String> relatesToSet = new HashSet<String>();
-    relatesToSet.add("relatesto1");
-    relatesToSet.add("relatesto3");
-    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
-    relatesTo.put("container", relatesToSet);
-    Set<String> relatesToSet11 = new HashSet<String>();
-    relatesToSet11.add("relatesto4");
-    relatesTo.put("container1", relatesToSet11);
-    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("start_event");
-    event.setTimestamp(ts);
-    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 + 20L);
-
-    // 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
-    Set<String> isRelatedToSet1 = new HashSet<String>();
-    isRelatedToSet1.add("relatedto3");
-    isRelatedToSet1.add("relatedto5");
-    Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
-    isRelatedTo1.put("task1", isRelatedToSet1);
-    Set<String> isRelatedToSet11 = new HashSet<String>();
-    isRelatedToSet11.add("relatedto4");
-    isRelatedTo1.put("task2", isRelatedToSet11);
-    entity1.setIsRelatedToEntities(isRelatedTo1);
-
-    // add the relatesTo info
-    Set<String> relatesToSet1 = new HashSet<String>();
-    relatesToSet1.add("relatesto1");
-    relatesToSet1.add("relatesto2");
-    Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
-    relatesTo1.put("container", 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);
-    TimelineEvent event11 = new TimelineEvent();
-    event11.setId("end_event");
-    event11.setTimestamp(ts);
-    entity1.addEvent(event11);
-    TimelineEvent event12 = new TimelineEvent();
-    event12.setId("update_event");
-    event12.setTimestamp(ts - 10);
-    entity1.addEvent(event12);
-    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 + 40L);
-    TimelineEvent event21 = new TimelineEvent();
-    event21.setId("update_event");
-    event21.setTimestamp(ts - 20);
-    entity2.addEvent(event21);
-    Set<String> isRelatedToSet2 = new HashSet<String>();
-    isRelatedToSet2.add("relatedto3");
-    Map<String, Set<String>> isRelatedTo2 = new HashMap<String, Set<String>>();
-    isRelatedTo2.put("task1", isRelatedToSet2);
-    entity2.setIsRelatedToEntities(isRelatedTo2);
-    Map<String, Set<String>> relatesTo3 = new HashMap<String, Set<String>>();
-    Set<String> relatesToSet14 = new HashSet<String>();
-    relatesToSet14.add("relatesto7");
-    relatesTo3.put("container2", relatesToSet14);
-    entity2.setRelatesToEntities(relatesTo3);
-
-    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 = 1425016502000L;
-    entity.setCreatedTime(cTime);
-    // add the info map in Timeline Entity
-    Map<String, Object> infoMap = new HashMap<String, Object>();
-    infoMap.put("infoMapKey1", "infoMapValue2");
-    infoMap.put("infoMapKey2", 20);
-    infoMap.put("infoMapKey3", 71.4);
-    entity.addInfo(infoMap);
-    // add the isRelatedToEntity info
-    Set<String> isRelatedToSet = new HashSet<String>();
-    isRelatedToSet.add("relatedto1");
-    Map<String, Set<String>> isRelatedTo = new HashMap<String, Set<String>>();
-    isRelatedTo.put("task", isRelatedToSet);
-    entity.setIsRelatedToEntities(isRelatedTo);
-
-    // add the relatesTo info
-    Set<String> relatesToSet = new HashSet<String>();
-    relatesToSet.add("relatesto1");
-    relatesToSet.add("relatesto3");
-    Map<String, Set<String>> relatesTo = new HashMap<String, Set<String>>();
-    relatesTo.put("container", relatesToSet);
-    Set<String> relatesToSet11 = new HashSet<String>();
-    relatesToSet11.add("relatesto4");
-    relatesTo.put("container1", relatesToSet11);
-    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, 70000000000L);
-    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("start_event");
-    event.setTimestamp(ts);
-    entity.addEvent(event);
-    te.addEntity(entity);
-
-    TimelineEntity entity1 = new TimelineEntity();
-    String id1 = "hello1";
-    entity1.setId(id1);
-    entity1.setType(type);
-    entity1.setCreatedTime(cTime + 20L);
-
-    // 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 event.
-    TimelineEvent event11 = new TimelineEvent();
-    event11.setId("end_event");
-    event11.setTimestamp(ts);
-    entity1.addEvent(event11);
-    TimelineEvent event12 = new TimelineEvent();
-    event12.setId("update_event");
-    event12.setTimestamp(ts - 10);
-    entity1.addEvent(event12);
-
-
-    // add the isRelatedToEntity info
-    Set<String> isRelatedToSet1 = new HashSet<String>();
-    isRelatedToSet1.add("relatedto3");
-    isRelatedToSet1.add("relatedto5");
-    Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
-    isRelatedTo1.put("task1", isRelatedToSet1);
-    Set<String> isRelatedToSet11 = new HashSet<String>();
-    isRelatedToSet11.add("relatedto4");
-    isRelatedTo1.put("task2", isRelatedToSet11);
-    entity1.setIsRelatedToEntities(isRelatedTo1);
-
-    // add the relatesTo info
-    Set<String> relatesToSet1 = new HashSet<String>();
-    relatesToSet1.add("relatesto1");
-    relatesToSet1.add("relatesto2");
-    Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
-    relatesTo1.put("container", 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 + 40L);
-    TimelineEvent event21 = new TimelineEvent();
-    event21.setId("update_event");
-    event21.setTimestamp(ts - 20);
-    entity2.addEvent(event21);
-    Set<String> isRelatedToSet2 = new HashSet<String>();
-    isRelatedToSet2.add("relatedto3");
-    Map<String, Set<String>> isRelatedTo2 = new HashMap<String, Set<String>>();
-    isRelatedTo2.put("task1", isRelatedToSet2);
-    entity2.setIsRelatedToEntities(isRelatedTo2);
-    Map<String, Set<String>> relatesTo3 = new HashMap<String, Set<String>>();
-    Set<String> relatesToSet14 = new HashSet<String>();
-    relatesToSet14.add("relatesto7");
-    relatesTo3.put("container2", relatesToSet14);
-    entity2.setRelatesToEntities(relatesTo3);
-    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()) {
-      Number val = m1.get(entry.getKey());
-      assertNotNull(val);
-      assertEquals(val.longValue(), entry.getValue().longValue());
-    }
-  }
-
-  @Test
-  public void testWriteApplicationToHBase() throws Exception {
-    TimelineEntities te = new TimelineEntities();
-    ApplicationEntity entity = new ApplicationEntity();
-    String appId = "application_1000178881110_2002";
-    entity.setId(appId);
-    long cTime = 1425016501000L;
-    entity.setCreatedTime(cTime);
-
-    // 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);
-
-    // add aggregated metrics
-    TimelineEntity aggEntity = new TimelineEntity();
-    String type = TimelineEntityType.YARN_APPLICATION.toString();
-    aggEntity.setId(appId);
-    aggEntity.setType(type);
-    long cTime2 = 1425016502000L;
-    long mTime2 = 1425026902000L;
-    aggEntity.setCreatedTime(cTime2);
-
-    TimelineMetric aggMetric = new TimelineMetric();
-    aggMetric.setId("MEM_USAGE");
-    Map<Long, Number> aggMetricValues = new HashMap<Long, Number>();
-    ts = System.currentTimeMillis();
-    aggMetricValues.put(ts - 120000, 102400000);
-    aggMetric.setType(Type.SINGLE_VALUE);
-    aggMetric.setRealtimeAggregationOp(TimelineMetricOperation.SUM);
-    aggMetric.setValues(aggMetricValues);
-    Set<TimelineMetric> aggMetrics = new HashSet<>();
-    aggMetrics.add(aggMetric);
-    entity.addMetrics(aggMetrics);
-    te.addEntity(entity);
-
-    HBaseTimelineWriterImpl hbi = null;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.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, appId, te);
-      hbi.stop();
-
-      // retrieve the row
-      byte[] rowKey =
-          ApplicationRowKey.getRowKey(cluster, user, flow, runid, appId);
-      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,
-          appId));
-
-      // check info column family
-      String id1 = ApplicationColumn.ID.readResult(result).toString();
-      assertEquals(appId, id1);
-
-      Number val =
-          (Number) ApplicationColumn.CREATED_TIME.readResult(result);
-      long cTime1 = val.longValue();
-      assertEquals(cTime1, cTime);
-
-      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());
-      matchMetrics(metricValues, metricMap);
-
-      // read the timeline entity using the reader this time
-      TimelineEntity e1 = reader.getEntity(
-          new TimelineReaderContext(cluster, user, flow, runid, appId,
-          entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(
-          null, null, EnumSet.of(TimelineReader.Field.ALL)));
-      assertNotNull(e1);
-
-      // verify attributes
-      assertEquals(appId, e1.getId());
-      assertEquals(TimelineEntityType.YARN_APPLICATION.toString(),
-          e1.getType());
-      assertEquals(cTime, e1.getCreatedTime());
-      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(2, metrics2.size());
-      for (TimelineMetric metric2 : metrics2) {
-        Map<Long, Number> metricValues2 = metric2.getValues();
-        assertTrue(metric2.getId().equals("MAP_SLOT_MILLIS") ||
-            metric2.getId().equals("MEM_USAGE"));
-        if (metric2.getId().equals("MAP_SLOT_MILLIS")) {
-          matchMetrics(metricValues, metricValues2);
-        }
-        if (metric2.getId().equals("MEM_USAGE")) {
-          matchMetrics(aggMetricValues, metricValues2);
-        }
-      }
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.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;
-    entity.setCreatedTime(cTime);
-
-    // 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;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      String cluster = "cluster_test_write_entity";
-      String user = "user1";
-      String flow = "some_flow_name";
-      String flowVersion = "AB7822C10F1111";
-      long runid = 1002345678919L;
-      String appName =
-          ApplicationId.newInstance(System.currentTimeMillis(), 1).toString();
-      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);
-
-          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());
-          matchMetrics(metricValues, metricMap);
-        }
-      }
-      assertEquals(1, rowCount);
-      assertEquals(16, colCount);
-
-      // read the timeline entity using the reader this time
-      TimelineEntity e1 = reader.getEntity(
-          new TimelineReaderContext(cluster, user, flow, runid, appName,
-          entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-      Set<TimelineEntity> es1 = reader.getEntities(
-          new TimelineReaderContext(cluster, user, flow, runid, appName,
-          entity.getType(), null),
-          new TimelineEntityFilters(),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-      assertNotNull(e1);
-      assertEquals(1, es1.size());
-
-      // verify attributes
-      assertEquals(id, e1.getId());
-      assertEquals(type, e1.getType());
-      assertEquals(cTime, e1.getCreatedTime());
-      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();
-        matchMetrics(metricValues, metricValues2);
-      }
-    } finally {
-      if (hbi != null) {
-        hbi.stop();
-        hbi.close();
-      }
-    }
-  }
-
-  private boolean isRowKeyCorrect(byte[] rowKey, String cluster, String user,
-      String flow, long runid, String appName, TimelineEntity te) {
-
-    EntityRowKey key = EntityRowKey.parseRowKey(rowKey);
-
-    assertEquals(user, key.getUserId());
-    assertEquals(cluster, key.getClusterId());
-    assertEquals(flow, key.getFlowName());
-    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) {
-
-    ApplicationRowKey key = ApplicationRowKey.parseRowKey(rowKey);
-
-    assertEquals(cluster, key.getClusterId());
-    assertEquals(user, key.getUserId());
-    assertEquals(flow, key.getFlowName());
-    assertEquals(runid, key.getFlowRunId());
-    assertEquals(appName, key.getAppId());
-    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;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      String cluster = "cluster_test_events";
-      String user = "user2";
-      String flow = "other_flow_name";
-      String flowVersion = "1111F01C2287BA";
-      long runid = 1009876543218L;
-      String appName = "application_123465899910_1001";
-      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(
-            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
-        assertEquals(expVal, value.toString());
-      }
-
-      // read the timeline entity using the reader this time
-      TimelineEntity e1 = reader.getEntity(
-          new TimelineReaderContext(cluster, user, flow, runid, appName,
-          entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-      TimelineEntity e2 = reader.getEntity(
-          new TimelineReaderContext(cluster, user, null, null, appName,
-          entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-      assertNotNull(e1);
-      assertNotNull(e2);
-      assertEquals(e1, e2);
-
-      // 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();
-      }
-    }
-  }
-
-  @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;
-    try {
-      Configuration c1 = util.getConfiguration();
-      hbi = new HBaseTimelineWriterImpl(c1);
-      hbi.init(c1);
-      hbi.start();
-      String cluster = "cluster_test_empty_eventkey";
-      String user = "user_emptyeventkey";
-      String flow = "other_flow_name";
-      String flowVersion = "1111F01C2287BA";
-      long runid = 1009876543218L;
-      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);
-      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(TimelineStorageUtils.invertLong(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 = reader.getEntity(
-          new TimelineReaderContext(cluster, user, flow, runid, appName,
-          entity.getType(), entity.getId()),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-      Set<TimelineEntity> es1 = reader.getEntities(
-          new TimelineReaderContext(cluster, user, flow, runid, appName,
-          entity.getType(), null),
-          new TimelineEntityFilters(),
-          new TimelineDataToRetrieve(null, null, EnumSet.of(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();
-    }
-  }
-
-  @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);
-    // 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);
-    // 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();
-      }
-    }
-  }
-
-  @Test
-  public void testReadEntities() throws Exception {
-    TimelineEntity entity = reader.getEntity(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", "hello"),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-    assertNotNull(entity);
-    assertEquals(3, entity.getConfigs().size());
-    assertEquals(1, entity.getIsRelatedToEntities().size());
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world",
-        null), new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-    assertEquals(3, entities.size());
-    int cfgCnt = 0;
-    int metricCnt = 0;
-    int infoCnt = 0;
-    int eventCnt = 0;
-    int relatesToCnt = 0;
-    int isRelatedToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      cfgCnt += (timelineEntity.getConfigs() == null) ? 0 :
-          timelineEntity.getConfigs().size();
-      metricCnt += (timelineEntity.getMetrics() == null) ? 0 :
-          timelineEntity.getMetrics().size();
-      infoCnt += (timelineEntity.getInfo() == null) ? 0 :
-          timelineEntity.getInfo().size();
-      eventCnt += (timelineEntity.getEvents() == null) ? 0 :
-          timelineEntity.getEvents().size();
-      relatesToCnt += (timelineEntity.getRelatesToEntities() == null) ? 0 :
-          timelineEntity.getRelatesToEntities().size();
-      isRelatedToCnt += (timelineEntity.getIsRelatedToEntities() == null) ? 0 :
-          timelineEntity.getIsRelatedToEntities().size();
-    }
-    assertEquals(5, cfgCnt);
-    assertEquals(3, metricCnt);
-    assertEquals(5, infoCnt);
-    assertEquals(4, eventCnt);
-    assertEquals(4, relatesToCnt);
-    assertEquals(4, isRelatedToCnt);
-  }
-
-  @Test
-  public void testFilterEntitiesByCreatedTime() throws Exception {
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, 1425016502000L, 1425016502040L, null,
-        null, null, null, null, null), new TimelineDataToRetrieve());
-    assertEquals(3, entities.size());
-    for (TimelineEntity entity : entities) {
-      if (!entity.getId().equals("hello") && !entity.getId().equals("hello1") &&
-          !entity.getId().equals("hello2")) {
-        Assert.fail("Entities with ids' hello, hello1 and hello2 should be" +
-           " present");
-      }
-    }
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, 1425016502015L, null, null, null, null,
-        null, null, null), new TimelineDataToRetrieve());
-    assertEquals(2, entities.size());
-    for (TimelineEntity entity : entities) {
-      if (!entity.getId().equals("hello1") &&
-          !entity.getId().equals("hello2")) {
-        Assert.fail("Entities with ids' hello1 and hello2 should be present");
-      }
-    }
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, 1425016502015L, null, null, null,
-        null, null, null), new TimelineDataToRetrieve());
-     assertEquals(1, entities.size());
-     for (TimelineEntity entity : entities) {
-       if (!entity.getId().equals("hello")) {
-         Assert.fail("Entity with id hello should be present");
-       }
-     }
-  }
-
-  @Test
-  public void testReadEntitiesRelationsAndEventFiltersDefaultView()
-      throws Exception {
-    TimelineFilterList eventFilter = new TimelineFilterList();
-    eventFilter.addFilter(new TimelineExistsFilter(TimelineCompareOp.NOT_EQUAL,
-        "end_event"));
-    TimelineFilterList relatesTo = new TimelineFilterList(Operator.OR);
-    relatesTo.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container2",
-        new HashSet<Object>(Arrays.asList("relatesto7"))));
-    relatesTo.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container1",
-        new HashSet<Object>(Arrays.asList("relatesto4"))));
-    TimelineFilterList isRelatedTo = new TimelineFilterList();
-    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto3"))));
-    isRelatedTo.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.NOT_EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto5"))));
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, relatesTo, isRelatedTo,
-        null, null, null, eventFilter), new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    int eventCnt = 0;
-    int isRelatedToCnt = 0;
-    int relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      eventCnt += timelineEntity.getEvents().size();
-      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity id should have been hello2");
-      }
-    }
-    assertEquals(0, eventCnt);
-    assertEquals(0, isRelatedToCnt);
-    assertEquals(0, relatesToCnt);
-  }
-
-  @Test
-  public void testReadEntitiesEventFilters() throws Exception {
-    TimelineFilterList ef = new TimelineFilterList();
-    ef.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "update_event"));
-    ef.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.NOT_EQUAL, "end_event"));
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-    assertEquals(1, entities.size());
-    int eventCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      eventCnt += timelineEntity.getEvents().size();
-      if (!timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity id should have been hello2");
-      }
-    }
-    assertEquals(1, eventCnt);
-
-    TimelineFilterList ef1 = new TimelineFilterList();
-    ef1.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "update_event"));
-    ef1.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.NOT_EQUAL, "end_event"));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef1),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    eventCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      eventCnt += timelineEntity.getEvents().size();
-      if (!timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity id should have been hello2");
-      }
-    }
-    assertEquals(0, eventCnt);
-
-    TimelineFilterList ef2 = new TimelineFilterList();
-    ef2.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.NOT_EQUAL, "end_event"));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef2),
-        new TimelineDataToRetrieve());
-    assertEquals(2, entities.size());
-    eventCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      eventCnt += timelineEntity.getEvents().size();
-      if (!timelineEntity.getId().equals("hello") &&
-          !timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity ids' should have been hello and hello2");
-      }
-    }
-    assertEquals(0, eventCnt);
-
-    TimelineFilterList ef3 = new TimelineFilterList();
-    ef3.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "update_event"));
-    ef3.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "dummy_event"));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef3),
-        new TimelineDataToRetrieve());
-    assertEquals(0, entities.size());
-
-    TimelineFilterList list1 = new TimelineFilterList();
-    list1.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "update_event"));
-    list1.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "dummy_event"));
-    TimelineFilterList list2 = new TimelineFilterList();
-    list2.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.EQUAL, "start_event"));
-    TimelineFilterList ef4 = new TimelineFilterList(Operator.OR, list1, list2);
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef4),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    eventCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      eventCnt += timelineEntity.getEvents().size();
-      if (!timelineEntity.getId().equals("hello")) {
-        Assert.fail("Entity id should have been hello");
-      }
-    }
-    assertEquals(0, eventCnt);
-
-    TimelineFilterList ef5 = new TimelineFilterList();
-    ef5.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.NOT_EQUAL, "update_event"));
-    ef5.addFilter(new TimelineExistsFilter(
-        TimelineCompareOp.NOT_EQUAL, "end_event"));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        null, ef5),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    eventCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      eventCnt += timelineEntity.getEvents().size();
-      if (!timelineEntity.getId().equals("hello")) {
-          Assert.fail("Entity id should have been hello");
-        }
-    }
-    assertEquals(0, eventCnt);
-  }
-
-  @Test
-  public void testReadEntitiesIsRelatedTo() throws Exception {
-    TimelineFilterList irt = new TimelineFilterList(Operator.OR);
-    irt.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task",
-        new HashSet<Object>(Arrays.asList("relatedto1"))));
-    irt.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task2",
-        new HashSet<Object>(Arrays.asList("relatedto4"))));
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, irt, null, null, null,
-        null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-    assertEquals(2, entities.size());
-    int isRelatedToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
-      if (!timelineEntity.getId().equals("hello") &&
-          !timelineEntity.getId().equals("hello1")) {
-        Assert.fail("Entity ids' should have been hello and hello1");
-      }
-    }
-    assertEquals(3, isRelatedToCnt);
-
-    TimelineFilterList irt1 = new TimelineFilterList();
-    irt1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto3"))));
-    irt1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.NOT_EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto5"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt1, null, null,
-        null, null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    isRelatedToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
-      if (!timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity id should have been hello2");
-      }
-    }
-    assertEquals(0, isRelatedToCnt);
-
-    TimelineFilterList irt2 = new TimelineFilterList(Operator.OR);
-    irt2.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task",
-        new HashSet<Object>(Arrays.asList("relatedto1"))));
-    irt2.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task2",
-        new HashSet<Object>(Arrays.asList("relatedto4"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt2, null, null,
-        null, null),
-        new TimelineDataToRetrieve());
-    assertEquals(2, entities.size());
-    isRelatedToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
-      if (!timelineEntity.getId().equals("hello") &&
-          !timelineEntity.getId().equals("hello1")) {
-        Assert.fail("Entity ids' should have been hello and hello1");
-      }
-    }
-    assertEquals(0, isRelatedToCnt);
-
-    TimelineFilterList irt3 = new TimelineFilterList();
-    irt3.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto5"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt3, null, null,
-        null, null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    isRelatedToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
-      if (!timelineEntity.getId().equals("hello1")) {
-        Assert.fail("Entity id should have been hello1");
-      }
-    }
-    assertEquals(0, isRelatedToCnt);
-
-    TimelineFilterList irt4 = new TimelineFilterList();
-    irt4.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto3"))));
-    irt4.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "dummy_task",
-        new HashSet<Object>(Arrays.asList("relatedto5"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt4, null, null,
-        null, null),
-        new TimelineDataToRetrieve());
-    assertEquals(0, entities.size());
-
-    TimelineFilterList irt5 = new TimelineFilterList();
-    irt5.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task1",
-        new HashSet<Object>(Arrays.asList("relatedto3", "relatedto7"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt5, null, null,
-        null, null),
-        new TimelineDataToRetrieve());
-    assertEquals(0, entities.size());
-
-    TimelineFilterList list1 = new TimelineFilterList();
-    list1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task",
-        new HashSet<Object>(Arrays.asList("relatedto1"))));
-    list1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "dummy_task",
-        new HashSet<Object>(Arrays.asList("relatedto4"))));
-    TimelineFilterList list2 = new TimelineFilterList();
-    list2.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "task2",
-        new HashSet<Object>(Arrays.asList("relatedto4"))));
-    TimelineFilterList irt6 = new TimelineFilterList(Operator.OR, list1, list2);
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, null, irt6, null, null,
-        null, null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    isRelatedToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      isRelatedToCnt += timelineEntity.getIsRelatedToEntities().size();
-      if (!timelineEntity.getId().equals("hello1")) {
-        Assert.fail("Entity id should have been hello1");
-      }
-    }
-    assertEquals(0, isRelatedToCnt);
-  }
-
-  @Test
-  public void testReadEntitiesRelatesTo() throws Exception {
-    TimelineFilterList rt = new TimelineFilterList(Operator.OR);
-    rt.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container2",
-        new HashSet<Object>(Arrays.asList("relatesto7"))));
-    rt.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container1",
-        new HashSet<Object>(Arrays.asList("relatesto4"))));
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, rt, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-    assertEquals(2, entities.size());
-    int relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello") &&
-          !timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity ids' should have been hello and hello2");
-      }
-    }
-    assertEquals(3, relatesToCnt);
-
-    TimelineFilterList rt1 = new TimelineFilterList();
-    rt1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto1"))));
-    rt1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.NOT_EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto3"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt1, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello1")) {
-        Assert.fail("Entity id should have been hello1");
-      }
-    }
-    assertEquals(0, relatesToCnt);
-
-    TimelineFilterList rt2 = new TimelineFilterList(Operator.OR);
-    rt2.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container2",
-        new HashSet<Object>(Arrays.asList("relatesto7"))));
-    rt2.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container1",
-        new HashSet<Object>(Arrays.asList("relatesto4"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt2, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(2, entities.size());
-    relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello") &&
-          !timelineEntity.getId().equals("hello2")) {
-        Assert.fail("Entity ids' should have been hello and hello2");
-      }
-    }
-    assertEquals(0, relatesToCnt);
-
-    TimelineFilterList rt3 = new TimelineFilterList();
-    rt3.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto1", "relatesto3"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt3, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello")) {
-        Assert.fail("Entity id should have been hello");
-      }
-    }
-    assertEquals(0, relatesToCnt);
-
-    TimelineFilterList rt4 = new TimelineFilterList();
-    rt4.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto1"))));
-    rt4.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "dummy_container",
-        new HashSet<Object>(Arrays.asList("relatesto5"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt4, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(0, entities.size());
-
-    TimelineFilterList rt5 = new TimelineFilterList();
-    rt5.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatedto1", "relatesto8"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt5, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(0, entities.size());
-
-    TimelineFilterList list1 = new TimelineFilterList();
-    list1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container2",
-        new HashSet<Object>(Arrays.asList("relatesto7"))));
-    list1.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "dummy_container",
-        new HashSet<Object>(Arrays.asList("relatesto4"))));
-    TimelineFilterList list2 = new TimelineFilterList();
-    list2.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container1",
-        new HashSet<Object>(Arrays.asList("relatesto4"))));
-    TimelineFilterList rt6 = new TimelineFilterList(Operator.OR, list1, list2);
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt6, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello")) {
-        Assert.fail("Entity id should have been hello");
-      }
-    }
-    assertEquals(0, relatesToCnt);
-
-    TimelineFilterList list3 = new TimelineFilterList();
-    list3.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto1"))));
-    list3.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container1",
-        new HashSet<Object>(Arrays.asList("relatesto4"))));
-    TimelineFilterList list4 = new TimelineFilterList();
-    list4.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto1"))));
-    list4.addFilter(new TimelineKeyValuesFilter(
-        TimelineCompareOp.EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto2"))));
-    TimelineFilterList combinedList =
-        new TimelineFilterList(Operator.OR, list3, list4);
-    TimelineFilterList rt7 = new TimelineFilterList(Operator.AND, combinedList,
-        new TimelineKeyValuesFilter(
-        TimelineCompareOp.NOT_EQUAL, "container",
-        new HashSet<Object>(Arrays.asList("relatesto3"))));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111", "world", null),
-        new TimelineEntityFilters(null, null, null, rt7, null, null, null, null,
-        null),
-        new TimelineDataToRetrieve());
-    assertEquals(1, entities.size());
-    relatesToCnt = 0;
-    for (TimelineEntity timelineEntity : entities) {
-      relatesToCnt += timelineEntity.getRelatesToEntities().size();
-      if (!timelineEntity.getId().equals("hello1")) {
-        Assert.fail("Entity id should have been hello1");
-      }
-    }
-    assertEquals(0, relatesToCnt);
-  }
-
-  @Test
-  public void testReadEntitiesDefaultView() throws Exception {
-    TimelineEntity e1 = reader.getEntity(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", "hello"),
-        new TimelineDataToRetrieve());
-    assertNotNull(e1);
-    assertTrue(e1.getInfo().isEmpty() && e1.getConfigs().isEmpty() &&
-        e1.getMetrics().isEmpty() && e1.getIsRelatedToEntities().isEmpty() &&
-        e1.getRelatesToEntities().isEmpty());
-    Set<TimelineEntity> es1 = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(),
-        new TimelineDataToRetrieve());
-    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(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", "hello"),
-        new TimelineDataToRetrieve(
-        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(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(
-        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(4, 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(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", "hello"),
-        new TimelineDataToRetrieve(list, null, null));
-    assertNotNull(e1);
-    assertEquals(1, e1.getConfigs().size());
-    Set<TimelineEntity> es1 = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(list, null, null));
-    int cfgCnt = 0;
-    for (TimelineEntity entity : es1) {
-      cfgCnt += entity.getConfigs().size();
-      for (String confKey : entity.getConfigs().keySet()) {
-        assertTrue("Config key returned should start with cfg_",
-            confKey.startsWith("cfg_"));
-      }
-    }
-    assertEquals(3, cfgCnt);
-  }
-
-  @Test
-  public void testReadEntitiesConfigFilters() throws Exception {
-    TimelineFilterList list1 = new TimelineFilterList();
-    list1.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
-    list1.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "cfg_param2", "value2"));
-    TimelineFilterList list2 = new TimelineFilterList();
-    list2.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "cfg_param1", "value3"));
-    list2.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "config_param2", "value2"));
-    TimelineFilterList confFilterList =
-        new TimelineFilterList(Operator.OR, list1, list2);
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
-    assertEquals(2, entities.size());
-    int cfgCnt = 0;
-    for (TimelineEntity entity : entities) {
-      cfgCnt += entity.getConfigs().size();
-    }
-    assertEquals(5, cfgCnt);
-
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-    assertEquals(2, entities.size());
-    cfgCnt = 0;
-    for (TimelineEntity entity : entities) {
-      cfgCnt += entity.getConfigs().size();
-    }
-    assertEquals(5, cfgCnt);
-
-    TimelineFilterList confFilterList1 = new TimelineFilterList(
-        new TimelineKeyValueFilter(
-        TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList1, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
-    assertEquals(1, entities.size());
-    cfgCnt = 0;
-    for (TimelineEntity entity : entities) {
-      cfgCnt += entity.getConfigs().size();
-    }
-    assertEquals(3, cfgCnt);
-
-    TimelineFilterList confFilterList2 = new TimelineFilterList(
-        new TimelineKeyValueFilter(
-        TimelineCompareOp.NOT_EQUAL, "cfg_param1", "value1"),
-        new TimelineKeyValueFilter(
-        TimelineCompareOp.NOT_EQUAL, "config_param2", "value2"));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList2, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
-    assertEquals(0, entities.size());
-
-    TimelineFilterList confFilterList3 = new TimelineFilterList(
-        new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "dummy_config", "value1"));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList3, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
-    assertEquals(0, entities.size());
-
-    TimelineFilterList confFilterList4 = new TimelineFilterList(
-        new TimelineKeyValueFilter(
-        TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1"));
-    entities = reader.getEntities(
-            new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-            1002345678919L, "application_1231111111_1111","world", null),
-            new TimelineEntityFilters(null, null, null, null, null, null,
-            confFilterList4, null, null),
-            new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
-    assertEquals(0, entities.size());
-
-    TimelineFilterList confFilterList5 = new TimelineFilterList(
-        new TimelineKeyValueFilter(
-        TimelineCompareOp.NOT_EQUAL, "dummy_config", "value1", false));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList5, null, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.CONFIGS)));
-    assertEquals(3, entities.size());
-  }
-
-  @Test
-  public void testReadEntitiesConfigFilterPrefix() throws Exception {
-    TimelineFilterList confFilterList = new TimelineFilterList();
-    confFilterList.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
-    TimelineFilterList list =
-        new TimelineFilterList(Operator.OR,
-            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "cfg_"));
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList, null, null),
-        new TimelineDataToRetrieve(list, null, null));
-    assertEquals(1, entities.size());
-    int cfgCnt = 0;
-    for (TimelineEntity entity : entities) {
-      cfgCnt += entity.getConfigs().size();
-      for (String confKey : entity.getConfigs().keySet()) {
-        assertTrue("Config key returned should start with cfg_",
-            confKey.startsWith("cfg_"));
-      }
-    }
-    assertEquals(2, cfgCnt);
-    TimelineFilterList list1 = new TimelineFilterList();
-    list1.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "cfg_param1", "value1"));
-    list1.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "cfg_param2", "value2"));
-    TimelineFilterList list2 = new TimelineFilterList();
-    list2.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "cfg_param1", "value3"));
-    list2.addFilter(new TimelineKeyValueFilter(
-        TimelineCompareOp.EQUAL, "config_param2", "value2"));
-    TimelineFilterList confFilterList1 =
-        new TimelineFilterList(Operator.OR, list1, list2);
-    TimelineFilterList confsToRetrieve =
-        new TimelineFilterList(Operator.OR,
-            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "config_"));
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null,
-        confFilterList1, null, null),
-        new TimelineDataToRetrieve(confsToRetrieve, null, null));
-    assertEquals(2, entities.size());
-    cfgCnt = 0;
-    for (TimelineEntity entity : entities) {
-      cfgCnt += entity.getConfigs().size();
-      for (String confKey : entity.getConfigs().keySet()) {
-        assertTrue("Config key returned should start with config_",
-            confKey.startsWith("config_"));
-       }
-    }
-    assertEquals(2, cfgCnt);
-  }
-
-  @Test
-  public void testReadEntitiesMetricPrefix() throws Exception {
-    TimelineFilterList list =
-        new TimelineFilterList(Operator.OR,
-            new TimelinePrefixFilter(TimelineCompareOp.EQUAL, "MAP1_"));
-    TimelineEntity e1 = reader.getEntity(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", "hello"),
-        new TimelineDataToRetrieve(null, list, null));
-    assertNotNull(e1);
-    assertEquals(1, e1.getMetrics().size());
-    Set<TimelineEntity> es1 = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(),
-        new TimelineDataToRetrieve(null, list, null));
-    int metricCnt = 0;
-    for (TimelineEntity entity : es1) {
-      metricCnt += entity.getMetrics().size();
-      for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue("Metric Id returned should start with MAP1_",
-            metric.getId().startsWith("MAP1_"));
-      }
-    }
-    assertEquals(2, metricCnt);
-  }
-
-  @Test
-  public void testReadEntitiesMetricFilters() throws Exception {
-    TimelineFilterList list1 = new TimelineFilterList();
-    list1.addFilter(new TimelineCompareFilter(
-        TimelineCompareOp.GREATER_OR_EQUAL, "MAP1_SLOT_MILLIS", 50000000900L));
-    TimelineFilterList list2 = new TimelineFilterList();
-    list2.addFilter(new TimelineCompareFilter(
-        TimelineCompareOp.LESS_THAN, "MAP_SLOT_MILLIS", 80000000000L));
-    list2.addFilter(new TimelineCompareFilter(
-        TimelineCompareOp.EQUAL, "MAP1_BYTES", 50));
-    TimelineFilterList metricFilterList =
-        new TimelineFilterList(Operator.OR, list1, list2);
-    Set<TimelineEntity> entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.METRICS)));
-    assertEquals(2, entities.size());
-    int metricCnt = 0;
-    for (TimelineEntity entity : entities) {
-      metricCnt += entity.getMetrics().size();
-    }
-    assertEquals(3, metricCnt);
-
-    entities = reader.getEntities(
-        new TimelineReaderContext("cluster1", "user1", "some_flow_name",
-        1002345678919L, "application_1231111111_1111","world", null),
-        new TimelineEntityFilters(null, null, null, null, null, null, null,
-        metricFilterList, null),
-        new TimelineDataToRetrieve(null, null, EnumSet.of(Field.ALL)));
-    assertEquals(2, entities.size());
-    metricCnt = 0;
-    for (TimelineEntity entity : entities) {
-      metricCnt += entity.getMetrics().size();
-    }
-    assertEquals(3, metricCnt);
-
-    TimelineFilterList metricFilterList1 = new TimelineFilterList(
-        new TimelineCompareFilter(
-        TimelineCompareOp.LESS_OR_EQUAL, "MAP_SLOT_MILLIS", 80000000000L),
-        new TimelineCompareFilter(
-        TimelineCompareOp.NOT_EQUAL, "MAP1_BYTES", 30));


<TRUNCATED>

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


[20/50] [abbrv] hadoop git commit: YARN-5045. hbase unit tests fail due to dependency issues. (Sangjin Lee via varunsaxena)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
new file mode 100644
index 0000000..e7e7ba4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRunCompaction.java
@@ -0,0 +1,671 @@
+/**
+ * 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 static org.junit.Assert.assertNotEquals;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.ArrayList;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.KeyValue;
+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.Put;
+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.regionserver.HRegion;
+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.server.timelineservice.storage.common.LongConverter;
+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.TimestampGenerator;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+
+/**
+ * Tests the FlowRun and FlowActivity Tables
+ */
+public class TestHBaseStorageFlowRunCompaction {
+
+  private static HBaseTestingUtility util;
+
+  private final String metric1 = "MAP_SLOT_MILLIS";
+  private final String metric2 = "HDFS_BYTES_READ";
+
+  private final byte[] aRowKey = Bytes.toBytes("a");
+  private final byte[] aFamily = Bytes.toBytes("family");
+  private final byte[] aQualifier = Bytes.toBytes("qualifier");
+
+  @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 non numeric data into flow run table
+   * reads it back
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testWriteNonNumericData() throws Exception {
+    String rowKey = "nonNumericRowKey";
+    String column = "nonNumericColumnName";
+    String value = "nonNumericValue";
+    byte[] rowKeyBytes = Bytes.toBytes(rowKey);
+    byte[] columnNameBytes = Bytes.toBytes(column);
+    byte[] valueBytes = Bytes.toBytes(value);
+    Put p = new Put(rowKeyBytes);
+    p.addColumn(FlowRunColumnFamily.INFO.getBytes(), columnNameBytes,
+        valueBytes);
+    Configuration hbaseConf = util.getConfiguration();
+    TableName table = TableName.valueOf(hbaseConf.get(
+        FlowRunTable.TABLE_NAME_CONF_NAME, FlowRunTable.DEFAULT_TABLE_NAME));
+    Connection conn = null;
+    conn = ConnectionFactory.createConnection(hbaseConf);
+    Table flowRunTable = conn.getTable(table);
+    flowRunTable.put(p);
+
+    Get g = new Get(rowKeyBytes);
+    Result r = flowRunTable.get(g);
+    assertNotNull(r);
+    assertTrue(r.size() >= 1);
+    Cell actualValue = r.getColumnLatestCell(
+        FlowRunColumnFamily.INFO.getBytes(), columnNameBytes);
+    assertNotNull(CellUtil.cloneValue(actualValue));
+    assertEquals(Bytes.toString(CellUtil.cloneValue(actualValue)), value);
+  }
+
+  @Test
+  public void testWriteFlowRunCompaction() throws Exception {
+    String cluster = "kompaction_cluster1";
+    String user = "kompaction_FlowRun__user1";
+    String flow = "kompaction_flowRun_flow_name";
+    String flowVersion = "AF1021C19F1351";
+    long runid = 1449526652000L;
+
+    int start = 10;
+    int count = 2000;
+    int appIdSuffix = 1;
+    HBaseTimelineWriterImpl hbi = null;
+    long insertTs = System.currentTimeMillis() - count;
+    Configuration c1 = util.getConfiguration();
+    TimelineEntities te1 = null;
+    TimelineEntity entityApp1 = null;
+    try {
+      hbi = new HBaseTimelineWriterImpl(c1);
+      hbi.init(c1);
+      // now insert count * ( 100 + 100) metrics
+      // each call to getEntityMetricsApp1 brings back 100 values
+      // of metric1 and 100 of metric2
+      for (int i = start; i < start + count; i++) {
+        String appName = "application_10240000000000_" + appIdSuffix;
+        insertTs++;
+        te1 = new TimelineEntities();
+        entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1(insertTs, c1);
+        te1.addEntity(entityApp1);
+        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+
+        appName = "application_2048000000000_7" + appIdSuffix;
+        insertTs++;
+        te1 = new TimelineEntities();
+        entityApp1 = TestFlowDataGenerator.getEntityMetricsApp2(insertTs);
+        te1.addEntity(entityApp1);
+        hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+      }
+    } finally {
+      String appName = "application_10240000000000_" + appIdSuffix;
+      te1 = new TimelineEntities();
+      entityApp1 = TestFlowDataGenerator.getEntityMetricsApp1Complete(
+          insertTs + 1, c1);
+      te1.addEntity(entityApp1);
+      hbi.write(cluster, user, flow, flowVersion, runid, appName, te1);
+      hbi.flush();
+      hbi.close();
+    }
+
+    // check in flow run table
+    HRegionServer server = util.getRSForFirstRegionInTable(TableName
+        .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
+    List<HRegion> regions = server.getOnlineRegions(TableName
+        .valueOf(FlowRunTable.DEFAULT_TABLE_NAME));
+    assertTrue("Didn't find any regions for primary table!", regions.size() > 0);
+    // flush and compact all the regions of the primary table
+    for (HRegion region : regions) {
+       region.flushcache();
+      region.compactStores(true);
+    }
+
+    // check flow run for one flow many apps
+    checkFlowRunTable(cluster, user, flow, runid, c1, 4);
+  }
+
+
+  private void checkFlowRunTable(String cluster, String user, String flow,
+      long runid, Configuration c1, int valueCount) 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());
+      assertEquals(valueCount, values.size());
+
+      rowCount++;
+      // check metric1
+      byte[] q = ColumnHelper.getColumnQualifier(
+          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric1);
+      assertTrue(values.containsKey(q));
+      assertEquals(141, Bytes.toLong(values.get(q)));
+
+      // check metric2
+      q = ColumnHelper.getColumnQualifier(
+          FlowRunColumnPrefix.METRIC.getColumnPrefixBytes(), metric2);
+      assertTrue(values.containsKey(q));
+      assertEquals(57, Bytes.toLong(values.get(q)));
+    }
+    assertEquals(1, rowCount);
+  }
+
+
+  private FlowScanner getFlowScannerForTestingCompaction() {
+    // create a FlowScanner object with the sole purpose of invoking a process
+    // summation;
+    CompactionRequest request = new CompactionRequest();
+    request.setIsMajor(true, true);
+    // okay to pass in nulls for the constructor arguments
+    // because all we want to do is invoke the process summation
+    FlowScanner fs = new FlowScanner(null, -1, null,
+        (request.isMajor() == true ? FlowScannerOperation.MAJOR_COMPACTION
+            : FlowScannerOperation.MINOR_COMPACTION));
+    assertNotNull(fs);
+    return fs;
+  }
+
+  @Test
+  public void checkProcessSummationMoreCellsSumFinal2()
+      throws IOException {
+    long cellValue1 = 1236L;
+    long cellValue2 = 28L;
+    long cellValue3 = 1236L;
+    long cellValue4 = 1236L;
+    FlowScanner fs = getFlowScannerForTestingCompaction();
+
+    // note down the current timestamp
+    long currentTimestamp = System.currentTimeMillis();
+    long cell1Ts = 1200120L;
+    long cell2Ts = TimestampGenerator.getSupplementedTimestamp(
+        System.currentTimeMillis(),"application_123746661110_11202");
+    long cell3Ts = 1277719L;
+    long cell4Ts = currentTimestamp - 10;
+
+    SortedSet<Cell> currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
+
+    List<Tag> tags = new ArrayList<>();
+    Tag t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
+        "application_1234588888_91188");
+    tags.add(t);
+    byte[] tagByteArray = Tag.fromList(tags);
+    // create a cell with a VERY old timestamp and attribute SUM_FINAL
+    Cell c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+        cell1Ts, Bytes.toBytes(cellValue1), tagByteArray);
+    currentColumnCells.add(c1);
+
+    tags = new ArrayList<>();
+    t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
+        "application_12700000001_29102");
+    tags.add(t);
+    tagByteArray = Tag.fromList(tags);
+    // create a cell with a recent timestamp and attribute SUM_FINAL
+    Cell c2 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+        cell2Ts, Bytes.toBytes(cellValue2), tagByteArray);
+    currentColumnCells.add(c2);
+
+    tags = new ArrayList<>();
+    t = new Tag(AggregationOperation.SUM.getTagType(),
+        "application_191780000000001_8195");
+    tags.add(t);
+    tagByteArray = Tag.fromList(tags);
+    // create a cell with a VERY old timestamp but has attribute SUM
+    Cell c3 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+        cell3Ts, Bytes.toBytes(cellValue3), tagByteArray);
+    currentColumnCells.add(c3);
+
+    tags = new ArrayList<>();
+    t = new Tag(AggregationOperation.SUM.getTagType(),
+        "application_191780000000001_98104");
+    tags.add(t);
+    tagByteArray = Tag.fromList(tags);
+    // create a cell with a VERY old timestamp but has attribute SUM
+    Cell c4 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+        cell4Ts, Bytes.toBytes(cellValue4), tagByteArray);
+    currentColumnCells.add(c4);
+
+    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
+        LongConverter.getInstance(), currentTimestamp);
+    assertNotNull(cells);
+
+    // we should be getting back 4 cells
+    // one is the flow sum cell
+    // two are the cells with SUM attribute
+    // one cell with SUM_FINAL
+    assertEquals(4, cells.size());
+
+    for (int i = 0; i < cells.size(); i++) {
+      Cell returnedCell = cells.get(0);
+      assertNotNull(returnedCell);
+
+      long returnTs = returnedCell.getTimestamp();
+      long returnValue = Bytes.toLong(CellUtil
+          .cloneValue(returnedCell));
+      if (returnValue == cellValue2) {
+        assertTrue(returnTs == cell2Ts);
+      } else if (returnValue == cellValue3) {
+        assertTrue(returnTs == cell3Ts);
+      } else if (returnValue == cellValue4) {
+        assertTrue(returnTs == cell4Ts);
+      } else if (returnValue == cellValue1) {
+        assertTrue(returnTs != cell1Ts);
+        assertTrue(returnTs > cell1Ts);
+        assertTrue(returnTs >= currentTimestamp);
+      } else {
+        // raise a failure since we expect only these two values back
+        Assert.fail();
+      }
+    }
+  }
+
+  // tests with many cells
+  // of type SUM and SUM_FINAL
+  // all cells of SUM_FINAL will expire
+  @Test
+  public void checkProcessSummationMoreCellsSumFinalMany() throws IOException {
+    FlowScanner fs = getFlowScannerForTestingCompaction();
+    int count = 200000;
+
+    long cellValueFinal = 1000L;
+    long cellValueNotFinal = 28L;
+
+    // note down the current timestamp
+    long currentTimestamp = System.currentTimeMillis();
+    long cellTsFinalStart = 10001120L;
+    long cellTsFinal = cellTsFinalStart;
+    long cellTsNotFinalStart = currentTimestamp - 5;
+    long cellTsNotFinal = cellTsNotFinalStart;
+
+    SortedSet<Cell> currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
+    List<Tag> tags = null;
+    Tag t = null;
+    Cell c1 = null;
+
+    // insert SUM_FINAL cells
+    for (int i = 0; i < count; i++) {
+      tags = new ArrayList<>();
+      t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
+          "application_123450000" + i + "01_19" + i);
+      tags.add(t);
+      byte[] tagByteArray = Tag.fromList(tags);
+      // create a cell with a VERY old timestamp and attribute SUM_FINAL
+      c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+          cellTsFinal, Bytes.toBytes(cellValueFinal), tagByteArray);
+      currentColumnCells.add(c1);
+      cellTsFinal++;
+    }
+
+    // add SUM cells
+    for (int i = 0; i < count; i++) {
+      tags = new ArrayList<>();
+      t = new Tag(AggregationOperation.SUM.getTagType(),
+          "application_1987650000" + i + "83_911" + i);
+      tags.add(t);
+      byte[] tagByteArray = Tag.fromList(tags);
+      // create a cell with attribute SUM
+      c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+          cellTsNotFinal, Bytes.toBytes(cellValueNotFinal), tagByteArray);
+      currentColumnCells.add(c1);
+      cellTsNotFinal++;
+    }
+
+    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
+        LongConverter.getInstance(), currentTimestamp);
+    assertNotNull(cells);
+
+    // we should be getting back count + 1 cells
+    // one is the flow sum cell
+    // others are the cells with SUM attribute
+    assertEquals(count + 1, cells.size());
+
+    for (int i = 0; i < cells.size(); i++) {
+      Cell returnedCell = cells.get(0);
+      assertNotNull(returnedCell);
+
+      long returnTs = returnedCell.getTimestamp();
+      long returnValue = Bytes.toLong(CellUtil
+          .cloneValue(returnedCell));
+      if (returnValue == (count * cellValueFinal)) {
+        assertTrue(returnTs > (cellTsFinalStart + count));
+        assertTrue(returnTs >= currentTimestamp);
+      } else if ((returnValue >= cellValueNotFinal)
+          && (returnValue <= cellValueNotFinal * count)) {
+        assertTrue(returnTs >= cellTsNotFinalStart);
+        assertTrue(returnTs <= cellTsNotFinalStart * count);
+      } else {
+        // raise a failure since we expect only these values back
+        Assert.fail();
+      }
+    }
+  }
+
+  // tests with many cells
+  // of type SUM and SUM_FINAL
+  // NOT cells of SUM_FINAL will expire
+  @Test
+  public void checkProcessSummationMoreCellsSumFinalVariedTags() throws IOException {
+    FlowScanner fs = getFlowScannerForTestingCompaction();
+    int countFinal = 20100;
+    int countNotFinal = 1000;
+    int countFinalNotExpire = 7009;
+
+    long cellValueFinal = 1000L;
+    long cellValueNotFinal = 28L;
+
+    // note down the current timestamp
+    long currentTimestamp = System.currentTimeMillis();
+    long cellTsFinalStart = 10001120L;
+    long cellTsFinal = cellTsFinalStart;
+
+    long cellTsFinalStartNotExpire = TimestampGenerator.getSupplementedTimestamp(
+        System.currentTimeMillis(), "application_10266666661166_118821");
+    long cellTsFinalNotExpire = cellTsFinalStartNotExpire;
+
+    long cellTsNotFinalStart = currentTimestamp - 5;
+    long cellTsNotFinal = cellTsNotFinalStart;
+
+    SortedSet<Cell> currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
+    List<Tag> tags = null;
+    Tag t = null;
+    Cell c1 = null;
+
+    // insert SUM_FINAL cells which will expire
+    for (int i = 0; i < countFinal; i++) {
+      tags = new ArrayList<>();
+      t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
+          "application_123450000" + i + "01_19" + i);
+      tags.add(t);
+      byte[] tagByteArray = Tag.fromList(tags);
+      // create a cell with a VERY old timestamp and attribute SUM_FINAL
+      c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+          cellTsFinal, Bytes.toBytes(cellValueFinal), tagByteArray);
+      currentColumnCells.add(c1);
+      cellTsFinal++;
+    }
+
+    // insert SUM_FINAL cells which will NOT expire
+    for (int i = 0; i < countFinalNotExpire; i++) {
+      tags = new ArrayList<>();
+      t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
+          "application_123450000" + i + "01_19" + i);
+      tags.add(t);
+      byte[] tagByteArray = Tag.fromList(tags);
+      // create a cell with a VERY old timestamp and attribute SUM_FINAL
+      c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+          cellTsFinalNotExpire, Bytes.toBytes(cellValueFinal), tagByteArray);
+      currentColumnCells.add(c1);
+      cellTsFinalNotExpire++;
+    }
+
+    // add SUM cells
+    for (int i = 0; i < countNotFinal; i++) {
+      tags = new ArrayList<>();
+      t = new Tag(AggregationOperation.SUM.getTagType(),
+          "application_1987650000" + i + "83_911" + i);
+      tags.add(t);
+      byte[] tagByteArray = Tag.fromList(tags);
+      // create a cell with attribute SUM
+      c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+          cellTsNotFinal, Bytes.toBytes(cellValueNotFinal), tagByteArray);
+      currentColumnCells.add(c1);
+      cellTsNotFinal++;
+    }
+
+    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
+        LongConverter.getInstance(), currentTimestamp);
+    assertNotNull(cells);
+
+    // we should be getting back
+    // countNotFinal + countFinalNotExpire + 1 cells
+    // one is the flow sum cell
+    // count = the cells with SUM attribute
+    // count = the cells with SUM_FINAL attribute but not expired
+    assertEquals(countFinalNotExpire + countNotFinal + 1, cells.size());
+
+    for (int i = 0; i < cells.size(); i++) {
+      Cell returnedCell = cells.get(0);
+      assertNotNull(returnedCell);
+
+      long returnTs = returnedCell.getTimestamp();
+      long returnValue = Bytes.toLong(CellUtil
+          .cloneValue(returnedCell));
+      if (returnValue == (countFinal * cellValueFinal)) {
+        assertTrue(returnTs > (cellTsFinalStart + countFinal));
+        assertTrue(returnTs >= currentTimestamp);
+      } else if (returnValue == cellValueNotFinal) {
+        assertTrue(returnTs >= cellTsNotFinalStart);
+        assertTrue(returnTs <= cellTsNotFinalStart + countNotFinal);
+      } else if (returnValue == cellValueFinal){
+        assertTrue(returnTs >= cellTsFinalStartNotExpire);
+        assertTrue(returnTs <= cellTsFinalStartNotExpire + countFinalNotExpire);
+      } else {
+        // raise a failure since we expect only these values back
+        Assert.fail();
+      }
+    }
+  }
+
+  @Test
+  public void testProcessSummationMoreCellsSumFinal() throws IOException {
+    FlowScanner fs = getFlowScannerForTestingCompaction();
+    // note down the current timestamp
+    long currentTimestamp = System.currentTimeMillis();
+    long cellValue1 = 1236L;
+    long cellValue2 = 28L;
+
+    List<Tag> tags = new ArrayList<>();
+    Tag t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
+        "application_1234588888_999888");
+    tags.add(t);
+    byte[] tagByteArray = Tag.fromList(tags);
+    SortedSet<Cell> currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
+
+    // create a cell with a VERY old timestamp and attribute SUM_FINAL
+    Cell c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+        120L, Bytes.toBytes(cellValue1), tagByteArray);
+    currentColumnCells.add(c1);
+
+    tags = new ArrayList<>();
+    t = new Tag(AggregationOperation.SUM.getTagType(),
+        "application_100000000001_119101");
+    tags.add(t);
+    tagByteArray = Tag.fromList(tags);
+
+    // create a cell with a VERY old timestamp but has attribute SUM
+    Cell c2 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+        130L, Bytes.toBytes(cellValue2), tagByteArray);
+    currentColumnCells.add(c2);
+    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
+        LongConverter.getInstance(), currentTimestamp);
+    assertNotNull(cells);
+
+    // we should be getting back two cells
+    // one is the flow sum cell
+    // another is the cell with SUM attribute
+    assertEquals(2, cells.size());
+
+    Cell returnedCell = cells.get(0);
+    assertNotNull(returnedCell);
+    long inputTs1 = c1.getTimestamp();
+    long inputTs2 = c2.getTimestamp();
+
+    long returnTs = returnedCell.getTimestamp();
+    long returnValue = Bytes.toLong(CellUtil
+        .cloneValue(returnedCell));
+    // the returned Ts will be far greater than input ts as well as the noted
+    // current timestamp
+    if (returnValue == cellValue2) {
+      assertTrue(returnTs == inputTs2);
+    } else if (returnValue == cellValue1) {
+      assertTrue(returnTs >= currentTimestamp);
+      assertTrue(returnTs != inputTs1);
+    } else {
+      // raise a failure since we expect only these two values back
+      Assert.fail();
+    }
+  }
+
+  @Test
+  public void testProcessSummationOneCellSumFinal() throws IOException {
+    FlowScanner fs = getFlowScannerForTestingCompaction();
+
+    // note down the current timestamp
+    long currentTimestamp = System.currentTimeMillis();
+    List<Tag> tags = new ArrayList<>();
+    Tag t = new Tag(AggregationOperation.SUM_FINAL.getTagType(),
+        "application_123458888888_999888");
+    tags.add(t);
+    byte[] tagByteArray = Tag.fromList(tags);
+    SortedSet<Cell> currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
+
+    // create a cell with a VERY old timestamp
+    Cell c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+        120L, Bytes.toBytes(1110L), tagByteArray);
+    currentColumnCells.add(c1);
+
+    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
+        LongConverter.getInstance(), currentTimestamp);
+    assertNotNull(cells);
+    // we should not get the same cell back
+    // but we get back the flow cell
+    assertEquals(1, cells.size());
+
+    Cell returnedCell = cells.get(0);
+    // it's NOT the same cell
+    assertNotEquals(c1, returnedCell);
+    long inputTs = c1.getTimestamp();
+    long returnTs = returnedCell.getTimestamp();
+    // the returned Ts will be far greater than input ts as well as the noted
+    // current timestamp
+    assertTrue(returnTs > inputTs);
+    assertTrue(returnTs >= currentTimestamp);
+  }
+
+  @Test
+  public void testProcessSummationOneCell() throws IOException {
+    FlowScanner fs = getFlowScannerForTestingCompaction();
+
+    // note down the current timestamp
+    long currentTimestamp = System.currentTimeMillis();
+
+    // try for 1 cell with tag SUM
+    List<Tag> tags = new ArrayList<>();
+    Tag t = new Tag(AggregationOperation.SUM.getTagType(),
+        "application_123458888888_999888");
+    tags.add(t);
+    byte[] tagByteArray = Tag.fromList(tags);
+
+    SortedSet<Cell> currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
+
+    Cell c1 = TimelineStorageUtils.createNewCell(aRowKey, aFamily, aQualifier,
+        currentTimestamp, Bytes.toBytes(1110L), tagByteArray);
+    currentColumnCells.add(c1);
+    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
+        LongConverter.getInstance(), currentTimestamp);
+    assertNotNull(cells);
+    // we expect the same cell back
+    assertEquals(1, cells.size());
+    Cell c2 = cells.get(0);
+    assertEquals(c1, c2);
+    assertEquals(currentTimestamp, c2.getTimestamp());
+  }
+
+  @Test
+  public void testProcessSummationEmpty() throws IOException {
+    FlowScanner fs = getFlowScannerForTestingCompaction();
+    long currentTimestamp = System.currentTimeMillis();
+
+    SortedSet<Cell> currentColumnCells = null;
+    List<Cell> cells = fs.processSummationMajorCompaction(currentColumnCells,
+        LongConverter.getInstance(), currentTimestamp);
+    assertNotNull(cells);
+    assertEquals(0, cells.size());
+
+    currentColumnCells = new TreeSet<Cell>(KeyValue.COMPARATOR);
+    cells = fs.processSummationMajorCompaction(currentColumnCells,
+        LongConverter.getInstance(), currentTimestamp);
+    assertNotNull(cells);
+    assertEquals(0, cells.size());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    util.shutdownMiniCluster();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/resources/log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/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-hbase-tests/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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ccdec4a1/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 3d17d98..dd15f44 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
@@ -97,67 +97,78 @@
     </dependency>
 
     <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-client</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>commons-cli</groupId>
+      <artifactId>commons-cli</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+    </dependency>
+
+    <dependency>
       <groupId>commons-logging</groupId>
       <artifactId>commons-logging</artifactId>
     </dependency>
 
-    <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-csv</artifactId>
     </dependency>
 
     <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
     </dependency>
 
     <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
     </dependency>
 
     <dependency>
-      <groupId>org.apache.phoenix</groupId>
-      <artifactId>phoenix-core</artifactId>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
     </dependency>
+
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-client</artifactId>
     </dependency>
-    <!-- for unit tests only -->
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <scope>test</scope>
-    </dependency>
+
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
     </dependency>
+
     <dependency>
       <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix-core</artifactId>
+    </dependency>
+
+    <!-- 'mvn dependency:analyze' fails to detect use of this dependency -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
+
     <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-it</artifactId>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
       <scope>test</scope>
-      <classifier>tests</classifier>
     </dependency>
+
     <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-testing-util</artifactId>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
       <scope>test</scope>
-      <optional>true</optional>
     </dependency>
   </dependencies>
 


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


[31/50] [abbrv] hadoop git commit: YARN-5095. flow activities and flow runs are populated with wrong timestamp when RM restarts w/ recovery enabled (Varun Saxena via sjlee)

Posted by vr...@apache.org.
YARN-5095. flow activities and flow runs are populated with wrong timestamp when RM restarts w/ recovery enabled (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/123ebb54
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/123ebb54
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/123ebb54

Branch: refs/heads/YARN-2928
Commit: 123ebb54c570c6e28794aeee8a8c965a7238867b
Parents: 9f6a75d
Author: Sangjin Lee <sj...@apache.org>
Authored: Wed May 25 16:56:49 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:11 2016 -0700

----------------------------------------------------------------------
 .../server/resourcemanager/RMAppManager.java    | 12 ++--
 .../server/resourcemanager/rmapp/RMAppImpl.java | 19 +++++-
 .../server/resourcemanager/TestRMRestart.java   | 63 ++++++++++++++++++++
 3 files changed, 87 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/123ebb54/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 30efc8e..2aa54fd 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
@@ -287,8 +287,10 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       String user) throws YarnException, AccessControlException {
     ApplicationId applicationId = submissionContext.getApplicationId();
 
-    RMAppImpl application =
-        createAndPopulateNewRMApp(submissionContext, submitTime, user, false);
+    // Passing start time as -1. It will be eventually set in RMAppImpl
+    // constructor.
+    RMAppImpl application = createAndPopulateNewRMApp(
+        submissionContext, submitTime, user, false, -1);
     Credentials credentials = null;
     try {
       credentials = parseCredentials(submissionContext);
@@ -326,14 +328,14 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
     // create and recover app.
     RMAppImpl application =
         createAndPopulateNewRMApp(appContext, appState.getSubmitTime(),
-            appState.getUser(), true);
+            appState.getUser(), true, appState.getStartTime());
 
     application.handle(new RMAppRecoverEvent(appId, rmState));
   }
 
   private RMAppImpl createAndPopulateNewRMApp(
       ApplicationSubmissionContext submissionContext, long submitTime,
-      String user, boolean isRecovery)
+      String user, boolean isRecovery, long startTime)
       throws YarnException, AccessControlException {
     // Do queue mapping
     if (!isRecovery) {
@@ -388,7 +390,7 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
             submissionContext.getQueue(),
             submissionContext, this.scheduler, this.masterService,
             submitTime, submissionContext.getApplicationType(),
-            submissionContext.getApplicationTags(), amReq);
+            submissionContext.getApplicationTags(), amReq, startTime);
     // Concurrent app submissions with same applicationId will fail here
     // Concurrent app submissions with different applicationIds will not
     // influence each other

http://git-wip-us.apache.org/repos/asf/hadoop/blob/123ebb54/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 8463f3a..6e448f7 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
@@ -416,8 +416,19 @@ public class RMAppImpl implements RMApp, Recoverable {
       Configuration config, String name, String user, String queue,
       ApplicationSubmissionContext submissionContext, YarnScheduler scheduler,
       ApplicationMasterService masterService, long submitTime,
-      String applicationType, Set<String> applicationTags, 
+      String applicationType, Set<String> applicationTags,
       ResourceRequest amReq) {
+    this(applicationId, rmContext, config, name, user, queue, submissionContext,
+      scheduler, masterService, submitTime, applicationType, applicationTags,
+      amReq, -1);
+  }
+
+  public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
+      Configuration config, String name, String user, String queue,
+      ApplicationSubmissionContext submissionContext, YarnScheduler scheduler,
+      ApplicationMasterService masterService, long submitTime,
+      String applicationType, Set<String> applicationTags,
+      ResourceRequest amReq, long startTime) {
 
     this.systemClock = SystemClock.getInstance();
 
@@ -433,7 +444,11 @@ public class RMAppImpl implements RMApp, Recoverable {
     this.scheduler = scheduler;
     this.masterService = masterService;
     this.submitTime = submitTime;
-    this.startTime = this.systemClock.getTime();
+    if (startTime <= 0) {
+      this.startTime = this.systemClock.getTime();
+    } else {
+      this.startTime = startTime;
+    }
     this.applicationType = applicationType;
     this.applicationTags = applicationTags;
     this.amReq = amReq;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/123ebb54/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.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/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index 364f9d1..bbb1d15 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -109,6 +109,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext;
 import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.log4j.Level;
@@ -1126,6 +1127,68 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
   }
 
   @Test (timeout = 60000)
+  public void testRMRestartTimelineCollectorContext() throws Exception {
+    conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+    conf.setFloat(YarnConfiguration.TIMELINE_SERVICE_VERSION, 2.0f);
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+    RMState rmState = memStore.getState();
+    Map<ApplicationId, ApplicationStateData> rmAppState =
+        rmState.getApplicationState();
+    MockRM rm1 = null;
+    MockRM rm2 = null;
+    try {
+      rm1 = createMockRM(conf, memStore);
+      rm1.start();
+      MockNM nm1 =
+          new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
+      nm1.registerNode();
+
+      // submit an app.
+      RMApp app = rm1.submitApp(200, "name", "user",
+          new HashMap<ApplicationAccessType, String>(), false, "default", -1,
+          null);
+      // Check if app info has been saved.
+      ApplicationStateData appState = rmAppState.get(app.getApplicationId());
+      Assert.assertNotNull(appState);
+      Assert.assertEquals(0, appState.getAttemptCount());
+      Assert.assertEquals(appState.getApplicationSubmissionContext()
+          .getApplicationId(), app.getApplicationSubmissionContext()
+          .getApplicationId());
+
+      // Allocate the AM
+      nm1.nodeHeartbeat(true);
+      RMAppAttempt attempt = app.getCurrentAppAttempt();
+      ApplicationAttemptId attemptId1 = attempt.getAppAttemptId();
+      rm1.waitForState(attemptId1, RMAppAttemptState.ALLOCATED);
+
+      ApplicationId appId = app.getApplicationId();
+      TimelineCollectorContext contextBeforeRestart =
+          rm1.getRMContext().getRMTimelineCollectorManager().get(appId).
+              getTimelineEntityContext();
+
+      // Restart RM.
+      rm2 = createMockRM(conf, memStore);
+      rm2.start();
+      Assert.assertEquals(1, rm2.getRMContext().getRMApps().size());
+      rm2.waitForState(app.getApplicationId(), RMAppState.ACCEPTED);
+      TimelineCollectorContext contextAfterRestart =
+          rm2.getRMContext().getRMTimelineCollectorManager().get(appId).
+              getTimelineEntityContext();
+      Assert.assertEquals("Collector contexts for an app should be same " +
+          "across restarts", contextBeforeRestart, contextAfterRestart);
+    } finally {
+      conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false);
+      if (rm1 != null) {
+        rm1.close();
+      }
+      if (rm2 != null) {
+        rm2.close();
+      }
+    }
+  }
+
+  @Test (timeout = 60000)
   public void testDelegationTokenRestoredInDelegationTokenRenewer()
       throws Exception {
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);


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


[10/50] [abbrv] hadoop git commit: YARN-4447. Provide a mechanism to represent complex filters and parse them at the REST layer (Varun Saxena via sjlee)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
index 57d75db..2e667d6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
@@ -18,29 +18,19 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.reader;
 
-import java.io.IOException;
 import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.Set;
 
 import javax.servlet.http.HttpServletRequest;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.server.timeline.GenericObjectMapper;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareFilter;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
-import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 
 /**
  * Set of utility methods to be used by timeline reader web services.
  */
 final class TimelineReaderWebServicesUtils {
-  private static final String COMMA_DELIMITER = ",";
-  private static final String COLON_DELIMITER = ":";
 
   private TimelineReaderWebServicesUtils() {
   }
@@ -56,11 +46,10 @@ final class TimelineReaderWebServicesUtils {
    * @param entityType Entity Type.
    * @param entityId Entity Id.
    * @return a {@link TimelineReaderContext} object.
-   * @throws Exception if any problem occurs during parsing.
    */
   static TimelineReaderContext createTimelineReaderContext(String clusterId,
       String userId, String flowName, String flowRunId, String appId,
-      String entityType, String entityId) throws Exception {
+      String entityType, String entityId) {
     return new TimelineReaderContext(parseStr(clusterId), parseStr(userId),
         parseStr(flowName), parseLongStr(flowRunId), parseStr(appId),
         parseStr(entityType), parseStr(entityId));
@@ -79,20 +68,17 @@ final class TimelineReaderWebServicesUtils {
    * @param metricfilters Entities to return must match these metric filters.
    * @param eventfilters Entities to return must match these event filters.
    * @return a {@link TimelineEntityFilters} object.
-   * @throws Exception if any problem occurs during parsing.
+   * @throws TimelineParseException if any problem occurs during parsing.
    */
   static TimelineEntityFilters createTimelineEntityFilters(String limit,
       String createdTimeStart, String createdTimeEnd, String relatesTo,
       String isRelatedTo, String infofilters, String conffilters,
-      String metricfilters, String eventfilters) throws Exception {
+      String metricfilters, String eventfilters) throws TimelineParseException {
     return new TimelineEntityFilters(parseLongStr(limit),
         parseLongStr(createdTimeStart), parseLongStr(createdTimeEnd),
-        parseKeyStrValuesStr(relatesTo, COMMA_DELIMITER, COLON_DELIMITER),
-        parseKeyStrValuesStr(isRelatedTo, COMMA_DELIMITER, COLON_DELIMITER),
-        parseKeyStrValueObj(infofilters, COMMA_DELIMITER, COLON_DELIMITER),
-        parseKeyStrValueStr(conffilters, COMMA_DELIMITER, COLON_DELIMITER),
-        parseMetricFilters(metricfilters, COMMA_DELIMITER),
-        parseValuesStr(eventfilters, COMMA_DELIMITER));
+        parseRelationFilters(relatesTo), parseRelationFilters(isRelatedTo),
+        parseKVFilters(infofilters, false), parseKVFilters(conffilters, true),
+        parseMetricFilters(metricfilters), parseEventFilters(eventfilters));
   }
 
   /**
@@ -102,12 +88,13 @@ final class TimelineReaderWebServicesUtils {
    * @param metrics metrics to retrieve.
    * @param fields fields to retrieve.
    * @return a {@link TimelineDataToRetrieve} object.
-   * @throws Exception if any problem occurs during parsing.
+   * @throws TimelineParseException if any problem occurs during parsing.
    */
   static TimelineDataToRetrieve createTimelineDataToRetrieve(String confs,
-      String metrics, String fields) throws Exception {
-    return new TimelineDataToRetrieve(
-        null, null, parseFieldsStr(fields, COMMA_DELIMITER));
+      String metrics, String fields) throws TimelineParseException {
+    return new TimelineDataToRetrieve(parseDataToRetrieve(confs),
+        parseDataToRetrieve(metrics), parseFieldsStr(
+            fields, TimelineParseConstants.COMMA_DELIMITER));
   }
 
   /**
@@ -118,110 +105,47 @@ final class TimelineReaderWebServicesUtils {
    * @param delimiter string is delimited by this delimiter.
    * @return set of strings.
    */
-  static TimelineFilterList parseValuesStr(String str, String delimiter) {
-    if (str == null || str.isEmpty()) {
-      return null;
-    }
-    TimelineFilterList filterList = new TimelineFilterList();
-    String[] strs = str.split(delimiter);
-    for (String aStr : strs) {
-      filterList.addFilter(new TimelineExistsFilter(TimelineCompareOp.EQUAL,
-          aStr.trim()));
-    }
-    return filterList;
-  }
-
-  private static TimelineFilterList parseKeyValues(String str,
-      String pairsDelim, String keyValuesDelim, boolean stringValue,
-      boolean multipleValues) {
-    if (str == null) {
-      return null;
-    }
-    TimelineFilterList list = new TimelineFilterList();
-    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());
-          list.addFilter(new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
-              pairStrs[0].trim(), value));
-        } catch (IOException e) {
-          list.addFilter(new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
-              pairStrs[0].trim(), pairStrs[1].trim()));
-        }
-      } else {
-        String key = pairStrs[0].trim();
-        if (multipleValues) {
-          Set<Object> values = new HashSet<Object>();
-          for (int i = 1; i < pairStrs.length; i++) {
-            values.add(pairStrs[i].trim());
-          }
-          list.addFilter(new TimelineKeyValuesFilter(
-              TimelineCompareOp.EQUAL, key, values));
-        } else {
-          list.addFilter(new TimelineKeyValueFilter(TimelineCompareOp.EQUAL,
-              key, pairStrs[1].trim()));
-        }
-      }
-    }
-    return list;
+  static TimelineFilterList parseEventFilters(String expr)
+      throws TimelineParseException {
+    return parseFilters(new TimelineParserForExistFilters(expr,
+        TimelineParseConstants.COMMA_CHAR));
   }
 
   /**
-   * Parse a delimited string and convert it into a map of key-values with each
-   * key having a set of values. Both the key and values are interpreted as
-   * strings.
-   * For instance, if pairsDelim is "," and keyValuesDelim is ":", then the
-   * string should be represented as
-   * "key1:value11:value12:value13,key2:value21,key3:value31:value32".
-   * @param str delimited string represented as multiple keys having multiple
-   *     values.
-   * @param pairsDelim key-values pairs are delimited by this delimiter.
-   * @param keyValuesDelim values for a key are delimited by this delimiter.
-   * @return a map of key-values with each key having a set of values.
+   * Parse relation filters.
+   * @param expr Relation filter expression
+   * @return a {@link TimelineFilterList} object.
+   *
+   * @throws Exception if any problem occurs.
    */
-  static TimelineFilterList parseKeyStrValuesStr(String str, String pairsDelim,
-      String keyValuesDelim) {
-    return parseKeyValues(str, pairsDelim, keyValuesDelim, true, true);
+  static TimelineFilterList parseRelationFilters(String expr)
+      throws TimelineParseException {
+    return parseFilters(new TimelineParserForRelationFilters(expr,
+        TimelineParseConstants.COMMA_CHAR,
+        TimelineParseConstants.COLON_DELIMITER));
   }
 
-  /**
-   * Parse a delimited string and convert it into a map of key-value pairs with
-   * both the key and value interpreted as strings.
-   * For instance, if pairsDelim is "," and keyValDelim is ":", then the string
-   * should be represented as "key1:value1,key2:value2,key3:value3".
-   * @param str delimited string represented as key-value pairs.
-   * @param pairsDelim key-value pairs are delimited by this delimiter.
-   * @param keyValDelim key and value are delimited by this delimiter.
-   * @return a map of key-value pairs with both key and value being strings.
-   */
-  static TimelineFilterList parseKeyStrValueStr(String str, String pairsDelim,
-      String keyValDelim) {
-    return parseKeyValues(str, pairsDelim, keyValDelim, true, false);
+  private static TimelineFilterList parseFilters(TimelineParser parser)
+      throws TimelineParseException {
+    try {
+      return parser.parse();
+    } finally {
+      IOUtils.closeQuietly(parser);
+    }
   }
 
   /**
-   * Parse a delimited string and convert it into a map of key-value pairs with
-   * key being a string and value interpreted as any object.
-   * For instance, if pairsDelim is "," and keyValDelim is ":", then the string
-   * should be represented as "key1:value1,key2:value2,key3:value3".
-   * @param str delimited string represented as key-value pairs.
-   * @param pairsDelim key-value pairs are delimited by this delimiter.
-   * @param keyValDelim key and value are delimited by this delimiter.
-   * @return a map of key-value pairs with key being a string and value, any
-   *     object.
+   * Parses config and info filters.
+   *
+   * @param expr Expression to be parsed.
+   * @param valueAsString true, if value has to be interpreted as string, false
+   *     otherwise. It is true for config filters and false for info filters.
+   * @return a {@link TimelineFilterList} object.
+   * @throws TimelineParseException if any problem occurs during parsing.
    */
-  static TimelineFilterList parseKeyStrValueObj(String str, String pairsDelim,
-      String keyValDelim) {
-    return parseKeyValues(str, pairsDelim, keyValDelim, false, false);
+  static TimelineFilterList parseKVFilters(String expr, boolean valueAsString)
+      throws TimelineParseException {
+    return parseFilters(new TimelineParserForKVFilters(expr, valueAsString));
   }
 
   /**
@@ -245,18 +169,16 @@ final class TimelineReaderWebServicesUtils {
     return fieldList;
   }
 
-  static TimelineFilterList parseMetricFilters(String str,
-      String delimiter) {
-    if (str == null || str.isEmpty()) {
-      return null;
-    }
-    TimelineFilterList list = new TimelineFilterList();
-    String[] strs = str.split(delimiter);
-    for (String aStr : strs) {
-      list.addFilter(new TimelineCompareFilter(
-          TimelineCompareOp.GREATER_OR_EQUAL, aStr.trim(), 0L));
-    }
-    return list;
+  /**
+   * Parses metric filters.
+   *
+   * @param expr Metric filter expression to be parsed.
+   * @return a {@link TimelineFilterList} object.
+   * @throws TimelineParseException if any problem occurs during parsing.
+   */
+  static TimelineFilterList parseMetricFilters(String expr)
+      throws TimelineParseException {
+    return parseFilters(new TimelineParserForNumericFilters(expr));
   }
 
   /**
@@ -299,4 +221,16 @@ final class TimelineReaderWebServicesUtils {
   static String getUserName(UserGroupInformation callerUGI) {
     return ((callerUGI != null) ? callerUGI.getUserName().trim() : "");
   }
+
+  /**
+   * Parses confstoretrieve and metricstoretrieve.
+   * @param str String representing confs/metrics to retrieve expression.
+   *
+   * @return a {@link TimelineFilterList} object.
+   * @throws TimelineParseException if any problem occurs during parsing.
+   */
+  static TimelineFilterList parseDataToRetrieve(String expr)
+        throws TimelineParseException {
+    return parseFilters(new TimelineParserForDataToRetrieve(expr));
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/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
index 81902ee..1127f4a 100644
--- 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
@@ -29,12 +29,15 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @Unstable
 public class TimelineCompareFilter extends TimelineFilter {
 
-  private final TimelineCompareOp compareOp;
-  private final String key;
-  private final Object value;
+  private TimelineCompareOp compareOp;
+  private String key;
+  private Object value;
   // If comparison operator is NOT_EQUAL, this flag decides if we should return
   // the entity if key does not exist.
-  private final boolean keyMustExist;
+  private boolean keyMustExist = true;
+
+  public TimelineCompareFilter() {
+  }
 
   public TimelineCompareFilter(TimelineCompareOp op, String key, Object val,
        boolean keyMustExistFlag) {
@@ -65,15 +68,77 @@ public class TimelineCompareFilter extends TimelineFilter {
     return key;
   }
 
+  public void setKey(String keyToBeSet) {
+    key = keyToBeSet;
+  }
+
   public Object getValue() {
     return value;
   }
 
+  public void setCompareOp(TimelineCompareOp timelineCompareOp,
+      boolean keyExistFlag) {
+    this.compareOp = timelineCompareOp;
+    if (timelineCompareOp == TimelineCompareOp.NOT_EQUAL) {
+      this.keyMustExist = keyExistFlag;
+    }
+  }
+
+  public void setValue(Object val) {
+    value = val;
+  }
+
   public boolean getKeyMustExist() {
     return keyMustExist;
   }
 
   @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((compareOp == null) ? 0 : compareOp.hashCode());
+    result = prime * result + ((key == null) ? 0 : key.hashCode());
+    result = prime * result + (keyMustExist ? 1231 : 1237);
+    result = prime * result + ((value == null) ? 0 : value.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    TimelineCompareFilter other = (TimelineCompareFilter) obj;
+    if (compareOp != other.compareOp) {
+      return false;
+    }
+    if (key == null) {
+      if (other.key != null) {
+        return false;
+      }
+    } else if (!key.equals(other.key)) {
+      return false;
+    }
+    if (keyMustExist != other.keyMustExist) {
+      return false;
+    }
+    if (value == null) {
+      if (other.value != null) {
+        return false;
+      }
+    } else if (!value.equals(other.value)) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
   public String toString() {
     return String.format("%s (%s, %s:%s:%b)",
         this.getClass().getSimpleName(), this.compareOp.name(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java
----------------------------------------------------------------------
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/TimelineExistsFilter.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/TimelineExistsFilter.java
index 36d0d7b..b4c8e25 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.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/TimelineExistsFilter.java
@@ -29,8 +29,11 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @Unstable
 public class TimelineExistsFilter extends TimelineFilter {
 
-  private final TimelineCompareOp compareOp;
-  private final String value;
+  private TimelineCompareOp compareOp;
+  private String value;
+
+  public TimelineExistsFilter() {
+  }
 
   public TimelineExistsFilter(TimelineCompareOp op, String value) {
     this.value = value;
@@ -42,14 +45,56 @@ public class TimelineExistsFilter extends TimelineFilter {
   }
 
   @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((compareOp == null) ? 0 : compareOp.hashCode());
+    result = prime * result + ((value == null) ? 0 : value.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    TimelineExistsFilter other = (TimelineExistsFilter) obj;
+    if (compareOp != other.compareOp) {
+      return false;
+    }
+    if (value == null) {
+      if (other.value != null) {
+        return false;
+      }
+    } else if (!value.equals(other.value)) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
   public TimelineFilterType getFilterType() {
     return TimelineFilterType.EXISTS;
   }
 
+  public void setValue(String val) {
+    value = val;
+  }
+
   public String getValue() {
     return value;
   }
 
+  public void setCompareOp(TimelineCompareOp op) {
+    compareOp = op;
+  }
+
   public TimelineCompareOp getCompareOp() {
     return compareOp;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/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
index 57b41a6..b4c7ad2 100644
--- 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
@@ -98,6 +98,42 @@ public class TimelineFilterList extends TimelineFilter {
   }
 
   @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result =
+        prime * result + ((filterList == null) ? 0 : filterList.hashCode());
+    result =
+        prime * result + ((operator == null) ? 0 : operator.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    TimelineFilterList other = (TimelineFilterList) obj;
+    if (operator != other.operator) {
+      return false;
+    }
+    if (filterList == null) {
+      if (other.filterList != null) {
+        return false;
+      }
+    } else if (!filterList.equals(other.filterList)) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
   public String toString() {
     return String.format("TimelineFilterList %s (%d): %s",
         this.operator, this.filterList.size(), this.filterList.toString());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java
----------------------------------------------------------------------
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/TimelineKeyValueFilter.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/TimelineKeyValueFilter.java
index 58f0ee9..8bc8584 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.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/TimelineKeyValueFilter.java
@@ -28,6 +28,9 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @Private
 @Unstable
 public class TimelineKeyValueFilter extends TimelineCompareFilter {
+  public TimelineKeyValueFilter() {
+  }
+
   public TimelineKeyValueFilter(TimelineCompareOp op, String key, Object val,
       boolean keyMustExistFlag) {
     super(op, key, val, keyMustExistFlag);
@@ -45,4 +48,14 @@ public class TimelineKeyValueFilter extends TimelineCompareFilter {
   public TimelineFilterType getFilterType() {
     return TimelineFilterType.KEY_VALUE;
   }
+
+  public void setCompareOp(TimelineCompareOp timelineCompareOp,
+      boolean keyExistFlag) {
+    if (timelineCompareOp != TimelineCompareOp.EQUAL &&
+        timelineCompareOp != TimelineCompareOp.NOT_EQUAL) {
+      throw new IllegalArgumentException("TimelineCompareOp for equality"
+          + " filter should be EQUAL or NOT_EQUAL");
+    }
+    super.setCompareOp(timelineCompareOp, keyExistFlag);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java
----------------------------------------------------------------------
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/TimelineKeyValuesFilter.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/TimelineKeyValuesFilter.java
index 0d34d47..fe4f6b2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.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/TimelineKeyValuesFilter.java
@@ -31,9 +31,13 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 @Private
 @Unstable
 public class TimelineKeyValuesFilter extends TimelineFilter {
-  private final TimelineCompareOp compareOp;
-  private final String key;
-  private final Set<Object> values;
+  private TimelineCompareOp compareOp;
+  private String key;
+  private Set<Object> values;
+
+  public TimelineKeyValuesFilter() {
+  }
+
   public TimelineKeyValuesFilter(TimelineCompareOp op, String key,
       Set<Object> values) {
     if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
@@ -58,6 +62,15 @@ public class TimelineKeyValuesFilter extends TimelineFilter {
     return values;
   }
 
+  public void setKeyAndValues(String keyForValues, Set<Object> vals) {
+    key = keyForValues;
+    values = vals;
+  }
+
+  public void setCompareOp(TimelineCompareOp op) {
+    compareOp = op;
+  }
+
   public TimelineCompareOp getCompareOp() {
     return compareOp;
   }
@@ -68,4 +81,46 @@ public class TimelineKeyValuesFilter extends TimelineFilter {
         this.getClass().getSimpleName(), this.compareOp.name(),
         this.key, (values == null) ? "" : values.toString());
   }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((compareOp == null) ? 0 : compareOp.hashCode());
+    result = prime * result + ((key == null) ? 0 : key.hashCode());
+    result = prime * result + ((values == null) ? 0 : values.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    TimelineKeyValuesFilter other = (TimelineKeyValuesFilter) obj;
+    if (compareOp != other.compareOp) {
+      return false;
+    }
+    if (key == null) {
+      if (other.key != null) {
+        return false;
+      }
+    } else if (!key.equals(other.key)) {
+      return false;
+    }
+    if (values == null) {
+      if (other.values != null) {
+        return false;
+      }
+    } else if (!values.equals(other.values)) {
+      return false;
+    }
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/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
index f36e593..bbdc960 100644
--- 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
@@ -32,6 +32,9 @@ public class TimelinePrefixFilter extends TimelineFilter {
   private TimelineCompareOp compareOp;
   private String prefix;
 
+  public TimelinePrefixFilter() {
+  }
+
   public TimelinePrefixFilter(TimelineCompareOp op, String prefix) {
     this.prefix = prefix;
     if (op != TimelineCompareOp.EQUAL && op != TimelineCompareOp.NOT_EQUAL) {
@@ -59,4 +62,38 @@ public class TimelinePrefixFilter extends TimelineFilter {
     return String.format("%s (%s %s)",
         this.getClass().getSimpleName(), this.compareOp.name(), this.prefix);
   }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((compareOp == null) ? 0 : compareOp.hashCode());
+    result = prime * result + ((prefix == null) ? 0 : prefix.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    TimelinePrefixFilter other = (TimelinePrefixFilter) obj;
+    if (compareOp != other.compareOp) {
+      return false;
+    }
+    if (prefix == null) {
+      if (other.prefix != null) {
+        return false;
+      }
+    } else if (!prefix.equals(other.prefix)){
+      return false;
+    }
+    return true;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/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 2bd2830..6c3f3e3 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
@@ -437,7 +437,7 @@ public class TestTimelineReaderWebServices {
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/app1/entities/app?" +
-          "conffilters=config_1:123,config_3:abc");
+          "conffilters=config_1%20eq%20123%20AND%20config_3%20eq%20abc");
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -457,7 +457,7 @@ public class TestTimelineReaderWebServices {
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/app1/entities/app?" +
-          "infofilters=info2:3.5");
+          "infofilters=info2%20eq%203.5");
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -477,7 +477,7 @@ public class TestTimelineReaderWebServices {
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/app1/entities/app?" +
-          "metricfilters=metric3");
+          "metricfilters=metric3%20ge%200");
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
@@ -519,10 +519,10 @@ public class TestTimelineReaderWebServices {
     try {
       URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/clusters/cluster1/apps/app1/entities/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");
+          "metricfilters=metric7%20ge%200&isrelatedto=type1:tid1_1:tid1_2,"+
+          "type2:tid2_1%60&relatesto=flow:flow1&eventfilters=event_2,event_4" +
+          "&infofilters=info2%20eq%203.5&createdtimestart=1425016502030&" +
+          "createdtimeend=1425016502060");
       ClientResponse resp = getResponse(client, uri);
       Set<TimelineEntity> entities =
           resp.getEntity(new GenericType<Set<TimelineEntity>>(){});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c00fef2/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 009b488..ca80ed5 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
@@ -60,6 +60,8 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
 import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.ClientResponse.Status;
@@ -109,18 +111,20 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     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);
+    Map<Long, Number> metricValues =
+        ImmutableMap.of(ts - 100000, (Number)2, ts - 80000, 40);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    m1 = new TimelineMetric();
+    m1.setId("MAP1_SLOT_MILLIS");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)2, 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);
+    metricValues = ImmutableMap.of(ts - 100000, (Number)31, ts - 80000, 57);
     m1.setType(Type.TIME_SERIES);
     m1.setValues(metricValues);
     metrics.add(m1);
@@ -198,6 +202,14 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     event4.setTimestamp(cTime);
     event4.addInfo("foo_event", "test");
     entity4.addEvent(event4);
+    metrics.clear();
+    m2 = new TimelineMetric();
+    m2.setId("MAP_SLOT_MILLIS");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)5L, ts - 80000, 101L);
+    m2.setType(Type.TIME_SERIES);
+    m2.setValues(metricValues);
+    metrics.add(m2);
+    entity4.addMetrics(metrics);
     te4.addEntity(entity4);
 
     TimelineEntities te5 = new TimelineEntities();
@@ -205,11 +217,116 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     entity5.setId("entity1");
     entity5.setType("type1");
     entity5.setCreatedTime(1425016501034L);
+    // add some config entries
+    entity5.addConfigs(ImmutableMap.of("config_param1", "value1",
+        "config_param2", "value2", "cfg_param1", "value3"));
+    entity5.addInfo(ImmutableMap.of("info1", (Object)"cluster1",
+        "info2", 2.0, "info3", 35000, "info4", 36000));
+    metrics = new HashSet<>();
+    m1 = new TimelineMetric();
+    m1.setId("MAP_SLOT_MILLIS");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)2, ts - 80000, 40);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    m1 = new TimelineMetric();
+    m1.setId("HDFS_BYTES_READ");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)31, ts - 80000, 57);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    entity5.addMetrics(metrics);
+    TimelineEvent event51 = new TimelineEvent();
+    event51.setId("event1");
+    event51.setTimestamp(cTime);
+    entity5.addEvent(event51);
+    TimelineEvent event52 = new TimelineEvent();
+    event52.setId("event2");
+    event52.setTimestamp(cTime);
+    entity5.addEvent(event52);
+    TimelineEvent event53 = new TimelineEvent();
+    event53.setId("event3");
+    event53.setTimestamp(cTime);
+    entity5.addEvent(event53);
+    TimelineEvent event54 = new TimelineEvent();
+    event54.setId("event4");
+    event54.setTimestamp(cTime);
+    entity5.addEvent(event54);
+    Map<String, Set<String>> isRelatedTo1 = new HashMap<String, Set<String>>();
+    isRelatedTo1.put("type2",
+        Sets.newHashSet("entity21","entity22","entity23","entity24"));
+    isRelatedTo1.put("type4", Sets.newHashSet("entity41","entity42"));
+    isRelatedTo1.put("type1", Sets.newHashSet("entity14","entity15"));
+    isRelatedTo1.put("type3",
+        Sets.newHashSet("entity31", "entity35", "entity32", "entity33"));
+    entity5.addIsRelatedToEntities(isRelatedTo1);
+    Map<String, Set<String>> relatesTo1 = new HashMap<String, Set<String>>();
+    relatesTo1.put("type2",
+        Sets.newHashSet("entity21","entity22","entity23","entity24"));
+    relatesTo1.put("type4", Sets.newHashSet("entity41","entity42"));
+    relatesTo1.put("type1", Sets.newHashSet("entity14","entity15"));
+    relatesTo1.put("type3",
+        Sets.newHashSet("entity31", "entity35", "entity32", "entity33"));
+    entity5.addRelatesToEntities(relatesTo1);
     te5.addEntity(entity5);
+
     TimelineEntity entity6 = new TimelineEntity();
     entity6.setId("entity2");
     entity6.setType("type1");
     entity6.setCreatedTime(1425016501034L);
+    entity6.addConfigs(ImmutableMap.of("cfg_param3", "value1",
+        "configuration_param2", "value2", "config_param1", "value3"));
+    entity6.addInfo(ImmutableMap.of("info1", (Object)"cluster2",
+        "info2", 2.0, "info4", 35000));
+    metrics = new HashSet<>();
+    m1 = new TimelineMetric();
+    m1.setId("MAP1_SLOT_MILLIS");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)12, ts - 80000, 140);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    m1 = new TimelineMetric();
+    m1.setId("HDFS_BYTES_READ");
+    metricValues = ImmutableMap.of(ts - 100000, (Number)78, ts - 80000, 157);
+    m1.setType(Type.TIME_SERIES);
+    m1.setValues(metricValues);
+    metrics.add(m1);
+    m1 = new TimelineMetric();
+    m1.setId("MAP11_SLOT_MILLIS");
+    m1.setType(Type.SINGLE_VALUE);
+    m1.addValue(ts - 100000, 122);
+    metrics.add(m1);
+    entity6.addMetrics(metrics);
+    TimelineEvent event61 = new TimelineEvent();
+    event61.setId("event1");
+    event61.setTimestamp(cTime);
+    entity6.addEvent(event61);
+    TimelineEvent event62 = new TimelineEvent();
+    event62.setId("event5");
+    event62.setTimestamp(cTime);
+    entity6.addEvent(event62);
+    TimelineEvent event63 = new TimelineEvent();
+    event63.setId("event3");
+    event63.setTimestamp(cTime);
+    entity6.addEvent(event63);
+    TimelineEvent event64 = new TimelineEvent();
+    event64.setId("event6");
+    event64.setTimestamp(cTime);
+    entity6.addEvent(event64);
+    Map<String, Set<String>> isRelatedTo2 = new HashMap<String, Set<String>>();
+    isRelatedTo2.put("type2",
+        Sets.newHashSet("entity21","entity22","entity23","entity24"));
+    isRelatedTo2.put("type5", Sets.newHashSet("entity51","entity52"));
+    isRelatedTo2.put("type6", Sets.newHashSet("entity61","entity66"));
+    isRelatedTo2.put("type3", Sets.newHashSet("entity31"));
+    entity6.addIsRelatedToEntities(isRelatedTo2);
+    Map<String, Set<String>> relatesTo2 = new HashMap<String, Set<String>>();
+    relatesTo2.put("type2",
+        Sets.newHashSet("entity21","entity22","entity23","entity24"));
+    relatesTo2.put("type5", Sets.newHashSet("entity51","entity52"));
+    relatesTo2.put("type6", Sets.newHashSet("entity61","entity66"));
+    relatesTo2.put("type3", Sets.newHashSet("entity31"));
+    entity6.addRelatesToEntities(relatesTo2);
     te5.addEntity(entity6);
 
     HBaseTimelineWriterImpl hbi = null;
@@ -357,13 +474,15 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
       assertNotNull(entity);
       assertEquals("user1@flow_name/1002345678919", entity.getId());
-      assertEquals(2, entity.getMetrics().size());
+      assertEquals(3, 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);
+      TimelineMetric m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+          "MAP1_SLOT_MILLIS", ts - 80000, 40L);
       for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue(verifyMetrics(metric, m1, m2));
+        assertTrue(verifyMetrics(metric, m1, m2, m3));
       }
 
       // Query without specifying cluster ID.
@@ -373,20 +492,21 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       entity = resp.getEntity(FlowRunEntity.class);
       assertNotNull(entity);
       assertEquals("user1@flow_name/1002345678919", entity.getId());
-      assertEquals(2, entity.getMetrics().size());
+      assertEquals(3, 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);
+      m3 = newMetric(TimelineMetric.Type.SINGLE_VALUE,
+          "MAP1_SLOT_MILLIS", ts - 80000, 40L);
       for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue(verifyMetrics(metric, m1, m2));
+        assertTrue(verifyMetrics(metric, m1, m2, m3));
       }
     } finally {
       client.destroy();
     }
   }
 
-
   @Test
   public void testGetFlowRuns() throws Exception {
     Client client = createClient();
@@ -489,12 +609,57 @@ public class TestTimelineReaderWebServicesHBaseStorage {
             ((entity.getId().equals("user1@flow_name/1002345678919")) &&
             (entity.getRunId() == 1002345678919L) &&
             (entity.getStartTime() == 1425016501000L) &&
-            (entity.getMetrics().size() == 2)) ||
+            (entity.getMetrics().size() == 3)) ||
             ((entity.getId().equals("user1@flow_name/1002345678920")) &&
             (entity.getRunId() == 1002345678920L) &&
             (entity.getStartTime() == 1425016501034L) &&
-            (entity.getMetrics().size() == 0)));
+            (entity.getMetrics().size() == 1)));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+
+  @Test
+  public void testGetFlowRunsMetricsToRetrieve() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
+          "metricstoretrieve=MAP_,HDFS_");
+      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());
+      int metricCnt = 0;
+      for (FlowRunEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP_") ||
+              metric.getId().startsWith("HDFS_"));
+        }
+      }
+      assertEquals(3, metricCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/users/user1/flows/flow_name/runs?" +
+          "metricstoretrieve=!(MAP_,HDFS_)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<FlowRunEntity>>(){});
+      assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getType());
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      metricCnt = 0;
+      for (FlowRunEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP1_"));
+        }
       }
+      assertEquals(1, metricCnt);
     } finally {
       client.destroy();
     }
@@ -858,15 +1023,18 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       TimelineEntity entity = resp.getEntity(TimelineEntity.class);
       assertNotNull(entity);
       assertEquals("application_1111111111_1111", entity.getId());
-      assertEquals(2, entity.getMetrics().size());
+      assertEquals(3, 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);
+      TimelineMetric m3 = newMetric(TimelineMetric.Type.TIME_SERIES,
+          "MAP1_SLOT_MILLIS", ts - 100000, 2L);
+      m3.addValue(ts - 80000, 40L);
       for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue(verifyMetrics(metric, m1, m2));
+        assertTrue(verifyMetrics(metric, m1, m2, m3));
       }
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
@@ -877,11 +1045,11 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       assertNotNull(entity);
       assertEquals("application_1111111111_2222", entity.getId());
       assertEquals(1, entity.getMetrics().size());
-      TimelineMetric m3 = newMetric(TimelineMetric.Type.TIME_SERIES,
+      TimelineMetric m4 = newMetric(TimelineMetric.Type.TIME_SERIES,
          "MAP_SLOT_MILLIS", ts - 100000, 5L);
-      m2.addValue(ts - 80000, 101L);
+      m4.addValue(ts - 80000, 101L);
       for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue(verifyMetrics(metric, m3));
+        assertTrue(verifyMetrics(metric, m4));
       }
     } finally {
         client.destroy();
@@ -899,15 +1067,18 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       TimelineEntity entity = resp.getEntity(TimelineEntity.class);
       assertNotNull(entity);
       assertEquals("application_1111111111_1111", entity.getId());
-      assertEquals(2, entity.getMetrics().size());
+      assertEquals(3, 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);
+      TimelineMetric m3 = newMetric(TimelineMetric.Type.TIME_SERIES,
+          "MAP1_SLOT_MILLIS", ts - 100000, 2L);
+      m3.addValue(ts - 80000, 40L);
       for (TimelineMetric metric : entity.getMetrics()) {
-        assertTrue(verifyMetrics(metric, m1, m2));
+        assertTrue(verifyMetrics(metric, m1, m2, m3));
       }
     } finally {
       client.destroy();
@@ -952,6 +1123,683 @@ public class TestTimelineReaderWebServicesHBaseStorage {
     }
   }
 
+  /**
+   * Tests if specific configs and metrics are retrieve for getEntities call.
+   */
+  @Test
+  public void testGetEntitiesDataToRetrieve() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?confstoretrieve=cfg_");
+      ClientResponse resp = getResponse(client, uri);
+      Set<TimelineEntity> entities =
+          resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      int cfgCnt = 0;
+      for (TimelineEntity entity : entities) {
+        cfgCnt += entity.getConfigs().size();
+        for (String configKey : entity.getConfigs().keySet()) {
+          assertTrue(configKey.startsWith("cfg_"));
+        }
+      }
+      assertEquals(2, cfgCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?confstoretrieve=cfg_,config_");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      cfgCnt = 0;
+      for (TimelineEntity entity : entities) {
+        cfgCnt += entity.getConfigs().size();
+        for (String configKey : entity.getConfigs().keySet()) {
+          assertTrue(configKey.startsWith("cfg_") ||
+              configKey.startsWith("config_"));
+        }
+      }
+      assertEquals(5, cfgCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?confstoretrieve=!(cfg_,config_)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      cfgCnt = 0;
+      for (TimelineEntity entity : entities) {
+        cfgCnt += entity.getConfigs().size();
+        for (String configKey : entity.getConfigs().keySet()) {
+          assertTrue(configKey.startsWith("configuration_"));
+        }
+      }
+      assertEquals(1, cfgCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricstoretrieve=MAP_");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      int metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP_"));
+        }
+      }
+      assertEquals(1, metricCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricstoretrieve=MAP1_,HDFS_");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP1_") ||
+              metric.getId().startsWith("HDFS_"));
+        }
+      }
+      assertEquals(3, metricCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricstoretrieve=!(MAP1_,HDFS_)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(2, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP_") ||
+              metric.getId().startsWith("MAP11_"));
+        }
+      }
+      assertEquals(2, metricCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricstoretrieve=!(MAP1_,HDFS_");
+      verifyHttpResponse(client, uri, Status.BAD_REQUEST);
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesConfigFilters() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=config_param1%20eq%20value1%20OR%20" +
+          "config_param1%20eq%20value3");
+      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"));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=config_param1%20eq%20value1%20AND" +
+          "%20configuration_param2%20eq%20value2");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+
+      // conffilters=(config_param1 eq value1 AND configuration_param2 eq
+      // value2) OR (config_param1 eq value3 AND cfg_param3 eq value1)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" +
+          "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" +
+          "%20value3%20AND%20cfg_param3%20eq%20value1)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      int cfgCnt = 0;
+      for (TimelineEntity entity : entities) {
+        cfgCnt += entity.getConfigs().size();
+        assertTrue(entity.getId().equals("entity2"));
+      }
+      assertEquals(0, cfgCnt);
+
+      // conffilters=(config_param1 eq value1 AND configuration_param2 eq
+      // value2) OR (config_param1 eq value3 AND cfg_param3 eq value1)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" +
+          "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" +
+          "%20value3%20AND%20cfg_param3%20eq%20value1)&fields=CONFIGS");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      cfgCnt = 0;
+      for (TimelineEntity entity : entities) {
+        cfgCnt += entity.getConfigs().size();
+        assertTrue(entity.getId().equals("entity2"));
+      }
+      assertEquals(3, cfgCnt);
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=(config_param1%20eq%20value1%20AND" +
+          "%20configuration_param2%20eq%20value2)%20OR%20(config_param1%20eq" +
+          "%20value3%20AND%20cfg_param3%20eq%20value1)&confstoretrieve=cfg_," +
+          "configuration_");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      cfgCnt = 0;
+      for (TimelineEntity entity : entities) {
+        cfgCnt += entity.getConfigs().size();
+        assertTrue(entity.getId().equals("entity2"));
+        for (String configKey : entity.getConfigs().keySet()) {
+          assertTrue(configKey.startsWith("cfg_") ||
+              configKey.startsWith("configuration_"));
+        }
+      }
+      assertEquals(2, cfgCnt);
+
+      // Test for behavior when compare op is ne(not equals) vs ene
+      // (exists and not equals). configuration_param2 does not exist for
+      // entity1. For ne, both entity1 and entity2 will be returned. For ene,
+      // only entity2 will be returned as we are checking for existence too.
+      // conffilters=configuration_param2 ne value3
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=configuration_param2%20ne%20value3");
+      resp = getResponse(client, uri);
+      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"));
+      }
+      // conffilters=configuration_param2 ene value3
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?conffilters=configuration_param2%20ene%20value3");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity2"));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesInfoFilters() throws Exception {
+    Client client = createClient();
+    try {
+      // infofilters=info1 eq cluster1 OR info1 eq cluster2
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=info1%20eq%20cluster1%20OR%20info1%20eq" +
+          "%20cluster2");
+      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"));
+      }
+
+      // infofilters=info1 eq cluster1 AND info4 eq 35000
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=info1%20eq%20cluster1%20AND%20info4%20" +
+          "eq%2035000");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+
+      // infofilters=info4 eq 35000 OR info4 eq 36000
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=info4%20eq%2035000%20OR%20info4%20eq" +
+          "%2036000");
+      resp = getResponse(client, uri);
+      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"));
+      }
+
+      // infofilters=(info1 eq cluster1 AND info4 eq 35000) OR
+      // (info1 eq cluster2 AND info2 eq 2.0)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=(info1%20eq%20cluster1%20AND%20info4%20" +
+          "eq%2035000)%20OR%20(info1%20eq%20cluster2%20AND%20info2%20eq%202.0)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      int infoCnt = 0;
+      for (TimelineEntity entity : entities) {
+        infoCnt += entity.getInfo().size();
+        assertTrue(entity.getId().equals("entity2"));
+      }
+      // Includes UID in info field even if fields not specified as INFO.
+      assertEquals(1, infoCnt);
+
+      // infofilters=(info1 eq cluster1 AND info4 eq 35000) OR
+      // (info1 eq cluster2 AND info2 eq 2.0)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=(info1%20eq%20cluster1%20AND%20info4%20" +
+          "eq%2035000)%20OR%20(info1%20eq%20cluster2%20AND%20info2%20eq%20" +
+          "2.0)&fields=INFO");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      infoCnt = 0;
+      for (TimelineEntity entity : entities) {
+        infoCnt += entity.getInfo().size();
+        assertTrue(entity.getId().equals("entity2"));
+      }
+      // Includes UID in info field.
+      assertEquals(4, infoCnt);
+
+      // Test for behavior when compare op is ne(not equals) vs ene
+      // (exists and not equals). info3 does not exist for entity2. For ne,
+      // both entity1 and entity2 will be returned. For ene, only entity2 will
+      // be returned as we are checking for existence too.
+      // infofilters=info3 ne 39000
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=info3%20ne%2039000");
+      resp = getResponse(client, uri);
+      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"));
+      }
+      // infofilters=info3 ene 39000
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?infofilters=info3%20ene%2039000");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity1"));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesMetricFilters() throws Exception {
+    Client client = createClient();
+    try {
+      // metricfilters=HDFS_BYTES_READ lt 60 OR HDFS_BYTES_READ eq 157
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=HDFS_BYTES_READ%20lt%2060%20OR%20" +
+          "HDFS_BYTES_READ%20eq%20157");
+      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"));
+      }
+
+      // metricfilters=HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=HDFS_BYTES_READ%20lt%2060%20AND%20" +
+          "MAP_SLOT_MILLIS%20gt%2040");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+
+      // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR
+      // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
+          "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
+          "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      int metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        assertTrue(entity.getId().equals("entity2"));
+      }
+      assertEquals(0, metricCnt);
+
+      // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR
+      // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
+          "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
+          "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)&fields=METRICS");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        assertTrue(entity.getId().equals("entity2"));
+      }
+      assertEquals(3, metricCnt);
+
+      // metricfilters=(HDFS_BYTES_READ lt 60 AND MAP_SLOT_MILLIS gt 40) OR
+      // (MAP1_SLOT_MILLIS ge 140 AND MAP11_SLOT_MILLIS le 122)
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=(HDFS_BYTES_READ%20lt%2060%20AND%20" +
+          "MAP_SLOT_MILLIS%20gt%2040)%20OR%20(MAP1_SLOT_MILLIS%20ge" +
+          "%20140%20AND%20MAP11_SLOT_MILLIS%20le%20122)&metricstoretrieve=" +
+          "!(HDFS)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      metricCnt = 0;
+      for (TimelineEntity entity : entities) {
+        metricCnt += entity.getMetrics().size();
+        assertTrue(entity.getId().equals("entity2"));
+        for (TimelineMetric metric : entity.getMetrics()) {
+          assertTrue(metric.getId().startsWith("MAP1"));
+        }
+      }
+      assertEquals(2, metricCnt);
+
+      // Test for behavior when compare op is ne(not equals) vs ene
+      // (exists and not equals). MAP11_SLOT_MILLIS does not exist for
+      // entity1. For ne, both entity1 and entity2 will be returned. For ene,
+      // only entity2 will be returned as we are checking for existence too.
+      // metricfilters=MAP11_SLOT_MILLIS ne 100
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=MAP11_SLOT_MILLIS%20ne%20100");
+      resp = getResponse(client, uri);
+      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"));
+      }
+      // metricfilters=MAP11_SLOT_MILLIS ene 100
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?metricfilters=MAP11_SLOT_MILLIS%20ene%20100");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity2"));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
+  @Test
+  public void testGetEntitiesEventFilters() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?eventfilters=event1,event3");
+      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"));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?eventfilters=!(event1,event3)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+
+      // eventfilters=!(event1,event3) OR event5,event6
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?eventfilters=!(event1,event3)%20OR%20event5,event6");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity2"));
+      }
+
+      //  eventfilters=(!(event1,event3) OR event5,event6) OR
+      // (event1,event2 AND (event3,event4))
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?eventfilters=(!(event1,event3)%20OR%20event5," +
+          "event6)%20OR%20(event1,event2%20AND%20(event3,event4))");
+      resp = getResponse(client, uri);
+      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 testGetEntitiesRelationFilters() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1?isrelatedto=type3:entity31,type2:entity21:entity22");
+      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"));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
+          "?isrelatedto=!(type3:entity31,type2:entity21:entity22)");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(0, entities.size());
+
+      // isrelatedto=!(type3:entity31,type2:entity21:entity22)OR type5:entity51,
+      // type6:entity61:entity66
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
+          "?isrelatedto=!(type3:entity31,type2:entity21:entity22)%20OR%20" +
+          "type5:entity51,type6:entity61:entity66");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity2"));
+      }
+
+      // isrelatedto=(!(type3:entity31,type2:entity21:entity22)OR type5:
+      // entity51,type6:entity61:entity66) OR (type1:entity14,type2:entity21:
+      // entity22 AND (type3:entity32:entity35,type4:entity42))
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
+          "?isrelatedto=(!(type3:entity31,type2:entity21:entity22)%20OR%20" +
+          "type5:entity51,type6:entity61:entity66)%20OR%20(type1:entity14," +
+          "type2:entity21:entity22%20AND%20(type3:entity32:entity35,"+
+          "type4:entity42))");
+      resp = getResponse(client, uri);
+      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"));
+      }
+
+      // relatesto=!(type3:entity31,type2:entity21:entity22)OR type5:entity51,
+      // type6:entity61:entity66
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
+          "?relatesto=!%20(type3:entity31,type2:entity21:entity22%20)%20OR%20" +
+          "type5:entity51,type6:entity61:entity66");
+      resp = getResponse(client, uri);
+      entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
+      assertNotNull(entities);
+      assertEquals(1, entities.size());
+      for (TimelineEntity entity : entities) {
+        assertTrue(entity.getId().equals("entity2"));
+      }
+
+      // relatesto=(!(type3:entity31,type2:entity21:entity22)OR type5:entity51,
+      // type6:entity61:entity66) OR (type1:entity14,type2:entity21:entity22 AND
+      // (type3:entity32:entity35 , type4:entity42))
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/timeline/" +
+          "clusters/cluster1/apps/application_1111111111_1111/entities/type1" +
+          "?relatesto=(!(%20type3:entity31,type2:entity21:entity22)%20OR%20" +
+          "type5:entity51,type6:entity61:entity66%20)%20OR%20(type1:entity14," +
+          "type2:entity21:entity22%20AND%20(type3:entity32:entity35%20,%20"+
+          "type4:entity42))");
+      resp = getResponse(client, uri);
+      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();
+    }
+  }
+
+  /**
+   * Tests if specific configs and metrics are retrieve for getEntity call.
+   */
+  @Test
+  public void testGetEntityDataToRetrieve() throws Exception {
+    Client client = createClient();
+    try {
+      URI uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1/entity2?confstoretrieve=cfg_,configuration_");
+      ClientResponse resp = getResponse(client, uri);
+      TimelineEntity entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("entity2", entity.getId());
+      assertEquals("type1", entity.getType());
+      assertEquals(2, entity.getConfigs().size());
+      for (String configKey : entity.getConfigs().keySet()) {
+        assertTrue(configKey.startsWith("configuration_") ||
+            configKey.startsWith("cfg_"));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1/entity2?confstoretrieve=!(cfg_,configuration_)");
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("entity2", entity.getId());
+      assertEquals("type1", entity.getType());
+      assertEquals(1, entity.getConfigs().size());
+      for (String configKey : entity.getConfigs().keySet()) {
+        assertTrue(configKey.startsWith("config_"));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1/entity2?metricstoretrieve=MAP1_,HDFS_");
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("entity2", entity.getId());
+      assertEquals("type1", entity.getType());
+      assertEquals(2, entity.getMetrics().size());
+      for (TimelineMetric  metric : entity.getMetrics()) {
+        assertTrue(metric.getId().startsWith("MAP1_") ||
+            metric.getId().startsWith("HDFS_"));
+      }
+
+      uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
+          "timeline/clusters/cluster1/apps/application_1111111111_1111/" +
+          "entities/type1/entity2?metricstoretrieve=!(MAP1_,HDFS_)");
+      resp = getResponse(client, uri);
+      entity = resp.getEntity(TimelineEntity.class);
+      assertNotNull(entity);
+      assertEquals("entity2", entity.getId());
+      assertEquals("type1", entity.getType());
+      assertEquals(1, entity.getMetrics().size());
+      for (TimelineMetric  metric : entity.getMetrics()) {
+        assertTrue(metric.getId().startsWith("MAP11_"));
+      }
+    } finally {
+      client.destroy();
+    }
+  }
+
   @Test
   public void testGetFlowRunApps() throws Exception {
     Client client = createClient();
@@ -967,7 +1815,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       for (TimelineEntity entity : entities) {
         assertTrue("Unexpected app in result",
             (entity.getId().equals("application_1111111111_1111") &&
-            entity.getMetrics().size() == 2) ||
+            entity.getMetrics().size() == 3) ||
             (entity.getId().equals("application_1111111111_2222") &&
             entity.getMetrics().size() == 1));
       }
@@ -1007,11 +1855,11 @@ public class TestTimelineReaderWebServicesHBaseStorage {
       for (TimelineEntity entity : entities) {
         assertTrue("Unexpected app in result",
             (entity.getId().equals("application_1111111111_1111") &&
-            entity.getMetrics().size() == 2) ||
+            entity.getMetrics().size() == 3) ||
             (entity.getId().equals("application_1111111111_2222") &&
             entity.getMetrics().size() == 1) ||
             (entity.getId().equals("application_1111111111_2224") &&
-            entity.getMetrics().size() == 0));
+            entity.getMetrics().size() == 1));
       }
 
       // Query without specifying cluster ID.
@@ -1051,7 +1899,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" +
-          "metricfilters=HDFS_BYTES_READ");
+          "metricfilters=HDFS_BYTES_READ%20ge%200");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
       assertNotNull(entities);
@@ -1061,7 +1909,7 @@ public class TestTimelineReaderWebServicesHBaseStorage {
 
       uri = URI.create("http://localhost:" + serverPort + "/ws/v2/" +
           "timeline/clusters/cluster1/users/user1/flows/flow_name/apps?" +
-          "conffilters=cfg1:value1");
+          "conffilters=cfg1%20eq%20value1");
       resp = getResponse(client, uri);
       entities = resp.getEntity(new GenericType<Set<TimelineEntity>>(){});
       assertNotNull(entities);


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


[38/50] [abbrv] hadoop git commit: YARN-5111. YARN container system metrics are not aggregated to application (Naganarasimha G R via sjlee)

Posted by vr...@apache.org.
YARN-5111. YARN container system metrics are not aggregated to application (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/c998a525
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c998a525
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c998a525

Branch: refs/heads/YARN-2928
Commit: c998a5252a43a5ee512f7e11d19fe438bb449510
Parents: 4ad33a0
Author: Sangjin Lee <sj...@apache.org>
Authored: Fri May 27 22:31:00 2016 -0700
Committer: Vrushali <vr...@twitter.com>
Committed: Sun Jun 19 00:20:12 2016 -0700

----------------------------------------------------------------------
 .../api/records/timelineservice/TimelineMetric.java  | 15 +++++++++------
 1 file changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c998a525/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 f0c6849..5c908d6 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
@@ -17,17 +17,18 @@
  */
 package org.apache.hadoop.yarn.api.records.timelineservice;
 
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.TreeMap;
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
-import java.util.Collections;
-import java.util.Map;
-import java.util.TreeMap;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 
 /**
  * This class contains the information of a metric that is related to some
@@ -89,6 +90,8 @@ public class TimelineMetric {
    *
    * @return Real time aggregation operation
    */
+  // required by JAXB
+  @XmlElement(name = "aggregationOp")
   public TimelineMetricOperation getRealtimeAggregationOp() {
     return realtimeAggregationOp;
   }


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


[46/50] [abbrv] hadoop git commit: YARN-5170. Eliminate singleton converters and static method access. (Joep Rottinghuis via Varun Saxena)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/EventColumnNameConverter.java
----------------------------------------------------------------------
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/EventColumnNameConverter.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/EventColumnNameConverter.java
index 32ef1c3..d3ef897 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/EventColumnNameConverter.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/EventColumnNameConverter.java
@@ -30,14 +30,8 @@ import org.apache.hadoop.hbase.util.Bytes;
  */
 public final class EventColumnNameConverter
     implements KeyConverter<EventColumnName> {
-  private static final EventColumnNameConverter INSTANCE =
-      new EventColumnNameConverter();
 
-  public static EventColumnNameConverter getInstance() {
-    return INSTANCE;
-  }
-
-  private EventColumnNameConverter() {
+  public EventColumnNameConverter() {
   }
 
   // eventId=timestamp=infokey are of types String, Long String
@@ -69,7 +63,7 @@ public final class EventColumnNameConverter
       return Separator.VALUES.join(first, Separator.EMPTY_BYTES);
     }
     byte[] second = Bytes.toBytes(
-        TimelineStorageUtils.invertLong(key.getTimestamp()));
+        LongConverter.invertLong(key.getTimestamp()));
     if (key.getInfoKey() == null) {
       return Separator.VALUES.join(first, second, Separator.EMPTY_BYTES);
     }
@@ -96,7 +90,7 @@ public final class EventColumnNameConverter
     }
     String id = Separator.decode(Bytes.toString(components[0]),
         Separator.VALUES, Separator.TAB, Separator.SPACE);
-    Long ts = TimelineStorageUtils.invertLong(Bytes.toLong(components[1]));
+    Long ts = LongConverter.invertLong(Bytes.toLong(components[1]));
     String infoKey = components[2].length == 0 ? null :
         Separator.decode(Bytes.toString(components[2]),
             Separator.VALUES, Separator.TAB, Separator.SPACE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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
index 48c56f9..600601a 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/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
@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage.common;
 
 import java.io.IOException;
+import java.io.Serializable;
 
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -26,14 +27,15 @@ 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();
+public final class LongConverter implements NumericValueConverter,
+    Serializable {
 
-  private LongConverter() {
-  }
+  /**
+   * Added because we implement Comparator<Number>.
+   */
+  private static final long serialVersionUID = 1L;
 
-  public static LongConverter getInstance() {
-    return INSTANCE;
+  public LongConverter() {
   }
 
   @Override
@@ -76,4 +78,17 @@ public final class LongConverter implements NumericValueConverter {
     }
     return sum;
   }
+
+  /**
+   * 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;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java
----------------------------------------------------------------------
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/LongKeyConverter.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/LongKeyConverter.java
index 3954145..4a724d6 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/LongKeyConverter.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/LongKeyConverter.java
@@ -23,13 +23,13 @@ import java.io.IOException;
  * Encodes and decodes column names / row keys which are long.
  */
 public final class LongKeyConverter implements KeyConverter<Long> {
-  private static final LongKeyConverter INSTANCE = new LongKeyConverter();
 
-  public static LongKeyConverter getInstance() {
-    return INSTANCE;
-  }
+  /**
+   * To delegate the actual work to.
+   */
+  private final LongConverter longConverter = new LongConverter();
 
-  private LongKeyConverter() {
+  public LongKeyConverter() {
   }
 
   /*
@@ -44,7 +44,7 @@ public final class LongKeyConverter implements KeyConverter<Long> {
     try {
       // IOException will not be thrown here as we are explicitly passing
       // Long.
-      return LongConverter.getInstance().encodeValue(key);
+      return longConverter.encodeValue(key);
     } catch (IOException e) {
       return null;
     }
@@ -60,7 +60,7 @@ public final class LongKeyConverter implements KeyConverter<Long> {
   @Override
   public Long decode(byte[] bytes) {
     try {
-      return (Long) LongConverter.getInstance().decodeValue(bytes);
+      return (Long) longConverter.decodeValue(bytes);
     } catch (IOException e) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java
----------------------------------------------------------------------
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/RowKeyPrefix.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/RowKeyPrefix.java
new file mode 100644
index 0000000..6159dc7
--- /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/RowKeyPrefix.java
@@ -0,0 +1,42 @@
+/**
+ * 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;
+
+/**
+ * In queries where a single result is needed, an exact rowkey can be used
+ * through the corresponding rowkey#getRowKey() method. For queries that need to
+ * scan over a range of rowkeys, a partial (the initial part) of rowkeys are
+ * used. Classes implementing RowKeyPrefix indicate that they are the initial
+ * part of rowkeys, with different constructors with fewer number of argument to
+ * form a partial rowkey, a prefix.
+ *
+ * @param <R> indicating the type of rowkey that a particular implementation is
+ *          a prefix for.
+ */
+public interface RowKeyPrefix<R> {
+
+  /**
+   * Create a row key prefix, meaning a partial rowkey that can be used in range
+   * scans. Which fields are included in the prefix will depend on the
+   * constructor of the specific instance that was used. Output depends on which
+   * constructor was used.
+   * @return a prefix of the following form {@code fist!second!...!last!}
+   */
+  byte[] getRowKeyPrefix();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java
----------------------------------------------------------------------
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/StringKeyConverter.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/StringKeyConverter.java
index b0f6d55..282848e 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/StringKeyConverter.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/StringKeyConverter.java
@@ -24,13 +24,8 @@ package org.apache.hadoop.yarn.server.timelineservice.storage.common;
  * added later, if required in the associated ColumnPrefix implementations.
  */
 public final class StringKeyConverter implements KeyConverter<String> {
-  private static final StringKeyConverter INSTANCE = new StringKeyConverter();
 
-  public static StringKeyConverter getInstance() {
-    return INSTANCE;
-  }
-
-  private StringKeyConverter() {
+  public StringKeyConverter() {
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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 d52a5d7..aa9a793 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
@@ -18,14 +18,12 @@
 package org.apache.hadoop.yarn.server.timelineservice.storage.common;
 
 import java.io.IOException;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.SortedSet;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -37,13 +35,10 @@ 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.client.Result;
 import org.apache.hadoop.hbase.util.Bytes;
 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.reader.filter.TimelineCompareFilter;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineCompareOp;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineExistsFilter;
@@ -52,7 +47,6 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilte
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineFilterList;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValueFilter;
 import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelineKeyValuesFilter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 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;
@@ -73,19 +67,6 @@ public final class TimelineStorageUtils {
   public static final long MILLIS_ONE_DAY = 86400000L;
 
   /**
-   * 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).
@@ -164,66 +145,6 @@ public final class TimelineStorageUtils {
   }
 
   /**
-   * checks if an application has finished.
-   *
-   * @param te TimlineEntity object.
-   * @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;
-  }
-
-  /**
-   * Check if we have a certain field amongst fields to retrieve. This method
-   * checks against {@link Field#ALL} as well because that would mean field
-   * passed needs to be matched.
-   *
-   * @param fieldsToRetrieve fields to be retrieved.
-   * @param requiredField fields to be checked in fieldsToRetrieve.
-   * @return true if has the required field, false otherwise.
-   */
-  public static boolean hasField(EnumSet<Field> fieldsToRetrieve,
-      Field requiredField) {
-    return fieldsToRetrieve.contains(Field.ALL) ||
-        fieldsToRetrieve.contains(requiredField);
-  }
-
-  /**
-   * Checks if the input TimelineEntity object is an ApplicationEntity.
-   *
-   * @param te TimelineEntity object.
-   * @return true if input is an ApplicationEntity, false otherwise
-   */
-  public static boolean isApplicationEntity(TimelineEntity te) {
-    return te.getType().equals(TimelineEntityType.YARN_APPLICATION.toString());
-  }
-
-  /**
-   * @param te TimelineEntity object.
-   * @param eventId event with this id needs to be fetched
-   * @return TimelineEvent if TimelineEntity contains the desired event.
-   */
-  public static TimelineEvent getApplicationEvent(TimelineEntity te,
-      String eventId) {
-    if (isApplicationEntity(te)) {
-      for (TimelineEvent event : te.getEvents()) {
-        if (event.getId().equals(eventId)) {
-          return event;
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
    * Returns the first seen aggregation operation as seen in the list of input
    * tags or null otherwise.
    *
@@ -646,98 +567,6 @@ public final class TimelineStorageUtils {
     return appId;
   }
 
-  /**
-   * Helper method for reading relationship.
-   *
-   * @param <T> Describes the type of column prefix.
-   * @param entity entity to fill.
-   * @param result result from HBase.
-   * @param prefix column prefix.
-   * @param isRelatedTo if true, means relationship is to be added to
-   *     isRelatedTo, otherwise its added to relatesTo.
-   * @throws IOException if any problem is encountered while reading result.
-   */
-  public 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, StringKeyConverter.getInstance());
-    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.
-   *
-   * @param <T> Describes the type of column prefix.
-   * @param entity entity to fill.
-   * @param result result from HBase.
-   * @param prefix column prefix.
-   * @param isConfig if true, means we are reading configs, otherwise info.
-   * @throws IOException if any problem is encountered while reading result.
-   */
-  public 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, StringKeyConverter.getInstance());
-    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.
-   *
-   * @param <T> Describes the type of column prefix.
-   * @param entity entity to fill.
-   * @param result HBase Result.
-   * @param prefix column prefix.
-   * @throws IOException if any problem is encountered while reading result.
-   */
-  public static <T> void readEvents(TimelineEntity entity, Result result,
-      ColumnPrefix<T> prefix) throws IOException {
-    Map<String, TimelineEvent> eventsMap = new HashMap<>();
-    Map<EventColumnName, Object> eventsResult =
-        prefix.readResults(result, EventColumnNameConverter.getInstance());
-    for (Map.Entry<EventColumnName, Object>
-             eventResult : eventsResult.entrySet()) {
-      EventColumnName eventColumnName = eventResult.getKey();
-      String key = eventColumnName.getId() +
-          Long.toString(eventColumnName.getTimestamp());
-      // Retrieve previously seen event to add to it
-      TimelineEvent event = eventsMap.get(key);
-      if (event == null) {
-        // First time we're seeing this event, add it to the eventsMap
-        event = new TimelineEvent();
-        event.setId(eventColumnName.getId());
-        event.setTimestamp(eventColumnName.getTimestamp());
-        eventsMap.put(key, event);
-      }
-      if (eventColumnName.getInfoKey() != null) {
-        event.addInfo(eventColumnName.getInfoKey(), eventResult.getValue());
-      }
-    }
-    Set<TimelineEvent> eventsSet = new HashSet<>(eventsMap.values());
-    entity.addEvents(eventsSet);
-  }
-
   public static boolean isFlowRunTable(HRegionInfo hRegionInfo,
       Configuration conf) {
     String regionTableName = hRegionInfo.getTable().getNameAsString();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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 775879a..93b4b36 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
@@ -49,8 +49,7 @@ public enum EntityColumn implements Column<EntityTable> {
   /**
    * When the entity was created.
    */
-  CREATED_TIME(EntityColumnFamily.INFO, "created_time",
-      LongConverter.getInstance()),
+  CREATED_TIME(EntityColumnFamily.INFO, "created_time", new LongConverter()),
 
   /**
    * The version of the flow that this entity belongs to.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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 02a4bb3..e410549 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
@@ -67,8 +67,7 @@ public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> {
   /**
    * Metrics are stored with the metric name as the column name.
    */
-  METRIC(EntityColumnFamily.METRICS, null,
-      LongConverter.getInstance());
+  METRIC(EntityColumnFamily.METRICS, null, new LongConverter());
 
   private final ColumnHelper<EntityTable> column;
   private final ColumnFamily<EntityTable> columnFamily;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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 6d08390..ff22178 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
@@ -17,6 +17,12 @@
  */
 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.AppIdKeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+
 /**
  * Represents a rowkey for the entity table.
  */
@@ -28,6 +34,8 @@ public class EntityRowKey {
   private final String appId;
   private final String entityType;
   private final String entityId;
+  private final KeyConverter<EntityRowKey> entityRowKeyConverter =
+      new EntityRowKeyConverter();
 
   public EntityRowKey(String clusterId, String userId, String flowName,
       Long flowRunId, String appId, String entityType, String entityId) {
@@ -69,61 +77,14 @@ public class EntityRowKey {
   }
 
   /**
-   * Constructs a row key prefix for the entity table as follows:
-   * {@code userName!clusterId!flowName!flowRunId!AppId}.
-   *
-   * @param clusterId Context cluster id.
-   * @param userId User name.
-   * @param flowName Flow name.
-   * @param flowRunId Run Id for the flow.
-   * @param appId Application Id.
-   * @return byte array with the row key prefix.
-   */
-  public static byte[] getRowKeyPrefix(String clusterId, String userId,
-      String flowName, Long flowRunId, String appId) {
-    return EntityRowKeyConverter.getInstance().encode(new EntityRowKey(
-        clusterId, userId, flowName, flowRunId, appId, null, null));
-  }
-
-  /**
-   * Constructs a row key prefix for the entity table as follows:
-   * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!}.
-   * Typically used while querying multiple entities of a particular entity
-   * type.
-   *
-   * @param clusterId Context cluster id.
-   * @param userId User name.
-   * @param flowName Flow name.
-   * @param flowRunId Run Id for the flow.
-   * @param appId Application Id.
-   * @param entityType Entity type.
-   * @return byte array with the row key prefix.
-   */
-  public static byte[] getRowKeyPrefix(String clusterId, String userId,
-      String flowName, Long flowRunId, String appId, String entityType) {
-    return EntityRowKeyConverter.getInstance().encode(new EntityRowKey(
-        clusterId, userId, flowName, flowRunId, appId, entityType, null));
-  }
-
-  /**
    * Constructs a row key for the entity table as follows:
    * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}.
    * Typically used while querying a specific entity.
    *
-   * @param clusterId Context cluster id.
-   * @param userId User name.
-   * @param flowName Flow name.
-   * @param flowRunId Run Id for the flow.
-   * @param appId Application Id.
-   * @param entityType Entity type.
-   * @param entityId Entity Id.
    * @return byte array with the row key.
    */
-  public static byte[] getRowKey(String clusterId, String userId,
-      String flowName, Long flowRunId, String appId, String entityType,
-      String entityId) {
-    return EntityRowKeyConverter.getInstance().encode(new EntityRowKey(
-        clusterId, userId, flowName, flowRunId, appId, entityType, entityId));
+  public byte[] getRowKey() {
+    return entityRowKeyConverter.encode(this);
   }
 
   /**
@@ -133,6 +94,132 @@ public class EntityRowKey {
    * @return An <cite>EntityRowKey</cite> object.
    */
   public static EntityRowKey parseRowKey(byte[] rowKey) {
-    return EntityRowKeyConverter.getInstance().decode(rowKey);
+    return new EntityRowKeyConverter().decode(rowKey);
+  }
+
+  /**
+   * Encodes and decodes row key for entity table. The row key is of the form :
+   * userName!clusterId!flowName!flowRunId!appId!entityType!entityId. flowRunId
+   * is a long, appId is encoded/decoded using {@link AppIdKeyConverter} and
+   * rest are strings.
+   * <p>
+   */
+  final private static class EntityRowKeyConverter implements
+      KeyConverter<EntityRowKey> {
+
+    private final AppIdKeyConverter appIDKeyConverter = new AppIdKeyConverter();
+
+    private EntityRowKeyConverter() {
+    }
+
+    /**
+     * Entity row key is of the form
+     * userName!clusterId!flowName!flowRunId!appId!entityType!entityId w. each
+     * segment separated by !. The sizes below indicate sizes of each one of
+     * these segments in sequence. clusterId, userName, flowName, entityType and
+     * entityId are strings. flowrunId is a long hence 8 bytes in size. app id
+     * is represented as 12 bytes with cluster timestamp part of appid being 8
+     * bytes (long) and seq id being 4 bytes(int). Strings are variable in size
+     * (i.e. end whenever separator is encountered). This is used while decoding
+     * and helps in determining where to split.
+     */
+    private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE,
+        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+        AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE,
+        Separator.VARIABLE_SIZE };
+
+    /*
+     * (non-Javadoc)
+     *
+     * Encodes EntityRowKey object into a byte array with each component/field
+     * in EntityRowKey separated by Separator#QUALIFIERS. This leads to an
+     * entity table row key of the form
+     * userName!clusterId!flowName!flowRunId!appId!entityType!entityId If
+     * entityType in passed EntityRowKey object is null (and the fields
+     * preceding it i.e. clusterId, userId and flowName, flowRunId and appId
+     * are not null), this returns a row key prefix of the form
+     * userName!clusterId!flowName!flowRunId!appId! and if entityId in
+     * EntityRowKey is null (other 6 components are not null), this returns a
+     * row key prefix of the form
+     * userName!clusterId!flowName!flowRunId!appId!entityType! flowRunId is
+     * inverted while encoding as it helps maintain a descending order for row
+     * keys in entity table.
+     *
+     * @see org.apache.hadoop.yarn.server.timelineservice.storage.common
+     * .KeyConverter#encode(java.lang.Object)
+     */
+    @Override
+    public byte[] encode(EntityRowKey rowKey) {
+      byte[] user =
+          Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
+              Separator.QUALIFIERS);
+      byte[] cluster =
+          Separator.encode(rowKey.getClusterId(), Separator.SPACE,
+              Separator.TAB, Separator.QUALIFIERS);
+      byte[] flow =
+          Separator.encode(rowKey.getFlowName(), Separator.SPACE,
+              Separator.TAB, Separator.QUALIFIERS);
+      byte[] first = Separator.QUALIFIERS.join(user, cluster, flow);
+      // Note that flowRunId is a long, so we can't encode them all at the same
+      // time.
+      byte[] second =
+          Bytes.toBytes(LongConverter.invertLong(rowKey.getFlowRunId()));
+      byte[] third = appIDKeyConverter.encode(rowKey.getAppId());
+      if (rowKey.getEntityType() == null) {
+        return Separator.QUALIFIERS.join(first, second, third,
+            Separator.EMPTY_BYTES);
+      }
+      byte[] entityType =
+          Separator.encode(rowKey.getEntityType(), Separator.SPACE,
+              Separator.TAB, Separator.QUALIFIERS);
+      byte[] entityId =
+          rowKey.getEntityId() == null ? Separator.EMPTY_BYTES : Separator
+              .encode(rowKey.getEntityId(), Separator.SPACE, Separator.TAB,
+                  Separator.QUALIFIERS);
+      byte[] fourth = Separator.QUALIFIERS.join(entityType, entityId);
+      return Separator.QUALIFIERS.join(first, second, third, fourth);
+    }
+
+    /*
+     * (non-Javadoc)
+     *
+     * Decodes an application row key of the form
+     * userName!clusterId!flowName!flowRunId!appId!entityType!entityId
+     * represented in byte format and converts it into an EntityRowKey object.
+     * flowRunId is inverted while decoding as it was inverted while encoding.
+     *
+     * @see
+     * org.apache.hadoop.yarn.server.timelineservice.storage.common
+     * .KeyConverter#decode(byte[])
+     */
+    @Override
+    public EntityRowKey decode(byte[] rowKey) {
+      byte[][] rowKeyComponents =
+          Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
+      if (rowKeyComponents.length != 7) {
+        throw new IllegalArgumentException("the row key is not valid for "
+            + "an entity");
+      }
+      String userId =
+          Separator.decode(Bytes.toString(rowKeyComponents[0]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      String clusterId =
+          Separator.decode(Bytes.toString(rowKeyComponents[1]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      String flowName =
+          Separator.decode(Bytes.toString(rowKeyComponents[2]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      Long flowRunId =
+          LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3]));
+      String appId = appIDKeyConverter.decode(rowKeyComponents[4]);
+      String entityType =
+          Separator.decode(Bytes.toString(rowKeyComponents[5]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      String entityId =
+          Separator.decode(Bytes.toString(rowKeyComponents[6]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      return new EntityRowKey(clusterId, userId, flowName, flowRunId, appId,
+          entityType, entityId);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyConverter.java
----------------------------------------------------------------------
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/EntityRowKeyConverter.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/EntityRowKeyConverter.java
deleted file mode 100644
index 43c0569..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyConverter.java
+++ /dev/null
@@ -1,143 +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.entity;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
-
-/**
- * Encodes and decodes row key for entity table.
- * The row key is of the form :
- * userName!clusterId!flowName!flowRunId!appId!entityType!entityId.
- * flowRunId is a long, appId is encoded/decoded using
- * {@link AppIdKeyConverter} and rest are strings.
- */
-public final class EntityRowKeyConverter implements KeyConverter<EntityRowKey> {
-  private static final EntityRowKeyConverter INSTANCE =
-      new EntityRowKeyConverter();
-
-  public static EntityRowKeyConverter getInstance() {
-    return INSTANCE;
-  }
-
-  private EntityRowKeyConverter() {
-  }
-
-  // Entity row key is of the form
-  // userName!clusterId!flowName!flowRunId!appId!entityType!entityId with each
-  // segment separated by !. The sizes below indicate sizes of each one of these
-  // segements in sequence. clusterId, userName, flowName, entityType and
-  // entityId are strings. flowrunId is a long hence 8 bytes in size. app id is
-  // represented as 12 bytes with cluster timestamp part of appid being 8 bytes
-  // (long) and seq id being 4 bytes(int).
-  // Strings are variable in size (i.e. end whenever separator is encountered).
-  // This is used while decoding and helps in determining where to split.
-  private static final int[] SEGMENT_SIZES = {
-      Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
-      Bytes.SIZEOF_LONG, AppIdKeyConverter.getKeySize(),
-      Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE };
-
-  /*
-   * (non-Javadoc)
-   *
-   * Encodes EntityRowKey object into a byte array with each component/field in
-   * EntityRowKey separated by Separator#QUALIFIERS. This leads to an entity
-   * table row key of the form
-   * userName!clusterId!flowName!flowRunId!appId!entityType!entityId
-   * If entityType in passed EntityRowKey object is null (and the fields
-   * preceding it i.e. clusterId, userId and flowName, flowRunId and appId are
-   * not null), this returns a row key prefix of the form
-   * userName!clusterId!flowName!flowRunId!appId! and if entityId in
-   * EntityRowKey is null (other 6 components are not null), this returns a row
-   * key prefix of the form
-   * userName!clusterId!flowName!flowRunId!appId!entityType!
-   * flowRunId is inverted while encoding as it helps maintain a descending
-   * order for row keys in entity table.
-   *
-   * @see
-   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
-   * #encode(java.lang.Object)
-   */
-  @Override
-  public byte[] encode(EntityRowKey rowKey) {
-    byte[] user = Separator.encode(rowKey.getUserId(),
-        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
-    byte[] cluster = Separator.encode(rowKey.getClusterId(),
-        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
-    byte[] flow = Separator.encode(rowKey.getFlowName(),
-        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
-    byte[] first = Separator.QUALIFIERS.join(user, cluster, flow);
-    // Note that flowRunId is a long, so we can't encode them all at the same
-    // time.
-    byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(
-        rowKey.getFlowRunId()));
-    byte[] third = AppIdKeyConverter.getInstance().encode(rowKey.getAppId());
-    if (rowKey.getEntityType() == null) {
-      return Separator.QUALIFIERS.join(
-          first, second, third, Separator.EMPTY_BYTES);
-    }
-    byte[] entityType = Separator.encode(rowKey.getEntityType(),
-        Separator.SPACE, Separator.TAB, Separator.QUALIFIERS);
-    byte[] entityId = rowKey.getEntityId() == null ? Separator.EMPTY_BYTES :
-        Separator.encode(rowKey.getEntityId(), Separator.SPACE, Separator.TAB,
-        Separator.QUALIFIERS);
-    byte[] fourth = Separator.QUALIFIERS.join(entityType, entityId);
-    return Separator.QUALIFIERS.join(first, second, third, fourth);
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * Decodes an application row key of the form
-   * userName!clusterId!flowName!flowRunId!appId!entityType!entityId represented
-   * in byte format and converts it into an EntityRowKey object. flowRunId is
-   * inverted while decoding as it was inverted while encoding.
-   *
-   * @see
-   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
-   * #decode(byte[])
-   */
-  @Override
-  public EntityRowKey decode(byte[] rowKey) {
-    byte[][] rowKeyComponents =
-        Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
-    if (rowKeyComponents.length != 7) {
-      throw new IllegalArgumentException("the row key is not valid for " +
-          "an entity");
-    }
-    String userId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[1]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    String flowName = Separator.decode(Bytes.toString(rowKeyComponents[2]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    Long flowRunId =
-        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
-    String appId = AppIdKeyConverter.getInstance().decode(rowKeyComponents[4]);
-    String entityType = Separator.decode(Bytes.toString(rowKeyComponents[5]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    String entityId =Separator.decode(Bytes.toString(rowKeyComponents[6]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    return new EntityRowKey(clusterId, userId, flowName, flowRunId, appId,
-        entityType, entityId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java
----------------------------------------------------------------------
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/EntityRowKeyPrefix.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/EntityRowKeyPrefix.java
new file mode 100644
index 0000000..9146180
--- /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/entity/EntityRowKeyPrefix.java
@@ -0,0 +1,74 @@
+/**
+ * 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.entity;
+
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix;
+
+/**
+ * Represents a partial rowkey without the entityId or without entityType and
+ * entityId for the entity table.
+ *
+ */
+public class EntityRowKeyPrefix extends EntityRowKey implements
+    RowKeyPrefix<EntityRowKey> {
+
+  /**
+   * Creates a prefix which generates the following rowKeyPrefixes for the
+   * entity table:
+   * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!}.
+   *
+   * @param clusterId identifying the cluster
+   * @param userId identifying the user
+   * @param flowName identifying the flow
+   * @param flowRunId identifying the individual run of this flow
+   * @param appId identifying the application
+   * @param entityType which entity type
+   */
+  public EntityRowKeyPrefix(String clusterId, String userId, String flowName,
+      Long flowRunId, String appId, String entityType) {
+    super(clusterId, userId, flowName, flowRunId, appId, entityType, null);
+  }
+
+  /**
+   * Creates a prefix which generates the following rowKeyPrefixes for the
+   * entity table:
+   * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}.
+   *
+   * @param clusterId identifying the cluster
+   * @param userId identifying the user
+   * @param flowName identifying the flow
+   * @param flowRunId identifying the individual run of this flow
+   * @param appId identifying the application
+   */
+  public EntityRowKeyPrefix(String clusterId, String userId, String flowName,
+      Long flowRunId, String appId) {
+    super(clusterId, userId, flowName, flowRunId, appId, null, null);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.application.
+   * RowKeyPrefix#getRowKeyPrefix()
+   */
+  public byte[] getRowKeyPrefix() {
+    return super.getRowKey();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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 eea38a5..d10608a 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
@@ -17,6 +17,10 @@
  */
 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.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
 
 /**
@@ -28,11 +32,37 @@ public class FlowActivityRowKey {
   private final Long dayTs;
   private final String userId;
   private final String flowName;
+  private final KeyConverter<FlowActivityRowKey> flowActivityRowKeyConverter =
+      new FlowActivityRowKeyConverter();
 
+  /**
+   * @param clusterId identifying the cluster
+   * @param dayTs to be converted to the top of the day timestamp
+   * @param userId identifying user
+   * @param flowName identifying the flow
+   */
   public FlowActivityRowKey(String clusterId, Long dayTs, String userId,
       String flowName) {
+    this(clusterId, dayTs, userId, flowName, true);
+  }
+
+  /**
+   * @param clusterId identifying the cluster
+   * @param timestamp when the flow activity happened. May be converted to the
+   *          top of the day depending on the convertDayTsToTopOfDay argument.
+   * @param userId identifying user
+   * @param flowName identifying the flow
+   * @param convertDayTsToTopOfDay if true and timestamp isn't null, then
+   *          timestamp will be converted to the top-of-the day timestamp
+   */
+  protected FlowActivityRowKey(String clusterId, Long timestamp, String userId,
+      String flowName, boolean convertDayTsToTopOfDay) {
     this.clusterId = clusterId;
-    this.dayTs = dayTs;
+    if (convertDayTsToTopOfDay && (timestamp != null)) {
+      this.dayTs = TimelineStorageUtils.getTopOfTheDayTimestamp(timestamp);
+    } else {
+      this.dayTs = timestamp;
+    }
     this.userId = userId;
     this.flowName = flowName;
   }
@@ -54,46 +84,13 @@ public class FlowActivityRowKey {
   }
 
   /**
-   * Constructs a row key prefix for the flow activity table as follows:
-   * {@code clusterId!}.
-   *
-   * @param clusterId Cluster Id.
-   * @return byte array with the row key prefix
-   */
-  public static byte[] getRowKeyPrefix(String clusterId) {
-    return FlowActivityRowKeyConverter.getInstance().encode(
-        new FlowActivityRowKey(clusterId, null, null, null));
-  }
-
-  /**
-   * Constructs a row key prefix for the flow activity table as follows:
-   * {@code clusterId!dayTimestamp!}.
-   *
-   * @param clusterId Cluster Id.
-   * @param dayTs Start of the day timestamp.
-   * @return byte array with the row key prefix
-   */
-  public static byte[] getRowKeyPrefix(String clusterId, long dayTs) {
-    return FlowActivityRowKeyConverter.getInstance().encode(
-        new FlowActivityRowKey(clusterId, dayTs, null, null));
-  }
-
-  /**
    * Constructs a row key for the flow activity table as follows:
    * {@code clusterId!dayTimestamp!user!flowName}.
    *
-   * @param clusterId Cluster Id.
-   * @param eventTs event's TimeStamp.
-   * @param userId User Id.
-   * @param flowName Flow Name.
    * @return byte array for the row key
    */
-  public static byte[] getRowKey(String clusterId, long eventTs, String userId,
-      String flowName) {
-    // convert it to Day's time stamp
-    eventTs = TimelineStorageUtils.getTopOfTheDayTimestamp(eventTs);
-    return FlowActivityRowKeyConverter.getInstance().encode(
-        new FlowActivityRowKey(clusterId, eventTs, userId, flowName));
+  public byte[] getRowKey() {
+    return flowActivityRowKeyConverter.encode(this);
   }
 
   /**
@@ -103,6 +100,97 @@ public class FlowActivityRowKey {
    * @return A <cite>FlowActivityRowKey</cite> object.
    */
   public static FlowActivityRowKey parseRowKey(byte[] rowKey) {
-    return FlowActivityRowKeyConverter.getInstance().decode(rowKey);
+    return new FlowActivityRowKeyConverter().decode(rowKey);
+  }
+
+  /**
+   * Encodes and decodes row key for flow activity table. The row key is of the
+   * form : clusterId!dayTimestamp!user!flowName. dayTimestamp(top of the day
+   * timestamp) is a long and rest are strings.
+   * <p>
+   */
+  final private static class FlowActivityRowKeyConverter implements
+      KeyConverter<FlowActivityRowKey> {
+
+    private FlowActivityRowKeyConverter() {
+    }
+
+    /**
+     * The flow activity row key is of the form
+     * clusterId!dayTimestamp!user!flowName with each segment separated by !.
+     * The sizes below indicate sizes of each one of these segements in
+     * sequence. clusterId, user and flowName are strings. Top of the day
+     * timestamp is a long hence 8 bytes in size. Strings are variable in size
+     * (i.e. they end whenever separator is encountered). This is used while
+     * decoding and helps in determining where to split.
+     */
+    private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE,
+        Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE };
+
+    /*
+     * (non-Javadoc)
+     *
+     * Encodes FlowActivityRowKey object into a byte array with each
+     * component/field in FlowActivityRowKey separated by Separator#QUALIFIERS.
+     * This leads to an flow activity table row key of the form
+     * clusterId!dayTimestamp!user!flowName. If dayTimestamp in passed
+     * FlowActivityRowKey object is null and clusterId is not null, then this
+     * returns a row key prefix as clusterId! and if userId in
+     * FlowActivityRowKey is null (and the fields preceding it i.e. clusterId
+     * and dayTimestamp are not null), this returns a row key prefix as
+     * clusterId!dayTimeStamp! dayTimestamp is inverted while encoding as it
+     * helps maintain a descending order for row keys in flow activity table.
+     *
+     * @see org.apache.hadoop.yarn.server.timelineservice.storage.common
+     * .KeyConverter#encode(java.lang.Object)
+     */
+    @Override
+    public byte[] encode(FlowActivityRowKey rowKey) {
+      if (rowKey.getDayTimestamp() == null) {
+        return Separator.QUALIFIERS.join(Separator.encode(
+            rowKey.getClusterId(), Separator.SPACE, Separator.TAB,
+            Separator.QUALIFIERS), Separator.EMPTY_BYTES);
+      }
+      if (rowKey.getUserId() == null) {
+        return Separator.QUALIFIERS.join(Separator.encode(
+            rowKey.getClusterId(), Separator.SPACE, Separator.TAB,
+            Separator.QUALIFIERS), Bytes.toBytes(LongConverter
+            .invertLong(rowKey.getDayTimestamp())), Separator.EMPTY_BYTES);
+      }
+      return Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(),
+          Separator.SPACE, Separator.TAB, Separator.QUALIFIERS), Bytes
+          .toBytes(LongConverter.invertLong(rowKey.getDayTimestamp())),
+          Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
+              Separator.QUALIFIERS), Separator.encode(rowKey.getFlowName(),
+              Separator.SPACE, Separator.TAB, Separator.QUALIFIERS));
+    }
+
+    /*
+     * (non-Javadoc)
+     *
+     * @see
+     * org.apache.hadoop.yarn.server.timelineservice.storage.common
+     * .KeyConverter#decode(byte[])
+     */
+    @Override
+    public FlowActivityRowKey decode(byte[] rowKey) {
+      byte[][] rowKeyComponents =
+          Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
+      if (rowKeyComponents.length != 4) {
+        throw new IllegalArgumentException("the row key is not valid for "
+            + "a flow activity");
+      }
+      String clusterId =
+          Separator.decode(Bytes.toString(rowKeyComponents[0]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      Long dayTs = LongConverter.invertLong(Bytes.toLong(rowKeyComponents[1]));
+      String userId =
+          Separator.decode(Bytes.toString(rowKeyComponents[2]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      String flowName =
+          Separator.decode(Bytes.toString(rowKeyComponents[3]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      return new FlowActivityRowKey(clusterId, dayTs, userId, flowName);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyConverter.java
----------------------------------------------------------------------
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/FlowActivityRowKeyConverter.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/FlowActivityRowKeyConverter.java
deleted file mode 100644
index 9dc4c98..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyConverter.java
+++ /dev/null
@@ -1,115 +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.flow;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
-
-/**
- * Encodes and decodes row key for flow activity table.
- * The row key is of the form : clusterId!dayTimestamp!user!flowName.
- * dayTimestamp(top of the day timestamp) is a long and rest are strings.
- */
-public final class FlowActivityRowKeyConverter implements
-    KeyConverter<FlowActivityRowKey> {
-  private static final FlowActivityRowKeyConverter INSTANCE =
-      new FlowActivityRowKeyConverter();
-
-  public static FlowActivityRowKeyConverter getInstance() {
-    return INSTANCE;
-  }
-
-  private FlowActivityRowKeyConverter() {
-  }
-
-  // Flow activity row key is of the form clusterId!dayTimestamp!user!flowName
-  // with each segment separated by !. The sizes below indicate sizes of each
-  // one of these segements in sequence. clusterId, user and flowName are
-  // strings. Top of the day timestamp is a long hence 8 bytes in size.
-  // Strings are variable in size (i.e. end whenever separator is encountered).
-  // This is used while decoding and helps in determining where to split.
-  private static final int[] SEGMENT_SIZES = {
-      Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE,
-      Separator.VARIABLE_SIZE };
-
-  /*
-   * (non-Javadoc)
-   *
-   * Encodes FlowActivityRowKey object into a byte array with each
-   * component/field in FlowActivityRowKey separated by Separator#QUALIFIERS.
-   * This leads to an flow activity table row key of the form
-   * clusterId!dayTimestamp!user!flowName
-   * If dayTimestamp in passed FlowActivityRowKey object is null and clusterId
-   * is not null, this returns a row key prefix as clusterId! and if userId in
-   * FlowActivityRowKey is null (and the fields preceding it i.e. clusterId and
-   * dayTimestamp are not null), this returns a row key prefix as
-   * clusterId!dayTimeStamp!
-   * dayTimestamp is inverted while encoding as it helps maintain a descending
-   * order for row keys in flow activity table.
-   *
-   * @see
-   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
-   * #encode(java.lang.Object)
-   */
-
-  @Override
-  public byte[] encode(FlowActivityRowKey rowKey) {
-    if (rowKey.getDayTimestamp() == null) {
-      return Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(),
-          Separator.SPACE, Separator.TAB, Separator.QUALIFIERS),
-              Separator.EMPTY_BYTES);
-    }
-    if (rowKey.getUserId() == null) {
-      return Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(),
-          Separator.SPACE, Separator.TAB, Separator.QUALIFIERS),
-          Bytes.toBytes(TimelineStorageUtils.invertLong(
-              rowKey.getDayTimestamp())), Separator.EMPTY_BYTES);
-    }
-    return Separator.QUALIFIERS.join(
-        Separator.encode(rowKey.getClusterId(), Separator.SPACE, Separator.TAB,
-            Separator.QUALIFIERS),
-        Bytes.toBytes(
-            TimelineStorageUtils.invertLong(rowKey.getDayTimestamp())),
-        Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
-            Separator.QUALIFIERS),
-        Separator.encode(rowKey.getFlowName(), Separator.SPACE, Separator.TAB,
-            Separator.QUALIFIERS));
-  }
-
-  @Override
-  public FlowActivityRowKey decode(byte[] rowKey) {
-    byte[][] rowKeyComponents =
-        Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
-    if (rowKeyComponents.length != 4) {
-      throw new IllegalArgumentException("the row key is not valid for "
-          + "a flow activity");
-    }
-    String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    Long dayTs =
-        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[1]));
-    String userId = Separator.decode(Bytes.toString(rowKeyComponents[2]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    String flowName = Separator.decode(Bytes.toString(rowKeyComponents[3]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    return new FlowActivityRowKey(clusterId, dayTs, userId, flowName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowActivityRowKeyPrefix.java
----------------------------------------------------------------------
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/FlowActivityRowKeyPrefix.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/FlowActivityRowKeyPrefix.java
new file mode 100644
index 0000000..eb88e54
--- /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/FlowActivityRowKeyPrefix.java
@@ -0,0 +1,60 @@
+/**
+ * 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.yarn.server.timelineservice.storage.common.RowKeyPrefix;
+
+/**
+ * A prefix partial rowkey for flow activities.
+ */
+public class FlowActivityRowKeyPrefix extends FlowActivityRowKey implements
+    RowKeyPrefix<FlowActivityRowKey> {
+
+  /**
+   * Constructs a row key prefix for the flow activity table as follows:
+   * {@code clusterId!dayTimestamp!}.
+   *
+   * @param clusterId Cluster Id.
+   * @param dayTs Start of the day timestamp.
+   */
+  public FlowActivityRowKeyPrefix(String clusterId, Long dayTs) {
+    super(clusterId, dayTs, null, null, false);
+  }
+
+  /**
+   * Constructs a row key prefix for the flow activity table as follows:
+   * {@code clusterId!}.
+   *
+   * @param clusterId identifying the cluster
+   */
+  public FlowActivityRowKeyPrefix(String clusterId) {
+    super(clusterId, null, null, null, false);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.application.
+   * RowKeyPrefix#getRowKeyPrefix()
+   */
+  public byte[] getRowKeyPrefix() {
+    return super.getRowKey();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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 f1553b8..2e7a9d8 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,10 +25,10 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.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;
 
 /**
@@ -41,14 +41,14 @@ public enum FlowRunColumn implements Column<FlowRunTable> {
    * application start times.
    */
   MIN_START_TIME(FlowRunColumnFamily.INFO, "min_start_time",
-      AggregationOperation.GLOBAL_MIN, LongConverter.getInstance()),
+      AggregationOperation.GLOBAL_MIN, new LongConverter()),
 
   /**
    * When the flow ended. This is the maximum of currently known application end
    * times.
    */
   MAX_END_TIME(FlowRunColumnFamily.INFO, "max_end_time",
-      AggregationOperation.GLOBAL_MAX, LongConverter.getInstance()),
+      AggregationOperation.GLOBAL_MAX, new LongConverter()),
 
   /**
    * The version of the flow that this flow belongs to.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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 0f14c89..e74282a 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
@@ -41,7 +41,7 @@ public enum FlowRunColumnPrefix implements ColumnPrefix<FlowRunTable> {
   /**
    * To store flow run info values.
    */
-  METRIC(FlowRunColumnFamily.INFO, "m", null, LongConverter.getInstance());
+  METRIC(FlowRunColumnFamily.INFO, "m", null, new LongConverter());
 
   private final ColumnHelper<FlowRunTable> column;
   private final ColumnFamily<FlowRunTable> columnFamily;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/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 925242b..8fda9a8 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
@@ -17,6 +17,11 @@
  */
 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.KeyConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
+
 /**
  * Represents a rowkey for the flow run table.
  */
@@ -25,6 +30,8 @@ public class FlowRunRowKey {
   private final String userId;
   private final String flowName;
   private final Long flowRunId;
+  private final FlowRunRowKeyConverter flowRunRowKeyConverter =
+      new FlowRunRowKeyConverter();
 
   public FlowRunRowKey(String clusterId, String userId, String flowName,
       Long flowRunId) {
@@ -51,36 +58,16 @@ public class FlowRunRowKey {
   }
 
   /**
-   * Constructs a row key prefix for the flow run table as follows: {
-   * clusterId!userI!flowName!}.
-   *
-   * @param clusterId Cluster Id.
-   * @param userId User Id.
-   * @param flowName Flow Name.
-   * @return byte array with the row key prefix
-   */
-  public static byte[] getRowKeyPrefix(String clusterId, String userId,
-      String flowName) {
-    return FlowRunRowKeyConverter.getInstance().encode(new FlowRunRowKey(
-        clusterId, userId, flowName, null));
-  }
-
-  /**
    * Constructs a row key for the entity table as follows: {
    * clusterId!userId!flowName!Inverted Flow Run Id}.
    *
-   * @param clusterId Cluster Id.
-   * @param userId User Id.
-   * @param flowName Flow Name.
-   * @param flowRunId Run Id for the flow name.
    * @return byte array with the row key
    */
-  public static byte[] getRowKey(String clusterId, String userId,
-      String flowName, Long flowRunId) {
-    return FlowRunRowKeyConverter.getInstance().encode(new FlowRunRowKey(
-        clusterId, userId, flowName, flowRunId));
+  public byte[] getRowKey() {
+    return flowRunRowKeyConverter.encode(this);
   }
 
+
   /**
    * Given the raw row key as bytes, returns the row key as an object.
    *
@@ -88,7 +75,7 @@ public class FlowRunRowKey {
    * @return A <cite>FlowRunRowKey</cite> object.
    */
   public static FlowRunRowKey parseRowKey(byte[] rowKey) {
-    return FlowRunRowKeyConverter.getInstance().decode(rowKey);
+    return new FlowRunRowKeyConverter().decode(rowKey);
   }
 
   /**
@@ -106,4 +93,98 @@ public class FlowRunRowKey {
     flowKeyStr.append("}");
     return flowKeyStr.toString();
   }
+
+  /**
+   * Encodes and decodes row key for flow run table.
+   * The row key is of the form : clusterId!userId!flowName!flowrunId.
+   * flowrunId is a long and rest are strings.
+   * <p>
+   */
+  final private static class FlowRunRowKeyConverter implements
+      KeyConverter<FlowRunRowKey> {
+
+    private FlowRunRowKeyConverter() {
+    }
+
+    /**
+     * The flow run row key is of the form clusterId!userId!flowName!flowrunId
+     * with each segment separated by !. The sizes below indicate sizes of each
+     * one of these segments in sequence. clusterId, userId and flowName are
+     * strings. flowrunId is a long hence 8 bytes in size. Strings are variable
+     * in size (i.e. end whenever separator is encountered). This is used while
+     * decoding and helps in determining where to split.
+     */
+    private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE,
+        Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG };
+
+    /*
+     * (non-Javadoc)
+     *
+     * Encodes FlowRunRowKey object into a byte array with each component/field
+     * in FlowRunRowKey separated by Separator#QUALIFIERS. This leads to an flow
+     * run row key of the form clusterId!userId!flowName!flowrunId If flowRunId
+     * in passed FlowRunRowKey object is null (and the fields preceding it i.e.
+     * clusterId, userId and flowName are not null), this returns a row key
+     * prefix of the form clusterId!userName!flowName! flowRunId is inverted
+     * while encoding as it helps maintain a descending order for flow keys in
+     * flow run table.
+     *
+     * @see
+     * org.apache.hadoop.yarn.server.timelineservice.storage.common
+     * .KeyConverter#encode(java.lang.Object)
+     */
+    @Override
+    public byte[] encode(FlowRunRowKey rowKey) {
+      byte[] first =
+          Separator.QUALIFIERS.join(Separator.encode(rowKey.getClusterId(),
+              Separator.SPACE, Separator.TAB, Separator.QUALIFIERS), Separator
+              .encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
+                  Separator.QUALIFIERS), Separator.encode(rowKey.getFlowName(),
+              Separator.SPACE, Separator.TAB, Separator.QUALIFIERS));
+      if (rowKey.getFlowRunId() == null) {
+        return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES);
+      } else {
+        // Note that flowRunId is a long, so we can't encode them all at the
+        // same
+        // time.
+        byte[] second =
+            Bytes.toBytes(LongConverter.invertLong(rowKey.getFlowRunId()));
+        return Separator.QUALIFIERS.join(first, second);
+      }
+    }
+
+    /*
+     * (non-Javadoc)
+     *
+     * Decodes an flow run row key of the form
+     * clusterId!userId!flowName!flowrunId represented in byte format and
+     * converts it into an FlowRunRowKey object. flowRunId is inverted while
+     * decoding as it was inverted while encoding.
+     *
+     * @see
+     * org.apache.hadoop.yarn.server.timelineservice.storage.common
+     * .KeyConverter#decode(byte[])
+     */
+    @Override
+    public FlowRunRowKey decode(byte[] rowKey) {
+      byte[][] rowKeyComponents =
+          Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
+      if (rowKeyComponents.length != 4) {
+        throw new IllegalArgumentException("the row key is not valid for "
+            + "a flow run");
+      }
+      String clusterId =
+          Separator.decode(Bytes.toString(rowKeyComponents[0]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      String userId =
+          Separator.decode(Bytes.toString(rowKeyComponents[1]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      String flowName =
+          Separator.decode(Bytes.toString(rowKeyComponents[2]),
+              Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
+      Long flowRunId =
+          LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3]));
+      return new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyConverter.java
----------------------------------------------------------------------
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/FlowRunRowKeyConverter.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/FlowRunRowKeyConverter.java
deleted file mode 100644
index 642f065..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyConverter.java
+++ /dev/null
@@ -1,120 +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.flow;
-
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
-import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineStorageUtils;
-
-/**
- * Encodes and decodes row key for flow run table.
- * The row key is of the form : clusterId!userId!flowName!flowrunId.
- * flowrunId is a long and rest are strings.
- */
-public final class FlowRunRowKeyConverter implements
-    KeyConverter<FlowRunRowKey> {
-  private static final FlowRunRowKeyConverter INSTANCE =
-      new FlowRunRowKeyConverter();
-
-  public static FlowRunRowKeyConverter getInstance() {
-    return INSTANCE;
-  }
-
-  private FlowRunRowKeyConverter() {
-  }
-
-  // Flow run row key is of the form
-  // clusterId!userId!flowName!flowrunId with each segment separated by !.
-  // The sizes below indicate sizes of each one of these segments in sequence.
-  // clusterId, userId and flowName are strings. flowrunId is a long hence 8
-  // bytes in size. Strings are variable in size (i.e. end whenever separator is
-  // encountered). This is used while decoding and helps in determining where to
-  // split.
-  private static final int[] SEGMENT_SIZES = {
-      Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
-      Bytes.SIZEOF_LONG };
-
-  /*
-   * (non-Javadoc)
-   *
-   * Encodes FlowRunRowKey object into a byte array with each component/field in
-   * FlowRunRowKey separated by Separator#QUALIFIERS. This leads to an
-   * flow run row key of the form clusterId!userId!flowName!flowrunId
-   * If flowRunId in passed FlowRunRowKey object is null (and the fields
-   * preceding it i.e. clusterId, userId and flowName are not null), this
-   * returns a row key prefix of the form clusterId!userName!flowName!
-   * flowRunId is inverted while encoding as it helps maintain a descending
-   * order for flow keys in flow run table.
-   *
-   * @see
-   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
-   * #encode(java.lang.Object)
-   */
-  @Override
-  public byte[] encode(FlowRunRowKey rowKey) {
-    byte[] first = Separator.QUALIFIERS.join(
-        Separator.encode(rowKey.getClusterId(), Separator.SPACE, Separator.TAB,
-            Separator.QUALIFIERS),
-        Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB,
-            Separator.QUALIFIERS),
-        Separator.encode(rowKey.getFlowName(), Separator.SPACE, Separator.TAB,
-            Separator.QUALIFIERS));
-    if (rowKey.getFlowRunId() == null) {
-      return Separator.QUALIFIERS.join(first, Separator.EMPTY_BYTES);
-    } else {
-      // Note that flowRunId is a long, so we can't encode them all at the same
-      // time.
-      byte[] second = Bytes.toBytes(TimelineStorageUtils.invertLong(
-          rowKey.getFlowRunId()));
-      return Separator.QUALIFIERS.join(first, second);
-    }
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * Decodes an flow run row key of the form
-   * clusterId!userId!flowName!flowrunId represented in byte format and converts
-   * it into an FlowRunRowKey object. flowRunId is inverted while decoding as
-   * it was inverted while encoding.
-   *
-   * @see
-   * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter
-   * #decode(byte[])
-   */
-  @Override
-  public FlowRunRowKey decode(byte[] rowKey) {
-    byte[][] rowKeyComponents =
-        Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES);
-    if (rowKeyComponents.length != 4) {
-      throw new IllegalArgumentException("the row key is not valid for " +
-          "a flow run");
-    }
-    String clusterId = Separator.decode(Bytes.toString(rowKeyComponents[0]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    String userId = Separator.decode(Bytes.toString(rowKeyComponents[1]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    String flowName = Separator.decode(Bytes.toString(rowKeyComponents[2]),
-        Separator.QUALIFIERS, Separator.TAB, Separator.SPACE);
-    Long flowRunId =
-        TimelineStorageUtils.invertLong(Bytes.toLong(rowKeyComponents[3]));
-    return new FlowRunRowKey(clusterId, userId, flowName, flowRunId);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/FlowRunRowKeyPrefix.java
----------------------------------------------------------------------
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/FlowRunRowKeyPrefix.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/FlowRunRowKeyPrefix.java
new file mode 100644
index 0000000..23ebc66
--- /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/FlowRunRowKeyPrefix.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.yarn.server.timelineservice.storage.common.RowKeyPrefix;
+
+/**
+ * Represents a partial rowkey (without the flowRunId) for the flow run table.
+ */
+public class FlowRunRowKeyPrefix extends FlowRunRowKey implements
+    RowKeyPrefix<FlowRunRowKey> {
+
+  /**
+   * Constructs a row key prefix for the flow run table as follows:
+   * {@code clusterId!userI!flowName!}.
+   *
+   * @param clusterId identifying the cluster
+   * @param userId identifying the user
+   * @param flowName identifying the flow
+   */
+  public FlowRunRowKeyPrefix(String clusterId, String userId,
+      String flowName) {
+    super(clusterId, userId, flowName, null);
+  }
+
+  /*
+   * (non-Javadoc)
+   *
+   * @see
+   * org.apache.hadoop.yarn.server.timelineservice.storage.application.
+   * RowKeyPrefix#getRowKeyPrefix()
+   */
+  public byte[] getRowKeyPrefix() {
+    // We know we're a FlowRunRowKey with null florRunId, so we can simply
+    // delegate
+    return super.getRowKey();
+  }
+
+}


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


[44/50] [abbrv] hadoop git commit: YARN-5170. Eliminate singleton converters and static method access. (Joep Rottinghuis via Varun Saxena)

Posted by vr...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/695e7edc/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestRowKeys.java
new file mode 100644
index 0000000..77370b7
--- /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/TestRowKeys.java
@@ -0,0 +1,244 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.timelineservice.storage.common;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationRowKeyPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.entity.EntityRowKeyPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKey;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowActivityRowKeyPrefix;
+import org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunRowKey;
+import org.junit.Test;
+
+
+public class TestRowKeys {
+
+  private final static String QUALIFIER_SEP = Separator.QUALIFIERS.getValue();
+  private final static byte[] QUALIFIER_SEP_BYTES = Bytes
+      .toBytes(QUALIFIER_SEP);
+  private final static String CLUSTER = "cl" + QUALIFIER_SEP + "uster";
+  private final static String USER = QUALIFIER_SEP + "user";
+  private final static String FLOW_NAME = "dummy_" + QUALIFIER_SEP + "flow"
+      + QUALIFIER_SEP;
+  private final static Long FLOW_RUN_ID;
+  private final static String APPLICATION_ID;
+  static {
+    long runid = Long.MAX_VALUE - 900L;
+    byte[] longMaxByteArr = Bytes.toBytes(Long.MAX_VALUE);
+    byte[] byteArr = Bytes.toBytes(runid);
+    int sepByteLen = QUALIFIER_SEP_BYTES.length;
+    if (sepByteLen <= byteArr.length) {
+      for (int i = 0; i < sepByteLen; i++) {
+        byteArr[i] = (byte) (longMaxByteArr[i] - QUALIFIER_SEP_BYTES[i]);
+      }
+    }
+    FLOW_RUN_ID = Bytes.toLong(byteArr);
+    long clusterTs = System.currentTimeMillis();
+    byteArr = Bytes.toBytes(clusterTs);
+    if (sepByteLen <= byteArr.length) {
+      for (int i = 0; i < sepByteLen; i++) {
+        byteArr[byteArr.length - sepByteLen + i] =
+            (byte) (longMaxByteArr[byteArr.length - sepByteLen + i] - QUALIFIER_SEP_BYTES[i]);
+      }
+    }
+    clusterTs = Bytes.toLong(byteArr);
+    int seqId = 222;
+    APPLICATION_ID = ApplicationId.newInstance(clusterTs, seqId).toString();
+  }
+
+  private static void verifyRowPrefixBytes(byte[] byteRowKeyPrefix) {
+    int sepLen = QUALIFIER_SEP_BYTES.length;
+    for (int i = 0; i < sepLen; i++) {
+      assertTrue(
+          "Row key prefix not encoded properly.",
+          byteRowKeyPrefix[byteRowKeyPrefix.length - sepLen + i] == QUALIFIER_SEP_BYTES[i]);
+    }
+  }
+
+  @Test
+  public void testApplicationRowKey() {
+    byte[] byteRowKey =
+        new ApplicationRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID,
+            APPLICATION_ID).getRowKey();
+    ApplicationRowKey rowKey = ApplicationRowKey.parseRowKey(byteRowKey);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(USER, rowKey.getUserId());
+    assertEquals(FLOW_NAME, rowKey.getFlowName());
+    assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
+    assertEquals(APPLICATION_ID, rowKey.getAppId());
+
+    byte[] byteRowKeyPrefix =
+        new ApplicationRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID)
+            .getRowKeyPrefix();
+    byte[][] splits =
+        Separator.QUALIFIERS.split(byteRowKeyPrefix,
+            new int[] { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+                Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+                Separator.VARIABLE_SIZE });
+    assertEquals(5, splits.length);
+    assertEquals(0, splits[4].length);
+    assertEquals(FLOW_NAME,
+        Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
+    assertEquals(FLOW_RUN_ID,
+        (Long) LongConverter.invertLong(Bytes.toLong(splits[3])));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+
+    byteRowKeyPrefix =
+        new ApplicationRowKeyPrefix(CLUSTER, USER, FLOW_NAME).getRowKeyPrefix();
+    splits =
+        Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
+            Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+            Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
+    assertEquals(4, splits.length);
+    assertEquals(0, splits[3].length);
+    assertEquals(FLOW_NAME,
+        Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+  }
+
+  /**
+   * Tests the converters indirectly through the public methods of the
+   * corresponding rowkey.
+   */
+  @Test
+  public void testAppToFlowRowKey() {
+    byte[] byteRowKey = new AppToFlowRowKey(CLUSTER,
+        APPLICATION_ID).getRowKey();
+    AppToFlowRowKey rowKey = AppToFlowRowKey.parseRowKey(byteRowKey);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(APPLICATION_ID, rowKey.getAppId());
+  }
+
+  @Test
+  public void testEntityRowKey() {
+    String entityId = "!ent!ity!!id!";
+    String entityType = "entity!Type";
+    byte[] byteRowKey =
+        new EntityRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID, APPLICATION_ID,
+            entityType, entityId).getRowKey();
+    EntityRowKey rowKey = EntityRowKey.parseRowKey(byteRowKey);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(USER, rowKey.getUserId());
+    assertEquals(FLOW_NAME, rowKey.getFlowName());
+    assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
+    assertEquals(APPLICATION_ID, rowKey.getAppId());
+    assertEquals(entityType, rowKey.getEntityType());
+    assertEquals(entityId, rowKey.getEntityId());
+
+    byte[] byteRowKeyPrefix =
+        new EntityRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID,
+            APPLICATION_ID, entityType).getRowKeyPrefix();
+    byte[][] splits =
+        Separator.QUALIFIERS.split(
+            byteRowKeyPrefix,
+            new int[] { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+                Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+                AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE,
+                Separator.VARIABLE_SIZE });
+    assertEquals(7, splits.length);
+    assertEquals(0, splits[6].length);
+    assertEquals(APPLICATION_ID, new AppIdKeyConverter().decode(splits[4]));
+    assertEquals(entityType,
+        Separator.QUALIFIERS.decode(Bytes.toString(splits[5])));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+
+    byteRowKeyPrefix =
+        new EntityRowKeyPrefix(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID,
+            APPLICATION_ID).getRowKeyPrefix();
+    splits =
+        Separator.QUALIFIERS.split(
+            byteRowKeyPrefix,
+            new int[] { Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+                Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+                AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE });
+    assertEquals(6, splits.length);
+    assertEquals(0, splits[5].length);
+    AppIdKeyConverter appIdKeyConverter = new AppIdKeyConverter();
+    assertEquals(APPLICATION_ID, appIdKeyConverter.decode(splits[4]));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+  }
+
+  @Test
+  public void testFlowActivityRowKey() {
+    Long ts = 1459900830000L;
+    Long dayTimestamp = TimelineStorageUtils.getTopOfTheDayTimestamp(ts);
+    byte[] byteRowKey =
+        new FlowActivityRowKey(CLUSTER, ts, USER, FLOW_NAME).getRowKey();
+    FlowActivityRowKey rowKey = FlowActivityRowKey.parseRowKey(byteRowKey);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(dayTimestamp, rowKey.getDayTimestamp());
+    assertEquals(USER, rowKey.getUserId());
+    assertEquals(FLOW_NAME, rowKey.getFlowName());
+
+    byte[] byteRowKeyPrefix =
+        new FlowActivityRowKeyPrefix(CLUSTER).getRowKeyPrefix();
+    byte[][] splits =
+        Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
+            Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
+    assertEquals(2, splits.length);
+    assertEquals(0, splits[1].length);
+    assertEquals(CLUSTER,
+        Separator.QUALIFIERS.decode(Bytes.toString(splits[0])));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+
+    byteRowKeyPrefix =
+        new FlowActivityRowKeyPrefix(CLUSTER, ts).getRowKeyPrefix();
+    splits =
+        Separator.QUALIFIERS.split(byteRowKeyPrefix,
+            new int[] { Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG,
+                Separator.VARIABLE_SIZE });
+    assertEquals(3, splits.length);
+    assertEquals(0, splits[2].length);
+    assertEquals(CLUSTER,
+        Separator.QUALIFIERS.decode(Bytes.toString(splits[0])));
+    assertEquals(ts,
+        (Long) LongConverter.invertLong(Bytes.toLong(splits[1])));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+  }
+
+  @Test
+  public void testFlowRunRowKey() {
+    byte[] byteRowKey =
+        new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, FLOW_RUN_ID).getRowKey();
+    FlowRunRowKey rowKey = FlowRunRowKey.parseRowKey(byteRowKey);
+    assertEquals(CLUSTER, rowKey.getClusterId());
+    assertEquals(USER, rowKey.getUserId());
+    assertEquals(FLOW_NAME, rowKey.getFlowName());
+    assertEquals(FLOW_RUN_ID, rowKey.getFlowRunId());
+
+    byte[] byteRowKeyPrefix =
+        new FlowRunRowKey(CLUSTER, USER, FLOW_NAME, null).getRowKey();
+    byte[][] splits =
+        Separator.QUALIFIERS.split(byteRowKeyPrefix, new int[] {
+            Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE,
+            Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE });
+    assertEquals(4, splits.length);
+    assertEquals(0, splits[3].length);
+    assertEquals(FLOW_NAME,
+        Separator.QUALIFIERS.decode(Bytes.toString(splits[2])));
+    verifyRowPrefixBytes(byteRowKeyPrefix);
+  }
+
+}


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