Browse Source

YARN-3863. Support complex filters in TimelineReader (Varun Saxena via sjlee)

Sangjin Lee 9 years ago
parent
commit
c2efdc415a
41 changed files with 3761 additions and 790 deletions
  1. 112 58
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineEntityFilters.java
  2. 50 38
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/TimelineReaderWebServicesUtils.java
  3. 28 7
      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
  4. 62 0
      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
  5. 14 2
      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
  6. 14 0
      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
  7. 199 7
      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
  8. 48 0
      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
  9. 71 0
      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
  10. 6 0
      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
  11. 18 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java
  12. 17 14
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java
  13. 26 2
      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
  14. 34 3
      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
  15. 16 0
      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
  16. 17 0
      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
  17. 16 0
      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
  18. 35 0
      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
  19. 71 0
      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
  20. 409 52
      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
  21. 26 2
      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
  22. 34 4
      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
  23. 35 0
      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
  24. 3 0
      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. 26 0
      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. 1 0
      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
  27. 272 154
      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
  28. 7 0
      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
  29. 74 23
      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
  30. 361 262
      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
  31. 68 3
      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
  32. 4 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServices.java
  33. 300 32
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestFileSystemTimelineReaderImpl.java
  34. 789 64
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java
  35. 16 0
      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
  36. 229 38
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/flow/TestHBaseStorageFlowRun.java
  37. 1 1
      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
  38. 62 0
      hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineExistsFilter.java
  39. 48 0
      hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValueFilter.java
  40. 71 0
      hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/reader/filter/TimelineKeyValuesFilter.java
  41. 71 0
      hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimelineEntityFiltersType.java

+ 112 - 58
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;
   }
 }

+ 50 - 38
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.

+ 28 - 7
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);
+  }
 }

+ 62 - 0
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);
+  }
+}

+ 14 - 2
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();

+ 14 - 0
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());
+  }
 }

+ 199 - 7
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(
@@ -89,6 +105,159 @@ public final class TimelineFilterUtils {
             colPrefix.getColumnPrefixBytes(filter.getPrefix())));
   }
 
+  /**
+   * 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})
@@ -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;
       }
     }

+ 48 - 0
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;
+  }
+}

+ 71 - 0
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());
+  }
+}

+ 6 - 0
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);
+  }
 }

+ 18 - 18
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(

+ 17 - 14
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());
                 }

+ 26 - 2
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

+ 34 - 3
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)
    *

+ 16 - 0
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);
   }

+ 17 - 0
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();
 }

+ 16 - 0
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

@@ -362,6 +362,22 @@ public class ColumnHelper<T> {
     return columnQualifier;
   }
 
+  /**
+   * 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}.

+ 35 - 0
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

@@ -111,6 +111,18 @@ public interface ColumnPrefix<T> {
   <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;
+
   /**
    * @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);
 }

+ 71 - 0
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);
+}

+ 409 - 52
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];
 
@@ -311,6 +327,21 @@ public final class TimelineStorageUtils {
     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.
    *
@@ -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);
+  }
 }

+ 26 - 2
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)}

+ 34 - 4
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;
   }
-
 }

+ 35 - 0
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;
+  }
 }

+ 3 - 0
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();
   }

+ 26 - 0
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;
+  }
 }

+ 1 - 0
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) {

+ 272 - 154
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;
   }

+ 7 - 0
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

+ 74 - 23
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);
     }
 

+ 361 - 262
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);
-  }
 }

+ 68 - 3
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;
@@ -125,6 +174,16 @@ public abstract class TimelineEntityReader {
     return filters;
   }
 
+  /**
+   * 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.
    *
@@ -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) {

+ 4 - 6
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();

+ 300 - 32
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),

File diff suppressed because it is too large
+ 789 - 64
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorage.java


+ 16 - 0
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;
   }
 

+ 229 - 38
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();

+ 1 - 1
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);
   }
 
 

+ 62 - 0
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);
+  }
+}

+ 48 - 0
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;
+  }
+}

+ 71 - 0
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());
+  }
+}

+ 71 - 0
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);
+}

Some files were not shown because too many files changed in this diff